[VOL-2235] Mocks and interfaces for rw-core

This update consists of mocks that are used by the rw-core
during unit testing.  It also includes interfaces used for unit
tests.

Change-Id: I20ca1455c358113c3aa897acc6355e0ddbc614b7
diff --git a/go.mod b/go.mod
index a4c8b07..91f0efe 100644
--- a/go.mod
+++ b/go.mod
@@ -8,9 +8,10 @@
 	github.com/golang/protobuf v1.3.2
 	github.com/google/uuid v1.1.1
 	github.com/gyuho/goraph v0.0.0-20160328020532-d460590d53a9
-	github.com/opencord/voltha-lib-go/v2 v2.2.12
+	github.com/opencord/voltha-lib-go/v2 v2.2.13
 	github.com/opencord/voltha-protos/v2 v2.0.1
 	github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2
 	github.com/stretchr/testify v1.4.0
+	google.golang.org/appengine v1.4.0 // indirect
 	google.golang.org/grpc v1.24.0
 )
diff --git a/go.sum b/go.sum
index 6a5fe52..130c6bc 100644
--- a/go.sum
+++ b/go.sum
@@ -192,8 +192,8 @@
 github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
 github.com/onsi/gomega v1.4.2 h1:3mYCb7aPxS/RU7TI1y4rkEn1oKmPRjNJLNEXgw7MH2I=
 github.com/onsi/gomega v1.4.2/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
-github.com/opencord/voltha-lib-go/v2 v2.2.12 h1:QymMvOZwrnW3caPmeUycNqMGZceNC8x3AC5exF5IO4E=
-github.com/opencord/voltha-lib-go/v2 v2.2.12/go.mod h1:Hql0xWiBFqYM6WpE5G+w9//NdaIoR9mVzcvVYDxEnZY=
+github.com/opencord/voltha-lib-go/v2 v2.2.13 h1:8TxMjhqOL2vcDxO5uIaAd6Lj2Ahq/gAFiNNoUkM90cQ=
+github.com/opencord/voltha-lib-go/v2 v2.2.13/go.mod h1:g8WH4WTOJ0f40ZoqYFR4nyOLIAC84dOKDXsbT1ZErY4=
 github.com/opencord/voltha-protos/v2 v2.0.1 h1:vcE0XxNVeu0SED0bW2lf2w24k/QMFrFqMexuedIyTEg=
 github.com/opencord/voltha-protos/v2 v2.0.1/go.mod h1:6kOcfYi1CadWowFxI2SH5wLfHrsRECZLZlD2MFK6WDI=
 github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc=
diff --git a/rw_core/core/adapter_proxy.go b/rw_core/core/adapter_proxy.go
index 3143b60..b986a0e 100755
--- a/rw_core/core/adapter_proxy.go
+++ b/rw_core/core/adapter_proxy.go
@@ -101,7 +101,7 @@
 	// Use a device specific topic as we are the only core handling requests for this device
 	//replyToTopic := kafka.CreateSubTopic(ap.kafkaICProxy.DefaultTopic.Name, device.Id)
 	replyToTopic := ap.getCoreTopic()
-	success, result := ap.kafkaICProxy.InvokeRPC(nil, rpc, &toTopic, &replyToTopic, true, device.Id, args...)
+	success, result := ap.kafkaICProxy.InvokeRPC(ctx, rpc, &toTopic, &replyToTopic, true, device.Id, args...)
 	log.Debugw("DisableDevice-response", log.Fields{"deviceId": device.Id, "success": success})
 	return unPackResponse(rpc, device.Id, success, result)
 }
diff --git a/rw_core/core/adapter_proxy_test.go b/rw_core/core/adapter_proxy_test.go
new file mode 100755
index 0000000..e6e0ecb
--- /dev/null
+++ b/rw_core/core/adapter_proxy_test.go
@@ -0,0 +1,216 @@
+/*
+ * Copyright 2019-present Open Networking Foundation
+ *
+ * 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 core
+
+import (
+	"context"
+	"crypto/rand"
+	cm "github.com/opencord/voltha-go/rw_core/mocks"
+	com "github.com/opencord/voltha-lib-go/v2/pkg/adapters/common"
+	"github.com/opencord/voltha-lib-go/v2/pkg/kafka"
+	"github.com/opencord/voltha-lib-go/v2/pkg/log"
+	lm "github.com/opencord/voltha-lib-go/v2/pkg/mocks"
+	of "github.com/opencord/voltha-protos/v2/go/openflow_13"
+	"github.com/opencord/voltha-protos/v2/go/voltha"
+	"github.com/stretchr/testify/assert"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+	"testing"
+	"time"
+)
+
+const (
+	coreName       = "rw_core"
+	adapterName    = "adapter_mock"
+	coreInstanceId = "1000"
+)
+
+var (
+	coreKafkaICProxy    *kafka.InterContainerProxy
+	adapterKafkaICProxy *kafka.InterContainerProxy
+	kc                  kafka.Client
+	adapterReqHandler   *com.RequestHandlerProxy
+	adapter             *cm.Adapter
+)
+
+func init() {
+	if _, err := log.SetDefaultLogger(log.JSON, 0, log.Fields{"instanceId": coreInstanceId}); err != nil {
+		log.With(log.Fields{"error": err}).Fatal("Cannot setup logging")
+	}
+	// Set the log level to Warning
+	log.SetAllLogLevel(2)
+
+	var err error
+
+	// Create the KV client
+	kc = lm.NewKafkaClient()
+
+	// Setup core inter-container proxy and core request handler
+	if coreKafkaICProxy, err = kafka.NewInterContainerProxy(
+		kafka.MsgClient(kc),
+		kafka.DefaultTopic(&kafka.Topic{Name: coreName})); err != nil || coreKafkaICProxy == nil {
+		log.Fatalw("Failure-creating-core-intercontainerProxy", log.Fields{"error": err})
+
+	}
+	if err := coreKafkaICProxy.Start(); err != nil {
+		log.Fatalw("Failure-starting-core-kafka-intercontainerProxy", log.Fields{"error": err})
+	}
+	if err := coreKafkaICProxy.SubscribeWithDefaultRequestHandler(kafka.Topic{Name: coreName}, 0); err != nil {
+		log.Fatalw("Failure-subscribing-core-request-handler", log.Fields{"error": err})
+	}
+
+	// Setup adapter inter-container proxy and adapter request handler
+	adapterCoreProxy := com.NewCoreProxy(nil, adapterName, coreName)
+	adapter = cm.NewAdapter(adapterCoreProxy)
+	adapterReqHandler = com.NewRequestHandlerProxy(coreInstanceId, adapter, adapterCoreProxy)
+	if adapterKafkaICProxy, err = kafka.NewInterContainerProxy(
+		kafka.MsgClient(kc),
+		kafka.DefaultTopic(&kafka.Topic{Name: adapterName}),
+		kafka.RequestHandlerInterface(adapterReqHandler)); err != nil || adapterKafkaICProxy == nil {
+		log.Fatalw("Failure-creating-adapter-intercontainerProxy", log.Fields{"error": err})
+	}
+	if err = adapterKafkaICProxy.Start(); err != nil {
+		log.Fatalw("Failure-starting-adapter-kafka-intercontainerProxy", log.Fields{"error": err})
+	}
+	if err = adapterKafkaICProxy.SubscribeWithDefaultRequestHandler(kafka.Topic{Name: adapterName}, 0); err != nil {
+		log.Fatalw("Failure-subscribing-adapter-request-handler", log.Fields{"error": err})
+	}
+}
+
+func getRandomBytes(size int) (bytes []byte, err error) {
+	bytes = make([]byte, size)
+	_, err = rand.Read(bytes)
+	return
+}
+
+func TestCreateAdapterProxy(t *testing.T) {
+	ap := NewAdapterProxy(coreKafkaICProxy, coreName)
+	assert.NotNil(t, ap)
+}
+
+func testSimpleRequests(t *testing.T) {
+	type simpleRequest func(context.Context, *voltha.Device) error
+	ap := NewAdapterProxy(coreKafkaICProxy, coreName)
+	simpleRequests := []simpleRequest{
+		ap.AdoptDevice,
+		ap.DisableDevice,
+		ap.RebootDevice,
+		ap.DeleteDevice,
+		ap.ReconcileDevice,
+		ap.ReEnableDevice,
+	}
+	for _, f := range simpleRequests {
+		//Success
+		d := &voltha.Device{Id: "deviceId", Adapter: adapterName}
+		ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
+		err := f(ctx, d)
+		cancel()
+		assert.Nil(t, err)
+
+		//	Failure - invalid adapter
+		expectedError := status.Error(codes.Canceled, "context deadline exceeded")
+		d = &voltha.Device{Id: "deviceId", Adapter: "adapter_mock_1"}
+		ctx, cancel = context.WithTimeout(context.Background(), 20*time.Millisecond)
+		err = f(ctx, d)
+		cancel()
+		assert.NotNil(t, err)
+		assert.Equal(t, expectedError.Error(), err.Error())
+
+		// Failure - short timeout
+		expectedError = status.Error(codes.Canceled, "context deadline exceeded")
+		d = &voltha.Device{Id: "deviceId", Adapter: adapterName}
+		ctx, cancel = context.WithTimeout(context.Background(), 100*time.Nanosecond)
+		err = f(ctx, d)
+		cancel()
+		assert.NotNil(t, err)
+		assert.Equal(t, expectedError.Error(), err.Error())
+	}
+}
+
+func testGetSwitchCapabilityFromAdapter(t *testing.T) {
+	ap := NewAdapterProxy(coreKafkaICProxy, coreName)
+	d := &voltha.Device{Id: "deviceId", Adapter: adapterName}
+	ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second)
+	switchCap, err := ap.GetOfpDeviceInfo(ctx, d)
+	cancel()
+	assert.Nil(t, err)
+	assert.NotNil(t, switchCap)
+	expectedCap, _ := adapter.Get_ofp_device_info(d)
+	assert.Equal(t, switchCap.String(), expectedCap.String())
+}
+
+func testGetPortInfoFromAdapter(t *testing.T) {
+	ap := NewAdapterProxy(coreKafkaICProxy, coreName)
+	d := &voltha.Device{Id: "deviceId", Adapter: adapterName}
+	ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second)
+	portNo := uint32(1)
+	portInfo, err := ap.GetOfpPortInfo(ctx, d, portNo)
+	cancel()
+	assert.Nil(t, err)
+	assert.NotNil(t, portInfo)
+	expectedPortInfo, _ := adapter.Get_ofp_port_info(d, int64(portNo))
+	assert.Equal(t, portInfo.String(), expectedPortInfo.String())
+}
+
+func testPacketOut(t *testing.T) {
+	ap := NewAdapterProxy(coreKafkaICProxy, coreName)
+	d := &voltha.Device{Id: "deviceId", Adapter: adapterName}
+	outPort := uint32(1)
+	packet, err := getRandomBytes(50)
+	assert.Nil(t, err)
+	err = ap.packetOut(adapterName, d.Id, outPort, &of.OfpPacketOut{Data: packet})
+	assert.Nil(t, err)
+}
+
+func testFlowUpdates(t *testing.T) {
+	ap := NewAdapterProxy(coreKafkaICProxy, coreName)
+	d := &voltha.Device{Id: "deviceId", Adapter: adapterName}
+	err := ap.UpdateFlowsBulk(d, &voltha.Flows{}, &voltha.FlowGroups{}, &voltha.FlowMetadata{})
+	assert.Nil(t, err)
+	flowChanges := &voltha.FlowChanges{ToAdd: &voltha.Flows{Items: nil}, ToRemove: &voltha.Flows{Items: nil}}
+	groupChanges := &voltha.FlowGroupChanges{ToAdd: &voltha.FlowGroups{Items: nil}, ToRemove: &voltha.FlowGroups{Items: nil}, ToUpdate: &voltha.FlowGroups{Items: nil}}
+	err = ap.UpdateFlowsIncremental(d, flowChanges, groupChanges, &voltha.FlowMetadata{})
+	assert.Nil(t, err)
+}
+
+func testPmUpdates(t *testing.T) {
+	ap := NewAdapterProxy(coreKafkaICProxy, coreName)
+	d := &voltha.Device{Id: "deviceId", Adapter: adapterName}
+	ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second)
+	err := ap.UpdatePmConfigs(ctx, d, &voltha.PmConfigs{})
+	cancel()
+	assert.Nil(t, err)
+}
+
+func TestSuite(t *testing.T) {
+	//1. Test the simple requests first
+	testSimpleRequests(t)
+
+	//2.  Test get switch capability
+	testGetSwitchCapabilityFromAdapter(t)
+
+	//3.  Test get port info
+	testGetPortInfoFromAdapter(t)
+
+	//4. Test PacketOut
+	testPacketOut(t)
+
+	//	5. Test flow updates
+	testFlowUpdates(t)
+
+	//6. Pm configs
+	testPmUpdates(t)
+}
diff --git a/rw_core/core/common_test.go b/rw_core/core/common_test.go
new file mode 100644
index 0000000..6642a36
--- /dev/null
+++ b/rw_core/core/common_test.go
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2019-present Open Networking Foundation
+ *
+ * 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 core
+
+import (
+	"github.com/opencord/voltha-lib-go/v2/pkg/log"
+)
+
+const (
+	logLevel = log.FatalLevel
+)
+
+// Unit test initialization. This init() function handles all unit tests in
+// the current directory.
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	_, err := log.AddPackage(log.JSON, logLevel, log.Fields{"instanceId": "mocks"})
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/rw_core/core/device_state_transitions_test.go b/rw_core/core/device_state_transitions_test.go
index 33171ee..0e7b4b3 100644
--- a/rw_core/core/device_state_transitions_test.go
+++ b/rw_core/core/device_state_transitions_test.go
@@ -17,6 +17,7 @@
 
 import (
 	"github.com/opencord/voltha-go/rw_core/coreIf"
+	"github.com/opencord/voltha-go/rw_core/mocks"
 	"github.com/opencord/voltha-lib-go/v2/pkg/log"
 	"github.com/opencord/voltha-protos/v2/go/voltha"
 	"github.com/stretchr/testify/assert"
@@ -28,60 +29,17 @@
 var tdm coreIf.DeviceManager
 
 type testDeviceManager struct {
+	mocks.DeviceManager
 }
 
 func newTestDeviceManager() *testDeviceManager {
 	return &testDeviceManager{}
 }
 
-func (tdm *testDeviceManager) GetDevice(string) (*voltha.Device, error) {
-	return nil, nil
-}
-
-func (tdm *testDeviceManager) IsRootDevice(string) (bool, error) {
-	return false, nil
-}
-
-func (tdm *testDeviceManager) NotifyInvalidTransition(pto *voltha.Device) error {
-	return nil
-}
-
-func (tdm *testDeviceManager) SetAdminStateToEnable(to *voltha.Device) error {
-	return nil
-}
-
-func (tdm *testDeviceManager) CreateLogicalDevice(to *voltha.Device) error {
-	return nil
-}
-
-func (tdm *testDeviceManager) SetupUNILogicalPorts(to *voltha.Device) error {
-	return nil
-}
-
-func (tdm *testDeviceManager) DisableAllChildDevices(to *voltha.Device) error {
-	return nil
-}
-
-func (tdm *testDeviceManager) DeleteLogicalDevice(to *voltha.Device) error {
-	return nil
-}
-
-func (tdm *testDeviceManager) DeleteLogicalPorts(to *voltha.Device) error {
-	return nil
-}
-
-func (tdm *testDeviceManager) DeleteAllChildDevices(to *voltha.Device) error {
-	return nil
-}
-
-func (tdm *testDeviceManager) RunPostDeviceDelete(to *voltha.Device) error {
-	return nil
-}
-
 func init() {
-	log.AddPackage(log.JSON, log.WarnLevel, nil)
-	//log.UpdateAllLoggers(log.Fields{"instanceId": "device-state-transition"})
-	//log.SetAllLogLevel(log.DebugLevel)
+	if _, err := log.AddPackage(log.JSON, log.WarnLevel, nil); err != nil {
+		log.Fatal("failure-adding-package-core")
+	}
 	tdm = newTestDeviceManager()
 	transitionMap = NewTransitionMap(tdm)
 }
diff --git a/rw_core/coreIf/adapter_manager_if.go b/rw_core/coreIf/adapter_manager_if.go
new file mode 100644
index 0000000..b48852c
--- /dev/null
+++ b/rw_core/coreIf/adapter_manager_if.go
@@ -0,0 +1,28 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+
+ * 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 coreIf
+
+import (
+	"context"
+	"github.com/opencord/voltha-protos/v2/go/voltha"
+)
+
+type AdapterManager interface {
+	ListAdapters(ctx context.Context) (*voltha.Adapters, error)
+	GetAdapterName(deviceType string) (string, error)
+	GetDeviceType(deviceType string) *voltha.DeviceType
+	RegisterAdapter(adapter *voltha.Adapter, deviceTypes *voltha.DeviceTypes) *voltha.CoreInstance
+}
diff --git a/rw_core/coreIf/core_if.go b/rw_core/coreIf/core_if.go
new file mode 100644
index 0000000..bf9a661
--- /dev/null
+++ b/rw_core/coreIf/core_if.go
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.
+ */
+/*
+Defines a DeviceManager Interface - Used for unit testing of the flow decomposer only at this
+time.
+*/
+package coreIf
+
+import (
+	"context"
+	"github.com/opencord/voltha-go/db/model"
+	"github.com/opencord/voltha-go/rw_core/config"
+	"github.com/opencord/voltha-lib-go/v2/pkg/kafka"
+)
+
+type Core interface {
+	Start(ctx context.Context)
+	Stop(ctx context.Context)
+	GetKafkaInterContainerProxy() *kafka.InterContainerProxy
+	GetConfig() *config.RWCoreFlags
+	GetInstanceId() string
+	GetClusterDataProxy() *model.Proxy
+	GetAdapterManager() AdapterManager
+	StartGRPCService(ctx context.Context)
+	GetDeviceManager() DeviceManager
+	GetLogicalDeviceManager() LogicalDeviceManager
+	GetDeviceOwnerShip() DeviceOwnership
+}
diff --git a/rw_core/coreIf/device_ownership_if.go b/rw_core/coreIf/device_ownership_if.go
new file mode 100644
index 0000000..fb19097
--- /dev/null
+++ b/rw_core/coreIf/device_ownership_if.go
@@ -0,0 +1,28 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+
+ * 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 coreIf
+
+import (
+	"context"
+)
+
+type DeviceOwnership interface {
+	Start(ctx context.Context)
+	Stop(ctx context.Context)
+	OwnedByMe(id interface{}) (bool, error)
+	AbandonDevice(id string) error
+	GetAllDeviceIdsOwnedByMe() []string
+}
diff --git a/rw_core/coreIf/logical_device_manager_if.go b/rw_core/coreIf/logical_device_manager_if.go
new file mode 100644
index 0000000..a10a3c8
--- /dev/null
+++ b/rw_core/coreIf/logical_device_manager_if.go
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.
+ */
+/*
+Defines a DeviceManager Interface - Used for unit testing of the flow decomposer only at this
+time.
+*/
+package coreIf
+
+import (
+	"context"
+	"github.com/opencord/voltha-protos/v2/go/openflow_13"
+	"github.com/opencord/voltha-protos/v2/go/voltha"
+)
+
+type LogicalDeviceManager interface {
+	GetLogicalPort(lPortId *voltha.LogicalPortId) (*voltha.LogicalPort, error)
+	EnableLogicalPort(ctx context.Context, id *voltha.LogicalPortId, ch chan interface{})
+	DisableLogicalPort(ctx context.Context, id *voltha.LogicalPortId, ch chan interface{})
+	UpdateFlowTable(ctx context.Context, id string, flow *openflow_13.OfpFlowMod, ch chan interface{})
+	UpdateMeterTable(ctx context.Context, id string, meter *openflow_13.OfpMeterMod, ch chan interface{})
+	UpdateGroupTable(ctx context.Context, id string, groupMod *openflow_13.OfpGroupMod, ch chan interface{})
+	GetLogicalDevice(id string) (*voltha.LogicalDevice, error)
+	ListManagedLogicalDevices() (*voltha.LogicalDevices, error)
+	ListLogicalDevices() (*voltha.LogicalDevices, error)
+	ListLogicalDeviceFlows(ctx context.Context, id string) (*openflow_13.Flows, error)
+	ListLogicalDeviceFlowGroups(ctx context.Context, id string) (*openflow_13.FlowGroups, error)
+	ListLogicalDevicePorts(ctx context.Context, id string) (*voltha.LogicalPorts, error)
+	ListLogicalDeviceMeters(ctx context.Context, id string) (*openflow_13.Meters, error)
+	PacketOut(packet *openflow_13.PacketOut) error
+}
diff --git a/rw_core/flow_decomposition/flow_decomposer_test.go b/rw_core/flow_decomposition/flow_decomposer_test.go
index f21f589..a24765e 100644
--- a/rw_core/flow_decomposition/flow_decomposer_test.go
+++ b/rw_core/flow_decomposition/flow_decomposer_test.go
@@ -18,6 +18,7 @@
 import (
 	"errors"
 	"github.com/opencord/voltha-go/rw_core/graph"
+	"github.com/opencord/voltha-go/rw_core/mocks"
 	fu "github.com/opencord/voltha-lib-go/v2/pkg/flows"
 	"github.com/opencord/voltha-lib-go/v2/pkg/log"
 	ofp "github.com/opencord/voltha-protos/v2/go/openflow_13"
@@ -43,6 +44,7 @@
 }
 
 type testDeviceManager struct {
+	mocks.DeviceManager
 	devices map[string]*voltha.Device
 }
 
@@ -110,42 +112,6 @@
 	return false, errors.New("ABSENT.")
 }
 
-func (tdm *testDeviceManager) NotifyInvalidTransition(pcDevice *voltha.Device) error {
-	return nil
-}
-
-func (tdm *testDeviceManager) SetAdminStateToEnable(cDevice *voltha.Device) error {
-	return nil
-}
-
-func (tdm *testDeviceManager) CreateLogicalDevice(cDevice *voltha.Device) error {
-	return nil
-}
-
-func (tdm *testDeviceManager) SetupUNILogicalPorts(cDevice *voltha.Device) error {
-	return nil
-}
-
-func (tdm *testDeviceManager) DisableAllChildDevices(cDevice *voltha.Device) error {
-	return nil
-}
-
-func (tdm *testDeviceManager) DeleteLogicalDevice(cDevice *voltha.Device) error {
-	return nil
-}
-
-func (tdm *testDeviceManager) DeleteLogicalPorts(cDevice *voltha.Device) error {
-	return nil
-}
-
-func (tdm *testDeviceManager) DeleteAllChildDevices(cDevice *voltha.Device) error {
-	return nil
-}
-
-func (tdm *testDeviceManager) RunPostDeviceDelete(cDevice *voltha.Device) error {
-	return nil
-}
-
 type testFlowDecomposer struct {
 	dMgr         *testDeviceManager
 	logicalPorts map[uint32]*voltha.LogicalPort
diff --git a/rw_core/mocks/adapter.go b/rw_core/mocks/adapter.go
new file mode 100644
index 0000000..a510e58
--- /dev/null
+++ b/rw_core/mocks/adapter.go
@@ -0,0 +1,211 @@
+/*
+ * Copyright 2019-present Open Networking Foundation
+
+ * 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 mocks
+
+import (
+	"github.com/opencord/voltha-lib-go/v2/pkg/adapters/adapterif"
+	ic "github.com/opencord/voltha-protos/v2/go/inter_container"
+	of "github.com/opencord/voltha-protos/v2/go/openflow_13"
+	"github.com/opencord/voltha-protos/v2/go/voltha"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+	"strconv"
+	"strings"
+	"sync"
+)
+
+func macAddressToUint32Array(mac string) []uint32 {
+	slist := strings.Split(mac, ":")
+	result := make([]uint32, len(slist))
+	var err error
+	var tmp int64
+	for index, val := range slist {
+		if tmp, err = strconv.ParseInt(val, 16, 32); err != nil {
+			return []uint32{1, 2, 3, 4, 5, 6}
+		}
+		result[index] = uint32(tmp)
+	}
+	return result
+}
+
+type Adapter struct {
+	coreProxy adapterif.CoreProxy
+	devices   sync.Map
+}
+
+func NewAdapter(cp adapterif.CoreProxy) *Adapter {
+	return &Adapter{
+		coreProxy: cp,
+	}
+}
+
+func (ta *Adapter) storeDevice(d *voltha.Device) {
+	if d != nil {
+		ta.devices.Store(d.Id, d)
+	}
+}
+
+func (ta *Adapter) deleteDevice(id string) {
+	ta.devices.Delete(id)
+}
+
+func (ta *Adapter) getDevice(id string) *voltha.Device {
+	if val, ok := ta.devices.Load(id); ok && val != nil {
+		if device, ok := val.(*voltha.Device); ok {
+			return device
+		}
+	}
+	return nil
+}
+
+func (ta *Adapter) updateDevice(d *voltha.Device) error {
+	if d != nil {
+		if _, ok := ta.devices.LoadOrStore(d.Id, d); !ok {
+			return status.Errorf(codes.Internal, "error updating device %s", d.Id)
+		}
+	}
+	return nil
+}
+
+func (ta *Adapter) Adapter_descriptor() error {
+	return nil
+}
+func (ta *Adapter) Device_types() (*voltha.DeviceTypes, error) {
+	return nil, nil
+}
+func (ta *Adapter) Health() (*voltha.HealthStatus, error) {
+	return nil, nil
+}
+func (ta *Adapter) Adopt_device(device *voltha.Device) error {
+	return nil
+}
+
+func (ta *Adapter) Reconcile_device(device *voltha.Device) error {
+	return nil
+}
+
+func (ta *Adapter) Abandon_device(device *voltha.Device) error {
+	return nil
+}
+
+func (ta *Adapter) Disable_device(device *voltha.Device) error {
+	return nil
+}
+
+func (ta *Adapter) Reenable_device(device *voltha.Device) error {
+	return nil
+}
+
+func (ta *Adapter) Reboot_device(device *voltha.Device) error {
+	return nil
+}
+
+func (ta *Adapter) Self_test_device(device *voltha.Device) error {
+	return nil
+}
+
+func (ta *Adapter) Delete_device(device *voltha.Device) error {
+	return nil
+}
+
+func (ta *Adapter) Get_device_details(device *voltha.Device) error {
+	return nil
+}
+
+func (ta *Adapter) Update_flows_bulk(device *voltha.Device, flows *voltha.Flows, groups *voltha.FlowGroups, flowMetadata *voltha.FlowMetadata) error {
+	return nil
+}
+
+func (ta *Adapter) Update_flows_incrementally(device *voltha.Device, flows *of.FlowChanges, groups *of.FlowGroupChanges, flowMetadata *voltha.FlowMetadata) error {
+	return nil
+}
+func (ta *Adapter) Update_pm_config(device *voltha.Device, pm_configs *voltha.PmConfigs) error {
+	return nil
+}
+
+func (ta *Adapter) Receive_packet_out(deviceId string, egress_port_no int, msg *of.OfpPacketOut) error {
+	return nil
+}
+
+func (ta *Adapter) Suppress_alarm(filter *voltha.AlarmFilter) error {
+	return nil
+}
+
+func (ta *Adapter) Unsuppress_alarm(filter *voltha.AlarmFilter) error {
+	return nil
+}
+
+func (ta *Adapter) Get_ofp_device_info(device *voltha.Device) (*ic.SwitchCapability, error) {
+	return &ic.SwitchCapability{
+		Desc: &of.OfpDesc{
+			HwDesc:    "adapter_mock",
+			SwDesc:    "adapter_mock",
+			SerialNum: "000000000",
+		},
+		SwitchFeatures: &of.OfpSwitchFeatures{
+			NBuffers: 256,
+			NTables:  2,
+			Capabilities: uint32(of.OfpCapabilities_OFPC_FLOW_STATS |
+				of.OfpCapabilities_OFPC_TABLE_STATS |
+				of.OfpCapabilities_OFPC_PORT_STATS |
+				of.OfpCapabilities_OFPC_GROUP_STATS),
+		},
+	}, nil
+}
+
+func (ta *Adapter) Get_ofp_port_info(device *voltha.Device, port_no int64) (*ic.PortCapability, error) {
+	capability := uint32(of.OfpPortFeatures_OFPPF_1GB_FD | of.OfpPortFeatures_OFPPF_FIBER)
+	return &ic.PortCapability{
+		Port: &voltha.LogicalPort{
+			OfpPort: &of.OfpPort{
+				HwAddr:     macAddressToUint32Array("11:11:33:44:55:66"),
+				Config:     0,
+				State:      uint32(of.OfpPortState_OFPPS_LIVE),
+				Curr:       capability,
+				Advertised: capability,
+				Peer:       capability,
+				CurrSpeed:  uint32(of.OfpPortFeatures_OFPPF_1GB_FD),
+				MaxSpeed:   uint32(of.OfpPortFeatures_OFPPF_1GB_FD),
+			},
+			DeviceId:     device.Id,
+			DevicePortNo: uint32(port_no),
+		},
+	}, nil
+}
+
+func (ta *Adapter) Process_inter_adapter_message(msg *ic.InterAdapterMessage) error {
+	return nil
+}
+
+func (ta *Adapter) Download_image(device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
+	return nil, nil
+}
+
+func (ta *Adapter) Get_image_download_status(device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
+	return nil, nil
+}
+
+func (ta *Adapter) Cancel_image_download(device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
+	return nil, nil
+}
+
+func (ta *Adapter) Activate_image_update(device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
+	return nil, nil
+}
+
+func (ta *Adapter) Revert_image_update(device *voltha.Device, request *voltha.ImageDownload) (*voltha.ImageDownload, error) {
+	return nil, nil
+}
diff --git a/rw_core/mocks/adapter_olt.go b/rw_core/mocks/adapter_olt.go
new file mode 100644
index 0000000..89f6c20
--- /dev/null
+++ b/rw_core/mocks/adapter_olt.go
@@ -0,0 +1,221 @@
+/*
+ * Copyright 2019-present Open Networking Foundation
+
+ * 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 mocks
+
+import (
+	"context"
+	"fmt"
+	"github.com/gogo/protobuf/proto"
+	"github.com/opencord/voltha-lib-go/v2/pkg/adapters/adapterif"
+	com "github.com/opencord/voltha-lib-go/v2/pkg/adapters/common"
+	"github.com/opencord/voltha-lib-go/v2/pkg/log"
+	ic "github.com/opencord/voltha-protos/v2/go/inter_container"
+	of "github.com/opencord/voltha-protos/v2/go/openflow_13"
+	"github.com/opencord/voltha-protos/v2/go/voltha"
+	"strings"
+)
+
+const (
+	numONUPerOLT = 4
+)
+
+type OLTAdapter struct {
+	Adapter
+}
+
+func NewOLTAdapter(cp adapterif.CoreProxy) *OLTAdapter {
+	a := &OLTAdapter{}
+	a.coreProxy = cp
+	return a
+}
+
+func (oltA *OLTAdapter) Adopt_device(device *voltha.Device) error {
+	go func() {
+		d := proto.Clone(device).(*voltha.Device)
+		d.Root = true
+		d.Vendor = "olt_adapter_mock"
+		d.Model = "go-mock"
+		d.SerialNumber = com.GetRandomSerialNumber()
+		d.MacAddress = strings.ToUpper(com.GetRandomMacAddress())
+		oltA.storeDevice(d)
+		if res := oltA.coreProxy.DeviceUpdate(context.TODO(), d); res != nil {
+			log.Fatalf("deviceUpdate-failed-%s", res)
+		}
+		nniPort := &voltha.Port{
+			PortNo:     2,
+			Label:      fmt.Sprintf("nni-%d", 2),
+			Type:       voltha.Port_ETHERNET_NNI,
+			OperStatus: voltha.OperStatus_ACTIVE,
+		}
+		var err error
+		if err = oltA.coreProxy.PortCreated(context.TODO(), d.Id, nniPort); err != nil {
+			log.Fatalf("PortCreated-failed-%s", err)
+		}
+
+		ponPort := &voltha.Port{
+			PortNo:     1,
+			Label:      fmt.Sprintf("pon-%d", 1),
+			Type:       voltha.Port_PON_OLT,
+			OperStatus: voltha.OperStatus_ACTIVE,
+		}
+		if err = oltA.coreProxy.PortCreated(context.TODO(), d.Id, ponPort); err != nil {
+			log.Fatalf("PortCreated-failed-%s", err)
+		}
+
+		d.ConnectStatus = voltha.ConnectStatus_REACHABLE
+		d.OperStatus = voltha.OperStatus_ACTIVE
+
+		if err = oltA.coreProxy.DeviceStateUpdate(context.TODO(), d.Id, d.ConnectStatus, d.OperStatus); err != nil {
+			log.Fatalf("Device-state-update-failed-%s", err)
+		}
+
+		//Get the latest device data from the Core
+		if d, err = oltA.coreProxy.GetDevice(context.TODO(), d.Id, d.Id); err != nil {
+			log.Fatalf("getting-device-failed-%s", err)
+		}
+
+		if err = oltA.updateDevice(d); err != nil {
+			log.Fatalf("saving-device-failed-%s", err)
+		}
+
+		// Register Child devices
+		initialUniPortNo := 100
+		for i := 0; i < numONUPerOLT; i++ {
+			go func(seqNo int) {
+				if _, err := oltA.coreProxy.ChildDeviceDetected(
+					context.TODO(),
+					d.Id,
+					1,
+					"onu_adapter_mock",
+					initialUniPortNo+seqNo,
+					"onu_adapter_mock",
+					com.GetRandomSerialNumber(),
+					int64(seqNo)); err != nil {
+					log.Fatalf("failure-sending-child-device-%s", err)
+				}
+			}(i)
+		}
+	}()
+	return nil
+}
+
+func (oltA *OLTAdapter) Get_ofp_device_info(device *voltha.Device) (*ic.SwitchCapability, error) {
+	if d := oltA.getDevice(device.Id); d == nil {
+		log.Fatalf("device-not-found-%s", device.Id)
+	}
+	return &ic.SwitchCapability{
+		Desc: &of.OfpDesc{
+			HwDesc:    "olt_adapter_mock",
+			SwDesc:    "olt_adapter_mock",
+			SerialNum: "12345678",
+		},
+		SwitchFeatures: &of.OfpSwitchFeatures{
+			NBuffers: 256,
+			NTables:  2,
+			Capabilities: uint32(of.OfpCapabilities_OFPC_FLOW_STATS |
+				of.OfpCapabilities_OFPC_TABLE_STATS |
+				of.OfpCapabilities_OFPC_PORT_STATS |
+				of.OfpCapabilities_OFPC_GROUP_STATS),
+		},
+	}, nil
+}
+
+func (oltA *OLTAdapter) Get_ofp_port_info(device *voltha.Device, port_no int64) (*ic.PortCapability, error) {
+	if d := oltA.getDevice(device.Id); d == nil {
+		log.Fatalf("device-not-found-%s", device.Id)
+	}
+	capability := uint32(of.OfpPortFeatures_OFPPF_1GB_FD | of.OfpPortFeatures_OFPPF_FIBER)
+	return &ic.PortCapability{
+		Port: &voltha.LogicalPort{
+			OfpPort: &of.OfpPort{
+				HwAddr:     macAddressToUint32Array("11:22:33:44:55:66"),
+				Config:     0,
+				State:      uint32(of.OfpPortState_OFPPS_LIVE),
+				Curr:       capability,
+				Advertised: capability,
+				Peer:       capability,
+				CurrSpeed:  uint32(of.OfpPortFeatures_OFPPF_1GB_FD),
+				MaxSpeed:   uint32(of.OfpPortFeatures_OFPPF_1GB_FD),
+			},
+			DeviceId:     device.Id,
+			DevicePortNo: uint32(port_no),
+		},
+	}, nil
+}
+
+func (oltA *OLTAdapter) GetNumONUPerOLT() int {
+	return numONUPerOLT
+}
+
+func (oltA *OLTAdapter) Disable_device(device *voltha.Device) error {
+	go func() {
+		if d := oltA.getDevice(device.Id); d == nil {
+			log.Fatalf("device-not-found-%s", device.Id)
+		}
+
+		cloned := proto.Clone(device).(*voltha.Device)
+		// Update the all ports state on that device to disable
+		if err := oltA.coreProxy.PortsStateUpdate(context.TODO(), cloned.Id, voltha.OperStatus_UNKNOWN); err != nil {
+			log.Fatalf("updating-ports-failed", log.Fields{"deviceId": device.Id, "error": err})
+		}
+
+		//Update the device state
+		cloned.ConnectStatus = voltha.ConnectStatus_UNREACHABLE
+		cloned.OperStatus = voltha.OperStatus_UNKNOWN
+
+		if err := oltA.coreProxy.DeviceStateUpdate(context.TODO(), cloned.Id, cloned.ConnectStatus, cloned.OperStatus); err != nil {
+			log.Fatalf("device-state-update-failed", log.Fields{"deviceId": device.Id, "error": err})
+		}
+
+		if err := oltA.updateDevice(cloned); err != nil {
+			log.Fatalf("saving-device-failed-%s", err)
+		}
+
+		// Tell the Core that all child devices have been disabled (by default it's an action already taken by the Core
+		if err := oltA.coreProxy.ChildDevicesLost(context.TODO(), cloned.Id); err != nil {
+			log.Fatalf("lost-notif-of-child-devices-failed", log.Fields{"deviceId": device.Id, "error": err})
+		}
+	}()
+	return nil
+}
+
+func (oltA *OLTAdapter) Reenable_device(device *voltha.Device) error {
+	go func() {
+		if d := oltA.getDevice(device.Id); d == nil {
+			log.Fatalf("device-not-found-%s", device.Id)
+		}
+
+		cloned := proto.Clone(device).(*voltha.Device)
+		// Update the all ports state on that device to enable
+		if err := oltA.coreProxy.PortsStateUpdate(context.TODO(), cloned.Id, voltha.OperStatus_ACTIVE); err != nil {
+			log.Fatalf("updating-ports-failed", log.Fields{"deviceId": device.Id, "error": err})
+		}
+
+		//Update the device state
+		cloned.ConnectStatus = voltha.ConnectStatus_REACHABLE
+		cloned.OperStatus = voltha.OperStatus_ACTIVE
+
+		if err := oltA.coreProxy.DeviceStateUpdate(context.TODO(), cloned.Id, cloned.ConnectStatus, cloned.OperStatus); err != nil {
+			log.Fatalf("device-state-update-failed", log.Fields{"deviceId": device.Id, "error": err})
+		}
+
+		// Tell the Core that all child devices have been enabled
+		if err := oltA.coreProxy.ChildDevicesDetected(context.TODO(), cloned.Id); err != nil {
+			log.Fatalf("detection-notif-of-child-devices-failed", log.Fields{"deviceId": device.Id, "error": err})
+		}
+	}()
+	return nil
+}
diff --git a/rw_core/mocks/adapter_olt_test.go b/rw_core/mocks/adapter_olt_test.go
new file mode 100644
index 0000000..6d61e1f
--- /dev/null
+++ b/rw_core/mocks/adapter_olt_test.go
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2019-present Open Networking Foundation
+ *
+ * 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 mocks
+
+import (
+	"github.com/opencord/voltha-lib-go/v2/pkg/adapters"
+	"testing"
+)
+
+func TestOLTAdapterImplementsIAdapter(t *testing.T) {
+	adapter := NewOLTAdapter(nil)
+
+	if _, ok := interface{}(adapter).(adapters.IAdapter); !ok {
+		t.Error("OLT adapter does not implement voltha-lib-go/v2/pkg/adapters/IAdapter interface")
+	}
+}
diff --git a/rw_core/mocks/adapter_onu.go b/rw_core/mocks/adapter_onu.go
new file mode 100644
index 0000000..66a3538
--- /dev/null
+++ b/rw_core/mocks/adapter_onu.go
@@ -0,0 +1,185 @@
+/*
+ * Copyright 2019-present Open Networking Foundation
+
+ * 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 mocks
+
+import (
+	"context"
+	"fmt"
+	"github.com/gogo/protobuf/proto"
+	"github.com/opencord/voltha-lib-go/v2/pkg/adapters/adapterif"
+	com "github.com/opencord/voltha-lib-go/v2/pkg/adapters/common"
+	"github.com/opencord/voltha-lib-go/v2/pkg/log"
+	ic "github.com/opencord/voltha-protos/v2/go/inter_container"
+	of "github.com/opencord/voltha-protos/v2/go/openflow_13"
+	"github.com/opencord/voltha-protos/v2/go/voltha"
+	"strings"
+)
+
+type ONUAdapter struct {
+	coreProxy adapterif.CoreProxy
+	Adapter
+}
+
+func NewONUAdapter(cp adapterif.CoreProxy) *ONUAdapter {
+	a := &ONUAdapter{}
+	a.coreProxy = cp
+	return a
+}
+
+func (onuA *ONUAdapter) Adopt_device(device *voltha.Device) error {
+	go func() {
+		d := proto.Clone(device).(*voltha.Device)
+		d.Root = false
+		d.Vendor = "onu_adapter_mock"
+		d.Model = "go-mock"
+		d.SerialNumber = com.GetRandomSerialNumber()
+		d.MacAddress = strings.ToUpper(com.GetRandomMacAddress())
+		onuA.storeDevice(d)
+		if res := onuA.coreProxy.DeviceUpdate(context.TODO(), d); res != nil {
+			log.Fatalf("deviceUpdate-failed-%s", res)
+		}
+		d.ConnectStatus = voltha.ConnectStatus_REACHABLE
+		d.OperStatus = voltha.OperStatus_DISCOVERED
+
+		if err := onuA.coreProxy.DeviceStateUpdate(context.TODO(), d.Id, d.ConnectStatus, d.OperStatus); err != nil {
+			log.Fatalf("device-state-update-failed-%s", err)
+		}
+
+		uniPortNo := uint32(2)
+		if device.ProxyAddress != nil {
+			if device.ProxyAddress.ChannelId != 0 {
+				uniPortNo = device.ProxyAddress.ChannelId
+			}
+		}
+
+		uniPort := &voltha.Port{
+			PortNo:     uniPortNo,
+			Label:      fmt.Sprintf("uni-%d", uniPortNo),
+			Type:       voltha.Port_ETHERNET_UNI,
+			OperStatus: voltha.OperStatus_ACTIVE,
+		}
+		var err error
+		if err = onuA.coreProxy.PortCreated(context.TODO(), d.Id, uniPort); err != nil {
+			log.Fatalf("PortCreated-failed-%s", err)
+		}
+
+		ponPortNo := uint32(1)
+		if device.ParentPortNo != 0 {
+			ponPortNo = device.ParentPortNo
+		}
+
+		ponPort := &voltha.Port{
+			PortNo:     ponPortNo,
+			Label:      fmt.Sprintf("pon-%d", ponPortNo),
+			Type:       voltha.Port_PON_ONU,
+			OperStatus: voltha.OperStatus_ACTIVE,
+			Peers: []*voltha.Port_PeerPort{{DeviceId: d.ParentId, // Peer device  is OLT
+				PortNo: uniPortNo}}, // Peer port is UNI port
+		}
+		if err := onuA.coreProxy.PortCreated(context.TODO(), d.Id, ponPort); err != nil {
+			log.Fatalf("PortCreated-failed-%s", err)
+		}
+
+		d.ConnectStatus = voltha.ConnectStatus_REACHABLE
+		d.OperStatus = voltha.OperStatus_ACTIVE
+
+		if err = onuA.coreProxy.DeviceStateUpdate(context.TODO(), d.Id, d.ConnectStatus, d.OperStatus); err != nil {
+			log.Fatalf("device-state-update-failed-%s", err)
+		}
+		//Get the latest device data from the Core
+		if d, err = onuA.coreProxy.GetDevice(context.TODO(), d.Id, d.Id); err != nil {
+			log.Fatalf("getting-device-failed-%s", err)
+		}
+
+		if err = onuA.updateDevice(d); err != nil {
+			log.Fatalf("saving-device-failed-%s", err)
+		}
+	}()
+	return nil
+}
+
+func (onuA *ONUAdapter) Get_ofp_port_info(device *voltha.Device, port_no int64) (*ic.PortCapability, error) {
+	if d := onuA.getDevice(device.Id); d == nil {
+		log.Fatalf("device-not-found-%s", device.Id)
+	}
+	capability := uint32(of.OfpPortFeatures_OFPPF_1GB_FD | of.OfpPortFeatures_OFPPF_FIBER)
+	return &ic.PortCapability{
+		Port: &voltha.LogicalPort{
+			OfpPort: &of.OfpPort{
+				HwAddr:     macAddressToUint32Array("12:12:12:12:12:12"),
+				Config:     0,
+				State:      uint32(of.OfpPortState_OFPPS_LIVE),
+				Curr:       capability,
+				Advertised: capability,
+				Peer:       capability,
+				CurrSpeed:  uint32(of.OfpPortFeatures_OFPPF_1GB_FD),
+				MaxSpeed:   uint32(of.OfpPortFeatures_OFPPF_1GB_FD),
+			},
+			DeviceId:     device.Id,
+			DevicePortNo: uint32(port_no),
+		},
+	}, nil
+}
+
+func (onuA *ONUAdapter) Disable_device(device *voltha.Device) error {
+	go func() {
+		if d := onuA.getDevice(device.Id); d == nil {
+			log.Fatalf("device-not-found-%s", device.Id)
+		}
+		cloned := proto.Clone(device).(*voltha.Device)
+		// Update the all ports state on that device to disable
+		if err := onuA.coreProxy.PortsStateUpdate(context.TODO(), cloned.Id, voltha.OperStatus_UNKNOWN); err != nil {
+			log.Fatalf("updating-ports-failed", log.Fields{"deviceId": device.Id, "error": err})
+		}
+		//Update the device state
+		cloned.ConnectStatus = voltha.ConnectStatus_UNREACHABLE
+		cloned.OperStatus = voltha.OperStatus_UNKNOWN
+
+		if err := onuA.coreProxy.DeviceStateUpdate(context.TODO(), cloned.Id, cloned.ConnectStatus, cloned.OperStatus); err != nil {
+			log.Fatalf("device-state-update-failed", log.Fields{"deviceId": device.Id, "error": err})
+		}
+		if err := onuA.updateDevice(cloned); err != nil {
+			log.Fatalf("saving-device-failed-%s", err)
+		}
+	}()
+	return nil
+}
+
+func (onuA *ONUAdapter) Reenable_device(device *voltha.Device) error {
+	go func() {
+		if d := onuA.getDevice(device.Id); d == nil {
+			log.Fatalf("device-not-found-%s", device.Id)
+		}
+
+		cloned := proto.Clone(device).(*voltha.Device)
+		// Update the all ports state on that device to enable
+		if err := onuA.coreProxy.PortsStateUpdate(context.TODO(), cloned.Id, voltha.OperStatus_ACTIVE); err != nil {
+			log.Fatalf("updating-ports-failed", log.Fields{"deviceId": device.Id, "error": err})
+		}
+
+		//Update the device state
+		cloned.ConnectStatus = voltha.ConnectStatus_REACHABLE
+		cloned.OperStatus = voltha.OperStatus_ACTIVE
+
+		if err := onuA.coreProxy.DeviceStateUpdate(context.TODO(), cloned.Id, cloned.ConnectStatus, cloned.OperStatus); err != nil {
+			log.Fatalf("device-state-update-failed", log.Fields{"deviceId": device.Id, "error": err})
+		}
+		if err := onuA.updateDevice(cloned); err != nil {
+			log.Fatalf("saving-device-failed-%s", err)
+		}
+	}()
+	return nil
+}
diff --git a/rw_core/mocks/adapter_onu_test.go b/rw_core/mocks/adapter_onu_test.go
new file mode 100644
index 0000000..748815a
--- /dev/null
+++ b/rw_core/mocks/adapter_onu_test.go
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2019-present Open Networking Foundation
+ *
+ * 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 mocks
+
+import (
+	"github.com/opencord/voltha-lib-go/v2/pkg/adapters"
+	"testing"
+)
+
+func TestONUAdapterImplementsIAdapter(t *testing.T) {
+	adapter := NewONUAdapter(nil)
+
+	if _, ok := interface{}(adapter).(adapters.IAdapter); !ok {
+		t.Error("ONU adapter does not implement voltha-lib-go/v2/pkg/adapters/IAdapter interface")
+	}
+}
diff --git a/rw_core/mocks/adapter_test.go b/rw_core/mocks/adapter_test.go
new file mode 100644
index 0000000..bc18cb5
--- /dev/null
+++ b/rw_core/mocks/adapter_test.go
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2019-present Open Networking Foundation
+ *
+ * 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 mocks
+
+import (
+	"github.com/opencord/voltha-lib-go/v2/pkg/adapters"
+	"testing"
+)
+
+func TestAdapterImplementsIAdapter(t *testing.T) {
+	adapter := NewAdapter(nil)
+
+	if _, ok := interface{}(adapter).(adapters.IAdapter); !ok {
+		t.Error("adapter does not implement voltha-lib-go/v2/pkg/adapters/IAdapter interface")
+	}
+}
diff --git a/rw_core/mocks/common_test.go b/rw_core/mocks/common_test.go
new file mode 100644
index 0000000..1ff5700
--- /dev/null
+++ b/rw_core/mocks/common_test.go
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2019-present Open Networking Foundation
+ *
+ * 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 mocks
+
+import (
+	"github.com/opencord/voltha-lib-go/v2/pkg/log"
+)
+
+const (
+	logLevel = log.FatalLevel
+)
+
+// Unit test initialization. This init() function handles all unit tests in
+// the current directory.
+func init() {
+	// Setup this package so that it's log level can be modified at run time
+	_, err := log.AddPackage(log.JSON, logLevel, log.Fields{"instanceId": "mocks"})
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/rw_core/mocks/device_manager.go b/rw_core/mocks/device_manager.go
new file mode 100644
index 0000000..c8bf1ca
--- /dev/null
+++ b/rw_core/mocks/device_manager.go
@@ -0,0 +1,141 @@
+/*
+ * Copyright 2019-present Open Networking Foundation
+
+ * 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 mocks
+
+import (
+	"context"
+	"github.com/opencord/voltha-protos/v2/go/voltha"
+)
+
+type DeviceManager struct {
+}
+
+func (dm *DeviceManager) GetDevice(deviceId string) (*voltha.Device, error) {
+	return nil, nil
+}
+func (dm *DeviceManager) IsRootDevice(deviceId string) (bool, error) {
+	return false, nil
+}
+
+func (dm *DeviceManager) NotifyInvalidTransition(pcDevice *voltha.Device) error {
+	return nil
+}
+
+func (dm *DeviceManager) SetAdminStateToEnable(cDevice *voltha.Device) error {
+	return nil
+}
+
+func (dm *DeviceManager) CreateLogicalDevice(cDevice *voltha.Device) error {
+	return nil
+}
+
+func (dm *DeviceManager) SetupUNILogicalPorts(cDevice *voltha.Device) error {
+	return nil
+}
+
+func (dm *DeviceManager) DisableAllChildDevices(cDevice *voltha.Device) error {
+	return nil
+}
+
+func (dm *DeviceManager) DeleteLogicalDevice(cDevice *voltha.Device) error {
+	return nil
+}
+
+func (dm *DeviceManager) DeleteLogicalPorts(cDevice *voltha.Device) error {
+	return nil
+}
+
+func (dm *DeviceManager) DeleteAllChildDevices(cDevice *voltha.Device) error {
+	return nil
+}
+
+func (dm *DeviceManager) RunPostDeviceDelete(cDevice *voltha.Device) error {
+	return nil
+}
+
+func (dm *DeviceManager) ListDevices() (*voltha.Devices, error) {
+	return nil, nil
+}
+
+func (dm *DeviceManager) ListDeviceIds() (*voltha.IDs, error) {
+	return nil, nil
+}
+
+func (dm *DeviceManager) ReconcileDevices(ctx context.Context, ids *voltha.IDs, ch chan interface{}) {
+}
+
+func (dm *DeviceManager) CreateDevice(ctx context.Context, device *voltha.Device, ch chan interface{}) {
+}
+
+func (dm *DeviceManager) EnableDevice(ctx context.Context, id *voltha.ID, ch chan interface{}) {
+}
+
+func (dm *DeviceManager) DisableDevice(ctx context.Context, id *voltha.ID, ch chan interface{}) {
+}
+
+func (dm *DeviceManager) RebootDevice(ctx context.Context, id *voltha.ID, ch chan interface{}) {
+}
+
+func (dm *DeviceManager) DeleteDevice(ctx context.Context, id *voltha.ID, ch chan interface{}) {
+}
+
+func (dm *DeviceManager) StopManagingDevice(id string) {
+}
+
+func (dm *DeviceManager) DownloadImage(ctx context.Context, img *voltha.ImageDownload, ch chan interface{}) {
+}
+
+func (dm *DeviceManager) CancelImageDownload(ctx context.Context, img *voltha.ImageDownload, ch chan interface{}) {
+}
+
+func (dm *DeviceManager) ActivateImage(ctx context.Context, img *voltha.ImageDownload, ch chan interface{}) {
+}
+
+func (dm *DeviceManager) RevertImage(ctx context.Context, img *voltha.ImageDownload, ch chan interface{}) {
+}
+
+func (dm *DeviceManager) GetImageDownloadStatus(ctx context.Context, img *voltha.ImageDownload, ch chan interface{}) {
+}
+
+func (dm *DeviceManager) UpdateImageDownload(deviceId string, img *voltha.ImageDownload) error {
+	return nil
+}
+
+func (dm *DeviceManager) SimulateAlarm(ctx context.Context, simulatereq *voltha.SimulateAlarmRequest, ch chan interface{}) {
+}
+
+func (dm *DeviceManager) GetImageDownload(ctx context.Context, img *voltha.ImageDownload) (*voltha.ImageDownload, error) {
+	return nil, nil
+}
+
+func (dm *DeviceManager) ListImageDownloads(ctx context.Context, deviceId string) (*voltha.ImageDownloads, error) {
+	return nil, nil
+}
+
+func (dm *DeviceManager) UpdatePmConfigs(ctx context.Context, pmConfigs *voltha.PmConfigs, ch chan interface{}) {
+}
+
+func (dm *DeviceManager) ListPmConfigs(ctx context.Context, deviceId string) (*voltha.PmConfigs, error) {
+	return nil, nil
+}
+
+func (dm *DeviceManager) DeletePeerPorts(fromDeviceId string, deviceId string) error {
+	return nil
+}
+
+func (dm *DeviceManager) ProcessTransition(previous *voltha.Device, current *voltha.Device) error {
+	return nil
+}
diff --git a/rw_core/mocks/device_manager_test.go b/rw_core/mocks/device_manager_test.go
new file mode 100644
index 0000000..ab2226b
--- /dev/null
+++ b/rw_core/mocks/device_manager_test.go
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2019-present Open Networking Foundation
+ *
+ * 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 mocks
+
+import (
+	"github.com/opencord/voltha-go/rw_core/coreIf"
+	"testing"
+)
+
+func TestDeviceManagerImplementsDeviceManagerIf(t *testing.T) {
+	deviceMgr := &DeviceManager{}
+
+	if _, ok := interface{}(deviceMgr).(coreIf.DeviceManager); !ok {
+		t.Error("Device manager does not implement the coreIf.DeviceManager interface")
+	}
+}
diff --git a/vendor/github.com/beorn7/perks/LICENSE b/vendor/github.com/beorn7/perks/LICENSE
new file mode 100644
index 0000000..339177b
--- /dev/null
+++ b/vendor/github.com/beorn7/perks/LICENSE
@@ -0,0 +1,20 @@
+Copyright (C) 2013 Blake Mizerany
+
+Permission is hereby granted, free of charge, to any person obtaining
+a copy of this software and associated documentation files (the
+"Software"), to deal in the Software without restriction, including
+without limitation the rights to use, copy, modify, merge, publish,
+distribute, sublicense, and/or sell copies of the Software, and to
+permit persons to whom the Software is furnished to do so, subject to
+the following conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/vendor/github.com/beorn7/perks/quantile/exampledata.txt b/vendor/github.com/beorn7/perks/quantile/exampledata.txt
new file mode 100644
index 0000000..1602287
--- /dev/null
+++ b/vendor/github.com/beorn7/perks/quantile/exampledata.txt
@@ -0,0 +1,2388 @@
+8
+5
+26
+12
+5
+235
+13
+6
+28
+30
+3
+3
+3
+3
+5
+2
+33
+7
+2
+4
+7
+12
+14
+5
+8
+3
+10
+4
+5
+3
+6
+6
+209
+20
+3
+10
+14
+3
+4
+6
+8
+5
+11
+7
+3
+2
+3
+3
+212
+5
+222
+4
+10
+10
+5
+6
+3
+8
+3
+10
+254
+220
+2
+3
+5
+24
+5
+4
+222
+7
+3
+3
+223
+8
+15
+12
+14
+14
+3
+2
+2
+3
+13
+3
+11
+4
+4
+6
+5
+7
+13
+5
+3
+5
+2
+5
+3
+5
+2
+7
+15
+17
+14
+3
+6
+6
+3
+17
+5
+4
+7
+6
+4
+4
+8
+6
+8
+3
+9
+3
+6
+3
+4
+5
+3
+3
+660
+4
+6
+10
+3
+6
+3
+2
+5
+13
+2
+4
+4
+10
+4
+8
+4
+3
+7
+9
+9
+3
+10
+37
+3
+13
+4
+12
+3
+6
+10
+8
+5
+21
+2
+3
+8
+3
+2
+3
+3
+4
+12
+2
+4
+8
+8
+4
+3
+2
+20
+1
+6
+32
+2
+11
+6
+18
+3
+8
+11
+3
+212
+3
+4
+2
+6
+7
+12
+11
+3
+2
+16
+10
+6
+4
+6
+3
+2
+7
+3
+2
+2
+2
+2
+5
+6
+4
+3
+10
+3
+4
+6
+5
+3
+4
+4
+5
+6
+4
+3
+4
+4
+5
+7
+5
+5
+3
+2
+7
+2
+4
+12
+4
+5
+6
+2
+4
+4
+8
+4
+15
+13
+7
+16
+5
+3
+23
+5
+5
+7
+3
+2
+9
+8
+7
+5
+8
+11
+4
+10
+76
+4
+47
+4
+3
+2
+7
+4
+2
+3
+37
+10
+4
+2
+20
+5
+4
+4
+10
+10
+4
+3
+7
+23
+240
+7
+13
+5
+5
+3
+3
+2
+5
+4
+2
+8
+7
+19
+2
+23
+8
+7
+2
+5
+3
+8
+3
+8
+13
+5
+5
+5
+2
+3
+23
+4
+9
+8
+4
+3
+3
+5
+220
+2
+3
+4
+6
+14
+3
+53
+6
+2
+5
+18
+6
+3
+219
+6
+5
+2
+5
+3
+6
+5
+15
+4
+3
+17
+3
+2
+4
+7
+2
+3
+3
+4
+4
+3
+2
+664
+6
+3
+23
+5
+5
+16
+5
+8
+2
+4
+2
+24
+12
+3
+2
+3
+5
+8
+3
+5
+4
+3
+14
+3
+5
+8
+2
+3
+7
+9
+4
+2
+3
+6
+8
+4
+3
+4
+6
+5
+3
+3
+6
+3
+19
+4
+4
+6
+3
+6
+3
+5
+22
+5
+4
+4
+3
+8
+11
+4
+9
+7
+6
+13
+4
+4
+4
+6
+17
+9
+3
+3
+3
+4
+3
+221
+5
+11
+3
+4
+2
+12
+6
+3
+5
+7
+5
+7
+4
+9
+7
+14
+37
+19
+217
+16
+3
+5
+2
+2
+7
+19
+7
+6
+7
+4
+24
+5
+11
+4
+7
+7
+9
+13
+3
+4
+3
+6
+28
+4
+4
+5
+5
+2
+5
+6
+4
+4
+6
+10
+5
+4
+3
+2
+3
+3
+6
+5
+5
+4
+3
+2
+3
+7
+4
+6
+18
+16
+8
+16
+4
+5
+8
+6
+9
+13
+1545
+6
+215
+6
+5
+6
+3
+45
+31
+5
+2
+2
+4
+3
+3
+2
+5
+4
+3
+5
+7
+7
+4
+5
+8
+5
+4
+749
+2
+31
+9
+11
+2
+11
+5
+4
+4
+7
+9
+11
+4
+5
+4
+7
+3
+4
+6
+2
+15
+3
+4
+3
+4
+3
+5
+2
+13
+5
+5
+3
+3
+23
+4
+4
+5
+7
+4
+13
+2
+4
+3
+4
+2
+6
+2
+7
+3
+5
+5
+3
+29
+5
+4
+4
+3
+10
+2
+3
+79
+16
+6
+6
+7
+7
+3
+5
+5
+7
+4
+3
+7
+9
+5
+6
+5
+9
+6
+3
+6
+4
+17
+2
+10
+9
+3
+6
+2
+3
+21
+22
+5
+11
+4
+2
+17
+2
+224
+2
+14
+3
+4
+4
+2
+4
+4
+4
+4
+5
+3
+4
+4
+10
+2
+6
+3
+3
+5
+7
+2
+7
+5
+6
+3
+218
+2
+2
+5
+2
+6
+3
+5
+222
+14
+6
+33
+3
+2
+5
+3
+3
+3
+9
+5
+3
+3
+2
+7
+4
+3
+4
+3
+5
+6
+5
+26
+4
+13
+9
+7
+3
+221
+3
+3
+4
+4
+4
+4
+2
+18
+5
+3
+7
+9
+6
+8
+3
+10
+3
+11
+9
+5
+4
+17
+5
+5
+6
+6
+3
+2
+4
+12
+17
+6
+7
+218
+4
+2
+4
+10
+3
+5
+15
+3
+9
+4
+3
+3
+6
+29
+3
+3
+4
+5
+5
+3
+8
+5
+6
+6
+7
+5
+3
+5
+3
+29
+2
+31
+5
+15
+24
+16
+5
+207
+4
+3
+3
+2
+15
+4
+4
+13
+5
+5
+4
+6
+10
+2
+7
+8
+4
+6
+20
+5
+3
+4
+3
+12
+12
+5
+17
+7
+3
+3
+3
+6
+10
+3
+5
+25
+80
+4
+9
+3
+2
+11
+3
+3
+2
+3
+8
+7
+5
+5
+19
+5
+3
+3
+12
+11
+2
+6
+5
+5
+5
+3
+3
+3
+4
+209
+14
+3
+2
+5
+19
+4
+4
+3
+4
+14
+5
+6
+4
+13
+9
+7
+4
+7
+10
+2
+9
+5
+7
+2
+8
+4
+6
+5
+5
+222
+8
+7
+12
+5
+216
+3
+4
+4
+6
+3
+14
+8
+7
+13
+4
+3
+3
+3
+3
+17
+5
+4
+3
+33
+6
+6
+33
+7
+5
+3
+8
+7
+5
+2
+9
+4
+2
+233
+24
+7
+4
+8
+10
+3
+4
+15
+2
+16
+3
+3
+13
+12
+7
+5
+4
+207
+4
+2
+4
+27
+15
+2
+5
+2
+25
+6
+5
+5
+6
+13
+6
+18
+6
+4
+12
+225
+10
+7
+5
+2
+2
+11
+4
+14
+21
+8
+10
+3
+5
+4
+232
+2
+5
+5
+3
+7
+17
+11
+6
+6
+23
+4
+6
+3
+5
+4
+2
+17
+3
+6
+5
+8
+3
+2
+2
+14
+9
+4
+4
+2
+5
+5
+3
+7
+6
+12
+6
+10
+3
+6
+2
+2
+19
+5
+4
+4
+9
+2
+4
+13
+3
+5
+6
+3
+6
+5
+4
+9
+6
+3
+5
+7
+3
+6
+6
+4
+3
+10
+6
+3
+221
+3
+5
+3
+6
+4
+8
+5
+3
+6
+4
+4
+2
+54
+5
+6
+11
+3
+3
+4
+4
+4
+3
+7
+3
+11
+11
+7
+10
+6
+13
+223
+213
+15
+231
+7
+3
+7
+228
+2
+3
+4
+4
+5
+6
+7
+4
+13
+3
+4
+5
+3
+6
+4
+6
+7
+2
+4
+3
+4
+3
+3
+6
+3
+7
+3
+5
+18
+5
+6
+8
+10
+3
+3
+3
+2
+4
+2
+4
+4
+5
+6
+6
+4
+10
+13
+3
+12
+5
+12
+16
+8
+4
+19
+11
+2
+4
+5
+6
+8
+5
+6
+4
+18
+10
+4
+2
+216
+6
+6
+6
+2
+4
+12
+8
+3
+11
+5
+6
+14
+5
+3
+13
+4
+5
+4
+5
+3
+28
+6
+3
+7
+219
+3
+9
+7
+3
+10
+6
+3
+4
+19
+5
+7
+11
+6
+15
+19
+4
+13
+11
+3
+7
+5
+10
+2
+8
+11
+2
+6
+4
+6
+24
+6
+3
+3
+3
+3
+6
+18
+4
+11
+4
+2
+5
+10
+8
+3
+9
+5
+3
+4
+5
+6
+2
+5
+7
+4
+4
+14
+6
+4
+4
+5
+5
+7
+2
+4
+3
+7
+3
+3
+6
+4
+5
+4
+4
+4
+3
+3
+3
+3
+8
+14
+2
+3
+5
+3
+2
+4
+5
+3
+7
+3
+3
+18
+3
+4
+4
+5
+7
+3
+3
+3
+13
+5
+4
+8
+211
+5
+5
+3
+5
+2
+5
+4
+2
+655
+6
+3
+5
+11
+2
+5
+3
+12
+9
+15
+11
+5
+12
+217
+2
+6
+17
+3
+3
+207
+5
+5
+4
+5
+9
+3
+2
+8
+5
+4
+3
+2
+5
+12
+4
+14
+5
+4
+2
+13
+5
+8
+4
+225
+4
+3
+4
+5
+4
+3
+3
+6
+23
+9
+2
+6
+7
+233
+4
+4
+6
+18
+3
+4
+6
+3
+4
+4
+2
+3
+7
+4
+13
+227
+4
+3
+5
+4
+2
+12
+9
+17
+3
+7
+14
+6
+4
+5
+21
+4
+8
+9
+2
+9
+25
+16
+3
+6
+4
+7
+8
+5
+2
+3
+5
+4
+3
+3
+5
+3
+3
+3
+2
+3
+19
+2
+4
+3
+4
+2
+3
+4
+4
+2
+4
+3
+3
+3
+2
+6
+3
+17
+5
+6
+4
+3
+13
+5
+3
+3
+3
+4
+9
+4
+2
+14
+12
+4
+5
+24
+4
+3
+37
+12
+11
+21
+3
+4
+3
+13
+4
+2
+3
+15
+4
+11
+4
+4
+3
+8
+3
+4
+4
+12
+8
+5
+3
+3
+4
+2
+220
+3
+5
+223
+3
+3
+3
+10
+3
+15
+4
+241
+9
+7
+3
+6
+6
+23
+4
+13
+7
+3
+4
+7
+4
+9
+3
+3
+4
+10
+5
+5
+1
+5
+24
+2
+4
+5
+5
+6
+14
+3
+8
+2
+3
+5
+13
+13
+3
+5
+2
+3
+15
+3
+4
+2
+10
+4
+4
+4
+5
+5
+3
+5
+3
+4
+7
+4
+27
+3
+6
+4
+15
+3
+5
+6
+6
+5
+4
+8
+3
+9
+2
+6
+3
+4
+3
+7
+4
+18
+3
+11
+3
+3
+8
+9
+7
+24
+3
+219
+7
+10
+4
+5
+9
+12
+2
+5
+4
+4
+4
+3
+3
+19
+5
+8
+16
+8
+6
+22
+3
+23
+3
+242
+9
+4
+3
+3
+5
+7
+3
+3
+5
+8
+3
+7
+5
+14
+8
+10
+3
+4
+3
+7
+4
+6
+7
+4
+10
+4
+3
+11
+3
+7
+10
+3
+13
+6
+8
+12
+10
+5
+7
+9
+3
+4
+7
+7
+10
+8
+30
+9
+19
+4
+3
+19
+15
+4
+13
+3
+215
+223
+4
+7
+4
+8
+17
+16
+3
+7
+6
+5
+5
+4
+12
+3
+7
+4
+4
+13
+4
+5
+2
+5
+6
+5
+6
+6
+7
+10
+18
+23
+9
+3
+3
+6
+5
+2
+4
+2
+7
+3
+3
+2
+5
+5
+14
+10
+224
+6
+3
+4
+3
+7
+5
+9
+3
+6
+4
+2
+5
+11
+4
+3
+3
+2
+8
+4
+7
+4
+10
+7
+3
+3
+18
+18
+17
+3
+3
+3
+4
+5
+3
+3
+4
+12
+7
+3
+11
+13
+5
+4
+7
+13
+5
+4
+11
+3
+12
+3
+6
+4
+4
+21
+4
+6
+9
+5
+3
+10
+8
+4
+6
+4
+4
+6
+5
+4
+8
+6
+4
+6
+4
+4
+5
+9
+6
+3
+4
+2
+9
+3
+18
+2
+4
+3
+13
+3
+6
+6
+8
+7
+9
+3
+2
+16
+3
+4
+6
+3
+2
+33
+22
+14
+4
+9
+12
+4
+5
+6
+3
+23
+9
+4
+3
+5
+5
+3
+4
+5
+3
+5
+3
+10
+4
+5
+5
+8
+4
+4
+6
+8
+5
+4
+3
+4
+6
+3
+3
+3
+5
+9
+12
+6
+5
+9
+3
+5
+3
+2
+2
+2
+18
+3
+2
+21
+2
+5
+4
+6
+4
+5
+10
+3
+9
+3
+2
+10
+7
+3
+6
+6
+4
+4
+8
+12
+7
+3
+7
+3
+3
+9
+3
+4
+5
+4
+4
+5
+5
+10
+15
+4
+4
+14
+6
+227
+3
+14
+5
+216
+22
+5
+4
+2
+2
+6
+3
+4
+2
+9
+9
+4
+3
+28
+13
+11
+4
+5
+3
+3
+2
+3
+3
+5
+3
+4
+3
+5
+23
+26
+3
+4
+5
+6
+4
+6
+3
+5
+5
+3
+4
+3
+2
+2
+2
+7
+14
+3
+6
+7
+17
+2
+2
+15
+14
+16
+4
+6
+7
+13
+6
+4
+5
+6
+16
+3
+3
+28
+3
+6
+15
+3
+9
+2
+4
+6
+3
+3
+22
+4
+12
+6
+7
+2
+5
+4
+10
+3
+16
+6
+9
+2
+5
+12
+7
+5
+5
+5
+5
+2
+11
+9
+17
+4
+3
+11
+7
+3
+5
+15
+4
+3
+4
+211
+8
+7
+5
+4
+7
+6
+7
+6
+3
+6
+5
+6
+5
+3
+4
+4
+26
+4
+6
+10
+4
+4
+3
+2
+3
+3
+4
+5
+9
+3
+9
+4
+4
+5
+5
+8
+2
+4
+2
+3
+8
+4
+11
+19
+5
+8
+6
+3
+5
+6
+12
+3
+2
+4
+16
+12
+3
+4
+4
+8
+6
+5
+6
+6
+219
+8
+222
+6
+16
+3
+13
+19
+5
+4
+3
+11
+6
+10
+4
+7
+7
+12
+5
+3
+3
+5
+6
+10
+3
+8
+2
+5
+4
+7
+2
+4
+4
+2
+12
+9
+6
+4
+2
+40
+2
+4
+10
+4
+223
+4
+2
+20
+6
+7
+24
+5
+4
+5
+2
+20
+16
+6
+5
+13
+2
+3
+3
+19
+3
+2
+4
+5
+6
+7
+11
+12
+5
+6
+7
+7
+3
+5
+3
+5
+3
+14
+3
+4
+4
+2
+11
+1
+7
+3
+9
+6
+11
+12
+5
+8
+6
+221
+4
+2
+12
+4
+3
+15
+4
+5
+226
+7
+218
+7
+5
+4
+5
+18
+4
+5
+9
+4
+4
+2
+9
+18
+18
+9
+5
+6
+6
+3
+3
+7
+3
+5
+4
+4
+4
+12
+3
+6
+31
+5
+4
+7
+3
+6
+5
+6
+5
+11
+2
+2
+11
+11
+6
+7
+5
+8
+7
+10
+5
+23
+7
+4
+3
+5
+34
+2
+5
+23
+7
+3
+6
+8
+4
+4
+4
+2
+5
+3
+8
+5
+4
+8
+25
+2
+3
+17
+8
+3
+4
+8
+7
+3
+15
+6
+5
+7
+21
+9
+5
+6
+6
+5
+3
+2
+3
+10
+3
+6
+3
+14
+7
+4
+4
+8
+7
+8
+2
+6
+12
+4
+213
+6
+5
+21
+8
+2
+5
+23
+3
+11
+2
+3
+6
+25
+2
+3
+6
+7
+6
+6
+4
+4
+6
+3
+17
+9
+7
+6
+4
+3
+10
+7
+2
+3
+3
+3
+11
+8
+3
+7
+6
+4
+14
+36
+3
+4
+3
+3
+22
+13
+21
+4
+2
+7
+4
+4
+17
+15
+3
+7
+11
+2
+4
+7
+6
+209
+6
+3
+2
+2
+24
+4
+9
+4
+3
+3
+3
+29
+2
+2
+4
+3
+3
+5
+4
+6
+3
+3
+2
+4
diff --git a/vendor/github.com/beorn7/perks/quantile/stream.go b/vendor/github.com/beorn7/perks/quantile/stream.go
new file mode 100644
index 0000000..d7d14f8
--- /dev/null
+++ b/vendor/github.com/beorn7/perks/quantile/stream.go
@@ -0,0 +1,316 @@
+// Package quantile computes approximate quantiles over an unbounded data
+// stream within low memory and CPU bounds.
+//
+// A small amount of accuracy is traded to achieve the above properties.
+//
+// Multiple streams can be merged before calling Query to generate a single set
+// of results. This is meaningful when the streams represent the same type of
+// data. See Merge and Samples.
+//
+// For more detailed information about the algorithm used, see:
+//
+// Effective Computation of Biased Quantiles over Data Streams
+//
+// http://www.cs.rutgers.edu/~muthu/bquant.pdf
+package quantile
+
+import (
+	"math"
+	"sort"
+)
+
+// Sample holds an observed value and meta information for compression. JSON
+// tags have been added for convenience.
+type Sample struct {
+	Value float64 `json:",string"`
+	Width float64 `json:",string"`
+	Delta float64 `json:",string"`
+}
+
+// Samples represents a slice of samples. It implements sort.Interface.
+type Samples []Sample
+
+func (a Samples) Len() int           { return len(a) }
+func (a Samples) Less(i, j int) bool { return a[i].Value < a[j].Value }
+func (a Samples) Swap(i, j int)      { a[i], a[j] = a[j], a[i] }
+
+type invariant func(s *stream, r float64) float64
+
+// NewLowBiased returns an initialized Stream for low-biased quantiles
+// (e.g. 0.01, 0.1, 0.5) where the needed quantiles are not known a priori, but
+// error guarantees can still be given even for the lower ranks of the data
+// distribution.
+//
+// The provided epsilon is a relative error, i.e. the true quantile of a value
+// returned by a query is guaranteed to be within (1±Epsilon)*Quantile.
+//
+// See http://www.cs.rutgers.edu/~muthu/bquant.pdf for time, space, and error
+// properties.
+func NewLowBiased(epsilon float64) *Stream {
+	ƒ := func(s *stream, r float64) float64 {
+		return 2 * epsilon * r
+	}
+	return newStream(ƒ)
+}
+
+// NewHighBiased returns an initialized Stream for high-biased quantiles
+// (e.g. 0.01, 0.1, 0.5) where the needed quantiles are not known a priori, but
+// error guarantees can still be given even for the higher ranks of the data
+// distribution.
+//
+// The provided epsilon is a relative error, i.e. the true quantile of a value
+// returned by a query is guaranteed to be within 1-(1±Epsilon)*(1-Quantile).
+//
+// See http://www.cs.rutgers.edu/~muthu/bquant.pdf for time, space, and error
+// properties.
+func NewHighBiased(epsilon float64) *Stream {
+	ƒ := func(s *stream, r float64) float64 {
+		return 2 * epsilon * (s.n - r)
+	}
+	return newStream(ƒ)
+}
+
+// NewTargeted returns an initialized Stream concerned with a particular set of
+// quantile values that are supplied a priori. Knowing these a priori reduces
+// space and computation time. The targets map maps the desired quantiles to
+// their absolute errors, i.e. the true quantile of a value returned by a query
+// is guaranteed to be within (Quantile±Epsilon).
+//
+// See http://www.cs.rutgers.edu/~muthu/bquant.pdf for time, space, and error properties.
+func NewTargeted(targetMap map[float64]float64) *Stream {
+	// Convert map to slice to avoid slow iterations on a map.
+	// ƒ is called on the hot path, so converting the map to a slice
+	// beforehand results in significant CPU savings.
+	targets := targetMapToSlice(targetMap)
+
+	ƒ := func(s *stream, r float64) float64 {
+		var m = math.MaxFloat64
+		var f float64
+		for _, t := range targets {
+			if t.quantile*s.n <= r {
+				f = (2 * t.epsilon * r) / t.quantile
+			} else {
+				f = (2 * t.epsilon * (s.n - r)) / (1 - t.quantile)
+			}
+			if f < m {
+				m = f
+			}
+		}
+		return m
+	}
+	return newStream(ƒ)
+}
+
+type target struct {
+	quantile float64
+	epsilon  float64
+}
+
+func targetMapToSlice(targetMap map[float64]float64) []target {
+	targets := make([]target, 0, len(targetMap))
+
+	for quantile, epsilon := range targetMap {
+		t := target{
+			quantile: quantile,
+			epsilon:  epsilon,
+		}
+		targets = append(targets, t)
+	}
+
+	return targets
+}
+
+// Stream computes quantiles for a stream of float64s. It is not thread-safe by
+// design. Take care when using across multiple goroutines.
+type Stream struct {
+	*stream
+	b      Samples
+	sorted bool
+}
+
+func newStream(ƒ invariant) *Stream {
+	x := &stream{ƒ: ƒ}
+	return &Stream{x, make(Samples, 0, 500), true}
+}
+
+// Insert inserts v into the stream.
+func (s *Stream) Insert(v float64) {
+	s.insert(Sample{Value: v, Width: 1})
+}
+
+func (s *Stream) insert(sample Sample) {
+	s.b = append(s.b, sample)
+	s.sorted = false
+	if len(s.b) == cap(s.b) {
+		s.flush()
+	}
+}
+
+// Query returns the computed qth percentiles value. If s was created with
+// NewTargeted, and q is not in the set of quantiles provided a priori, Query
+// will return an unspecified result.
+func (s *Stream) Query(q float64) float64 {
+	if !s.flushed() {
+		// Fast path when there hasn't been enough data for a flush;
+		// this also yields better accuracy for small sets of data.
+		l := len(s.b)
+		if l == 0 {
+			return 0
+		}
+		i := int(math.Ceil(float64(l) * q))
+		if i > 0 {
+			i -= 1
+		}
+		s.maybeSort()
+		return s.b[i].Value
+	}
+	s.flush()
+	return s.stream.query(q)
+}
+
+// Merge merges samples into the underlying streams samples. This is handy when
+// merging multiple streams from separate threads, database shards, etc.
+//
+// ATTENTION: This method is broken and does not yield correct results. The
+// underlying algorithm is not capable of merging streams correctly.
+func (s *Stream) Merge(samples Samples) {
+	sort.Sort(samples)
+	s.stream.merge(samples)
+}
+
+// Reset reinitializes and clears the list reusing the samples buffer memory.
+func (s *Stream) Reset() {
+	s.stream.reset()
+	s.b = s.b[:0]
+}
+
+// Samples returns stream samples held by s.
+func (s *Stream) Samples() Samples {
+	if !s.flushed() {
+		return s.b
+	}
+	s.flush()
+	return s.stream.samples()
+}
+
+// Count returns the total number of samples observed in the stream
+// since initialization.
+func (s *Stream) Count() int {
+	return len(s.b) + s.stream.count()
+}
+
+func (s *Stream) flush() {
+	s.maybeSort()
+	s.stream.merge(s.b)
+	s.b = s.b[:0]
+}
+
+func (s *Stream) maybeSort() {
+	if !s.sorted {
+		s.sorted = true
+		sort.Sort(s.b)
+	}
+}
+
+func (s *Stream) flushed() bool {
+	return len(s.stream.l) > 0
+}
+
+type stream struct {
+	n float64
+	l []Sample
+	ƒ invariant
+}
+
+func (s *stream) reset() {
+	s.l = s.l[:0]
+	s.n = 0
+}
+
+func (s *stream) insert(v float64) {
+	s.merge(Samples{{v, 1, 0}})
+}
+
+func (s *stream) merge(samples Samples) {
+	// TODO(beorn7): This tries to merge not only individual samples, but
+	// whole summaries. The paper doesn't mention merging summaries at
+	// all. Unittests show that the merging is inaccurate. Find out how to
+	// do merges properly.
+	var r float64
+	i := 0
+	for _, sample := range samples {
+		for ; i < len(s.l); i++ {
+			c := s.l[i]
+			if c.Value > sample.Value {
+				// Insert at position i.
+				s.l = append(s.l, Sample{})
+				copy(s.l[i+1:], s.l[i:])
+				s.l[i] = Sample{
+					sample.Value,
+					sample.Width,
+					math.Max(sample.Delta, math.Floor(s.ƒ(s, r))-1),
+					// TODO(beorn7): How to calculate delta correctly?
+				}
+				i++
+				goto inserted
+			}
+			r += c.Width
+		}
+		s.l = append(s.l, Sample{sample.Value, sample.Width, 0})
+		i++
+	inserted:
+		s.n += sample.Width
+		r += sample.Width
+	}
+	s.compress()
+}
+
+func (s *stream) count() int {
+	return int(s.n)
+}
+
+func (s *stream) query(q float64) float64 {
+	t := math.Ceil(q * s.n)
+	t += math.Ceil(s.ƒ(s, t) / 2)
+	p := s.l[0]
+	var r float64
+	for _, c := range s.l[1:] {
+		r += p.Width
+		if r+c.Width+c.Delta > t {
+			return p.Value
+		}
+		p = c
+	}
+	return p.Value
+}
+
+func (s *stream) compress() {
+	if len(s.l) < 2 {
+		return
+	}
+	x := s.l[len(s.l)-1]
+	xi := len(s.l) - 1
+	r := s.n - 1 - x.Width
+
+	for i := len(s.l) - 2; i >= 0; i-- {
+		c := s.l[i]
+		if c.Width+x.Width+x.Delta <= s.ƒ(s, r) {
+			x.Width += c.Width
+			s.l[xi] = x
+			// Remove element at i.
+			copy(s.l[i:], s.l[i+1:])
+			s.l = s.l[:len(s.l)-1]
+			xi -= 1
+		} else {
+			x = c
+			xi = i
+		}
+		r -= c.Width
+	}
+}
+
+func (s *stream) samples() Samples {
+	samples := make(Samples, len(s.l))
+	copy(samples, s.l)
+	return samples
+}
diff --git a/vendor/github.com/coreos/go-semver/LICENSE b/vendor/github.com/coreos/go-semver/LICENSE
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/vendor/github.com/coreos/go-semver/LICENSE
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
diff --git a/vendor/github.com/coreos/go-semver/semver/semver.go b/vendor/github.com/coreos/go-semver/semver/semver.go
new file mode 100644
index 0000000..110fc23
--- /dev/null
+++ b/vendor/github.com/coreos/go-semver/semver/semver.go
@@ -0,0 +1,268 @@
+// Copyright 2013-2015 CoreOS, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// Semantic Versions http://semver.org
+package semver
+
+import (
+	"bytes"
+	"errors"
+	"fmt"
+	"strconv"
+	"strings"
+)
+
+type Version struct {
+	Major      int64
+	Minor      int64
+	Patch      int64
+	PreRelease PreRelease
+	Metadata   string
+}
+
+type PreRelease string
+
+func splitOff(input *string, delim string) (val string) {
+	parts := strings.SplitN(*input, delim, 2)
+
+	if len(parts) == 2 {
+		*input = parts[0]
+		val = parts[1]
+	}
+
+	return val
+}
+
+func New(version string) *Version {
+	return Must(NewVersion(version))
+}
+
+func NewVersion(version string) (*Version, error) {
+	v := Version{}
+
+	if err := v.Set(version); err != nil {
+		return nil, err
+	}
+
+	return &v, nil
+}
+
+// Must is a helper for wrapping NewVersion and will panic if err is not nil.
+func Must(v *Version, err error) *Version {
+	if err != nil {
+		panic(err)
+	}
+	return v
+}
+
+// Set parses and updates v from the given version string. Implements flag.Value
+func (v *Version) Set(version string) error {
+	metadata := splitOff(&version, "+")
+	preRelease := PreRelease(splitOff(&version, "-"))
+	dotParts := strings.SplitN(version, ".", 3)
+
+	if len(dotParts) != 3 {
+		return fmt.Errorf("%s is not in dotted-tri format", version)
+	}
+
+	parsed := make([]int64, 3, 3)
+
+	for i, v := range dotParts[:3] {
+		val, err := strconv.ParseInt(v, 10, 64)
+		parsed[i] = val
+		if err != nil {
+			return err
+		}
+	}
+
+	v.Metadata = metadata
+	v.PreRelease = preRelease
+	v.Major = parsed[0]
+	v.Minor = parsed[1]
+	v.Patch = parsed[2]
+	return nil
+}
+
+func (v Version) String() string {
+	var buffer bytes.Buffer
+
+	fmt.Fprintf(&buffer, "%d.%d.%d", v.Major, v.Minor, v.Patch)
+
+	if v.PreRelease != "" {
+		fmt.Fprintf(&buffer, "-%s", v.PreRelease)
+	}
+
+	if v.Metadata != "" {
+		fmt.Fprintf(&buffer, "+%s", v.Metadata)
+	}
+
+	return buffer.String()
+}
+
+func (v *Version) UnmarshalYAML(unmarshal func(interface{}) error) error {
+	var data string
+	if err := unmarshal(&data); err != nil {
+		return err
+	}
+	return v.Set(data)
+}
+
+func (v Version) MarshalJSON() ([]byte, error) {
+	return []byte(`"` + v.String() + `"`), nil
+}
+
+func (v *Version) UnmarshalJSON(data []byte) error {
+	l := len(data)
+	if l == 0 || string(data) == `""` {
+		return nil
+	}
+	if l < 2 || data[0] != '"' || data[l-1] != '"' {
+		return errors.New("invalid semver string")
+	}
+	return v.Set(string(data[1 : l-1]))
+}
+
+// Compare tests if v is less than, equal to, or greater than versionB,
+// returning -1, 0, or +1 respectively.
+func (v Version) Compare(versionB Version) int {
+	if cmp := recursiveCompare(v.Slice(), versionB.Slice()); cmp != 0 {
+		return cmp
+	}
+	return preReleaseCompare(v, versionB)
+}
+
+// Equal tests if v is equal to versionB.
+func (v Version) Equal(versionB Version) bool {
+	return v.Compare(versionB) == 0
+}
+
+// LessThan tests if v is less than versionB.
+func (v Version) LessThan(versionB Version) bool {
+	return v.Compare(versionB) < 0
+}
+
+// Slice converts the comparable parts of the semver into a slice of integers.
+func (v Version) Slice() []int64 {
+	return []int64{v.Major, v.Minor, v.Patch}
+}
+
+func (p PreRelease) Slice() []string {
+	preRelease := string(p)
+	return strings.Split(preRelease, ".")
+}
+
+func preReleaseCompare(versionA Version, versionB Version) int {
+	a := versionA.PreRelease
+	b := versionB.PreRelease
+
+	/* Handle the case where if two versions are otherwise equal it is the
+	 * one without a PreRelease that is greater */
+	if len(a) == 0 && (len(b) > 0) {
+		return 1
+	} else if len(b) == 0 && (len(a) > 0) {
+		return -1
+	}
+
+	// If there is a prerelease, check and compare each part.
+	return recursivePreReleaseCompare(a.Slice(), b.Slice())
+}
+
+func recursiveCompare(versionA []int64, versionB []int64) int {
+	if len(versionA) == 0 {
+		return 0
+	}
+
+	a := versionA[0]
+	b := versionB[0]
+
+	if a > b {
+		return 1
+	} else if a < b {
+		return -1
+	}
+
+	return recursiveCompare(versionA[1:], versionB[1:])
+}
+
+func recursivePreReleaseCompare(versionA []string, versionB []string) int {
+	// A larger set of pre-release fields has a higher precedence than a smaller set,
+	// if all of the preceding identifiers are equal.
+	if len(versionA) == 0 {
+		if len(versionB) > 0 {
+			return -1
+		}
+		return 0
+	} else if len(versionB) == 0 {
+		// We're longer than versionB so return 1.
+		return 1
+	}
+
+	a := versionA[0]
+	b := versionB[0]
+
+	aInt := false
+	bInt := false
+
+	aI, err := strconv.Atoi(versionA[0])
+	if err == nil {
+		aInt = true
+	}
+
+	bI, err := strconv.Atoi(versionB[0])
+	if err == nil {
+		bInt = true
+	}
+
+	// Handle Integer Comparison
+	if aInt && bInt {
+		if aI > bI {
+			return 1
+		} else if aI < bI {
+			return -1
+		}
+	}
+
+	// Handle String Comparison
+	if a > b {
+		return 1
+	} else if a < b {
+		return -1
+	}
+
+	return recursivePreReleaseCompare(versionA[1:], versionB[1:])
+}
+
+// BumpMajor increments the Major field by 1 and resets all other fields to their default values
+func (v *Version) BumpMajor() {
+	v.Major += 1
+	v.Minor = 0
+	v.Patch = 0
+	v.PreRelease = PreRelease("")
+	v.Metadata = ""
+}
+
+// BumpMinor increments the Minor field by 1 and resets all other fields to their default values
+func (v *Version) BumpMinor() {
+	v.Minor += 1
+	v.Patch = 0
+	v.PreRelease = PreRelease("")
+	v.Metadata = ""
+}
+
+// BumpPatch increments the Patch field by 1 and resets all other fields to their default values
+func (v *Version) BumpPatch() {
+	v.Patch += 1
+	v.PreRelease = PreRelease("")
+	v.Metadata = ""
+}
diff --git a/vendor/github.com/coreos/go-semver/semver/sort.go b/vendor/github.com/coreos/go-semver/semver/sort.go
new file mode 100644
index 0000000..e256b41
--- /dev/null
+++ b/vendor/github.com/coreos/go-semver/semver/sort.go
@@ -0,0 +1,38 @@
+// Copyright 2013-2015 CoreOS, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package semver
+
+import (
+	"sort"
+)
+
+type Versions []*Version
+
+func (s Versions) Len() int {
+	return len(s)
+}
+
+func (s Versions) Swap(i, j int) {
+	s[i], s[j] = s[j], s[i]
+}
+
+func (s Versions) Less(i, j int) bool {
+	return s[i].LessThan(*s[j])
+}
+
+// Sort sorts the given slice of Version
+func Sort(versions []*Version) {
+	sort.Sort(Versions(versions))
+}
diff --git a/vendor/github.com/dgrijalva/jwt-go/.gitignore b/vendor/github.com/dgrijalva/jwt-go/.gitignore
new file mode 100644
index 0000000..80bed65
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/.gitignore
@@ -0,0 +1,4 @@
+.DS_Store
+bin
+
+
diff --git a/vendor/github.com/dgrijalva/jwt-go/.travis.yml b/vendor/github.com/dgrijalva/jwt-go/.travis.yml
new file mode 100644
index 0000000..1027f56
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/.travis.yml
@@ -0,0 +1,13 @@
+language: go
+
+script:
+    - go vet ./...
+    - go test -v ./...
+
+go:
+  - 1.3
+  - 1.4
+  - 1.5
+  - 1.6
+  - 1.7
+  - tip
diff --git a/vendor/github.com/dgrijalva/jwt-go/LICENSE b/vendor/github.com/dgrijalva/jwt-go/LICENSE
new file mode 100644
index 0000000..df83a9c
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/LICENSE
@@ -0,0 +1,8 @@
+Copyright (c) 2012 Dave Grijalva
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
diff --git a/vendor/github.com/dgrijalva/jwt-go/MIGRATION_GUIDE.md b/vendor/github.com/dgrijalva/jwt-go/MIGRATION_GUIDE.md
new file mode 100644
index 0000000..7fc1f79
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/MIGRATION_GUIDE.md
@@ -0,0 +1,97 @@
+## Migration Guide from v2 -> v3
+
+Version 3 adds several new, frequently requested features.  To do so, it introduces a few breaking changes.  We've worked to keep these as minimal as possible.  This guide explains the breaking changes and how you can quickly update your code.
+
+### `Token.Claims` is now an interface type
+
+The most requested feature from the 2.0 verison of this library was the ability to provide a custom type to the JSON parser for claims. This was implemented by introducing a new interface, `Claims`, to replace `map[string]interface{}`.  We also included two concrete implementations of `Claims`: `MapClaims` and `StandardClaims`.
+
+`MapClaims` is an alias for `map[string]interface{}` with built in validation behavior.  It is the default claims type when using `Parse`.  The usage is unchanged except you must type cast the claims property.
+
+The old example for parsing a token looked like this..
+
+```go
+	if token, err := jwt.Parse(tokenString, keyLookupFunc); err == nil {
+		fmt.Printf("Token for user %v expires %v", token.Claims["user"], token.Claims["exp"])
+	}
+```
+
+is now directly mapped to...
+
+```go
+	if token, err := jwt.Parse(tokenString, keyLookupFunc); err == nil {
+		claims := token.Claims.(jwt.MapClaims)
+		fmt.Printf("Token for user %v expires %v", claims["user"], claims["exp"])
+	}
+```
+
+`StandardClaims` is designed to be embedded in your custom type.  You can supply a custom claims type with the new `ParseWithClaims` function.  Here's an example of using a custom claims type.
+
+```go
+	type MyCustomClaims struct {
+		User string
+		*StandardClaims
+	}
+	
+	if token, err := jwt.ParseWithClaims(tokenString, &MyCustomClaims{}, keyLookupFunc); err == nil {
+		claims := token.Claims.(*MyCustomClaims)
+		fmt.Printf("Token for user %v expires %v", claims.User, claims.StandardClaims.ExpiresAt)
+	}
+```
+
+### `ParseFromRequest` has been moved
+
+To keep this library focused on the tokens without becoming overburdened with complex request processing logic, `ParseFromRequest` and its new companion `ParseFromRequestWithClaims` have been moved to a subpackage, `request`.  The method signatues have also been augmented to receive a new argument: `Extractor`.
+
+`Extractors` do the work of picking the token string out of a request.  The interface is simple and composable.
+
+This simple parsing example:
+
+```go
+	if token, err := jwt.ParseFromRequest(tokenString, req, keyLookupFunc); err == nil {
+		fmt.Printf("Token for user %v expires %v", token.Claims["user"], token.Claims["exp"])
+	}
+```
+
+is directly mapped to:
+
+```go
+	if token, err := request.ParseFromRequest(req, request.OAuth2Extractor, keyLookupFunc); err == nil {
+		claims := token.Claims.(jwt.MapClaims)
+		fmt.Printf("Token for user %v expires %v", claims["user"], claims["exp"])
+	}
+```
+
+There are several concrete `Extractor` types provided for your convenience:
+
+* `HeaderExtractor` will search a list of headers until one contains content.
+* `ArgumentExtractor` will search a list of keys in request query and form arguments until one contains content.
+* `MultiExtractor` will try a list of `Extractors` in order until one returns content.
+* `AuthorizationHeaderExtractor` will look in the `Authorization` header for a `Bearer` token.
+* `OAuth2Extractor` searches the places an OAuth2 token would be specified (per the spec): `Authorization` header and `access_token` argument
+* `PostExtractionFilter` wraps an `Extractor`, allowing you to process the content before it's parsed.  A simple example is stripping the `Bearer ` text from a header
+
+
+### RSA signing methods no longer accept `[]byte` keys
+
+Due to a [critical vulnerability](https://auth0.com/blog/2015/03/31/critical-vulnerabilities-in-json-web-token-libraries/), we've decided the convenience of accepting `[]byte` instead of `rsa.PublicKey` or `rsa.PrivateKey` isn't worth the risk of misuse.
+
+To replace this behavior, we've added two helper methods: `ParseRSAPrivateKeyFromPEM(key []byte) (*rsa.PrivateKey, error)` and `ParseRSAPublicKeyFromPEM(key []byte) (*rsa.PublicKey, error)`.  These are just simple helpers for unpacking PEM encoded PKCS1 and PKCS8 keys. If your keys are encoded any other way, all you need to do is convert them to the `crypto/rsa` package's types.
+
+```go 
+	func keyLookupFunc(*Token) (interface{}, error) {
+		// Don't forget to validate the alg is what you expect:
+		if _, ok := token.Method.(*jwt.SigningMethodRSA); !ok {
+			return nil, fmt.Errorf("Unexpected signing method: %v", token.Header["alg"])
+		}
+		
+		// Look up key 
+		key, err := lookupPublicKey(token.Header["kid"])
+		if err != nil {
+			return nil, err
+		}
+		
+		// Unpack key from PEM encoded PKCS8
+		return jwt.ParseRSAPublicKeyFromPEM(key)
+	}
+```
diff --git a/vendor/github.com/dgrijalva/jwt-go/README.md b/vendor/github.com/dgrijalva/jwt-go/README.md
new file mode 100644
index 0000000..d358d88
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/README.md
@@ -0,0 +1,100 @@
+# jwt-go
+
+[![Build Status](https://travis-ci.org/dgrijalva/jwt-go.svg?branch=master)](https://travis-ci.org/dgrijalva/jwt-go)
+[![GoDoc](https://godoc.org/github.com/dgrijalva/jwt-go?status.svg)](https://godoc.org/github.com/dgrijalva/jwt-go)
+
+A [go](http://www.golang.org) (or 'golang' for search engine friendliness) implementation of [JSON Web Tokens](http://self-issued.info/docs/draft-ietf-oauth-json-web-token.html)
+
+**NEW VERSION COMING:** There have been a lot of improvements suggested since the version 3.0.0 released in 2016. I'm working now on cutting two different releases: 3.2.0 will contain any non-breaking changes or enhancements. 4.0.0 will follow shortly which will include breaking changes. See the 4.0.0 milestone to get an idea of what's coming. If you have other ideas, or would like to participate in 4.0.0, now's the time. If you depend on this library and don't want to be interrupted, I recommend you use your dependency mangement tool to pin to version 3. 
+
+**SECURITY NOTICE:** Some older versions of Go have a security issue in the cryotp/elliptic. Recommendation is to upgrade to at least 1.8.3. See issue #216 for more detail.
+
+**SECURITY NOTICE:** It's important that you [validate the `alg` presented is what you expect](https://auth0.com/blog/2015/03/31/critical-vulnerabilities-in-json-web-token-libraries/). This library attempts to make it easy to do the right thing by requiring key types match the expected alg, but you should take the extra step to verify it in your usage.  See the examples provided.
+
+## What the heck is a JWT?
+
+JWT.io has [a great introduction](https://jwt.io/introduction) to JSON Web Tokens.
+
+In short, it's a signed JSON object that does something useful (for example, authentication).  It's commonly used for `Bearer` tokens in Oauth 2.  A token is made of three parts, separated by `.`'s.  The first two parts are JSON objects, that have been [base64url](http://tools.ietf.org/html/rfc4648) encoded.  The last part is the signature, encoded the same way.
+
+The first part is called the header.  It contains the necessary information for verifying the last part, the signature.  For example, which encryption method was used for signing and what key was used.
+
+The part in the middle is the interesting bit.  It's called the Claims and contains the actual stuff you care about.  Refer to [the RFC](http://self-issued.info/docs/draft-jones-json-web-token.html) for information about reserved keys and the proper way to add your own.
+
+## What's in the box?
+
+This library supports the parsing and verification as well as the generation and signing of JWTs.  Current supported signing algorithms are HMAC SHA, RSA, RSA-PSS, and ECDSA, though hooks are present for adding your own.
+
+## Examples
+
+See [the project documentation](https://godoc.org/github.com/dgrijalva/jwt-go) for examples of usage:
+
+* [Simple example of parsing and validating a token](https://godoc.org/github.com/dgrijalva/jwt-go#example-Parse--Hmac)
+* [Simple example of building and signing a token](https://godoc.org/github.com/dgrijalva/jwt-go#example-New--Hmac)
+* [Directory of Examples](https://godoc.org/github.com/dgrijalva/jwt-go#pkg-examples)
+
+## Extensions
+
+This library publishes all the necessary components for adding your own signing methods.  Simply implement the `SigningMethod` interface and register a factory method using `RegisterSigningMethod`.  
+
+Here's an example of an extension that integrates with the Google App Engine signing tools: https://github.com/someone1/gcp-jwt-go
+
+## Compliance
+
+This library was last reviewed to comply with [RTF 7519](http://www.rfc-editor.org/info/rfc7519) dated May 2015 with a few notable differences:
+
+* In order to protect against accidental use of [Unsecured JWTs](http://self-issued.info/docs/draft-ietf-oauth-json-web-token.html#UnsecuredJWT), tokens using `alg=none` will only be accepted if the constant `jwt.UnsafeAllowNoneSignatureType` is provided as the key.
+
+## Project Status & Versioning
+
+This library is considered production ready.  Feedback and feature requests are appreciated.  The API should be considered stable.  There should be very few backwards-incompatible changes outside of major version updates (and only with good reason).
+
+This project uses [Semantic Versioning 2.0.0](http://semver.org).  Accepted pull requests will land on `master`.  Periodically, versions will be tagged from `master`.  You can find all the releases on [the project releases page](https://github.com/dgrijalva/jwt-go/releases).
+
+While we try to make it obvious when we make breaking changes, there isn't a great mechanism for pushing announcements out to users.  You may want to use this alternative package include: `gopkg.in/dgrijalva/jwt-go.v3`.  It will do the right thing WRT semantic versioning.
+
+**BREAKING CHANGES:*** 
+* Version 3.0.0 includes _a lot_ of changes from the 2.x line, including a few that break the API.  We've tried to break as few things as possible, so there should just be a few type signature changes.  A full list of breaking changes is available in `VERSION_HISTORY.md`.  See `MIGRATION_GUIDE.md` for more information on updating your code.
+
+## Usage Tips
+
+### Signing vs Encryption
+
+A token is simply a JSON object that is signed by its author. this tells you exactly two things about the data:
+
+* The author of the token was in the possession of the signing secret
+* The data has not been modified since it was signed
+
+It's important to know that JWT does not provide encryption, which means anyone who has access to the token can read its contents. If you need to protect (encrypt) the data, there is a companion spec, `JWE`, that provides this functionality. JWE is currently outside the scope of this library.
+
+### Choosing a Signing Method
+
+There are several signing methods available, and you should probably take the time to learn about the various options before choosing one.  The principal design decision is most likely going to be symmetric vs asymmetric.
+
+Symmetric signing methods, such as HSA, use only a single secret. This is probably the simplest signing method to use since any `[]byte` can be used as a valid secret. They are also slightly computationally faster to use, though this rarely is enough to matter. Symmetric signing methods work the best when both producers and consumers of tokens are trusted, or even the same system. Since the same secret is used to both sign and validate tokens, you can't easily distribute the key for validation.
+
+Asymmetric signing methods, such as RSA, use different keys for signing and verifying tokens. This makes it possible to produce tokens with a private key, and allow any consumer to access the public key for verification.
+
+### Signing Methods and Key Types
+
+Each signing method expects a different object type for its signing keys. See the package documentation for details. Here are the most common ones:
+
+* The [HMAC signing method](https://godoc.org/github.com/dgrijalva/jwt-go#SigningMethodHMAC) (`HS256`,`HS384`,`HS512`) expect `[]byte` values for signing and validation
+* The [RSA signing method](https://godoc.org/github.com/dgrijalva/jwt-go#SigningMethodRSA) (`RS256`,`RS384`,`RS512`) expect `*rsa.PrivateKey` for signing and `*rsa.PublicKey` for validation
+* The [ECDSA signing method](https://godoc.org/github.com/dgrijalva/jwt-go#SigningMethodECDSA) (`ES256`,`ES384`,`ES512`) expect `*ecdsa.PrivateKey` for signing and `*ecdsa.PublicKey` for validation
+
+### JWT and OAuth
+
+It's worth mentioning that OAuth and JWT are not the same thing. A JWT token is simply a signed JSON object. It can be used anywhere such a thing is useful. There is some confusion, though, as JWT is the most common type of bearer token used in OAuth2 authentication.
+
+Without going too far down the rabbit hole, here's a description of the interaction of these technologies:
+
+* OAuth is a protocol for allowing an identity provider to be separate from the service a user is logging in to. For example, whenever you use Facebook to log into a different service (Yelp, Spotify, etc), you are using OAuth.
+* OAuth defines several options for passing around authentication data. One popular method is called a "bearer token". A bearer token is simply a string that _should_ only be held by an authenticated user. Thus, simply presenting this token proves your identity. You can probably derive from here why a JWT might make a good bearer token.
+* Because bearer tokens are used for authentication, it's important they're kept secret. This is why transactions that use bearer tokens typically happen over SSL.
+
+## More
+
+Documentation can be found [on godoc.org](http://godoc.org/github.com/dgrijalva/jwt-go).
+
+The command line utility included in this project (cmd/jwt) provides a straightforward example of token creation and parsing as well as a useful tool for debugging your own integration. You'll also find several implementation examples in the documentation.
diff --git a/vendor/github.com/dgrijalva/jwt-go/VERSION_HISTORY.md b/vendor/github.com/dgrijalva/jwt-go/VERSION_HISTORY.md
new file mode 100644
index 0000000..6370298
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/VERSION_HISTORY.md
@@ -0,0 +1,118 @@
+## `jwt-go` Version History
+
+#### 3.2.0
+
+* Added method `ParseUnverified` to allow users to split up the tasks of parsing and validation
+* HMAC signing method returns `ErrInvalidKeyType` instead of `ErrInvalidKey` where appropriate
+* Added options to `request.ParseFromRequest`, which allows for an arbitrary list of modifiers to parsing behavior. Initial set include `WithClaims` and `WithParser`. Existing usage of this function will continue to work as before.
+* Deprecated `ParseFromRequestWithClaims` to simplify API in the future.
+
+#### 3.1.0
+
+* Improvements to `jwt` command line tool
+* Added `SkipClaimsValidation` option to `Parser`
+* Documentation updates
+
+#### 3.0.0
+
+* **Compatibility Breaking Changes**: See MIGRATION_GUIDE.md for tips on updating your code
+	* Dropped support for `[]byte` keys when using RSA signing methods.  This convenience feature could contribute to security vulnerabilities involving mismatched key types with signing methods.
+	* `ParseFromRequest` has been moved to `request` subpackage and usage has changed
+	* The `Claims` property on `Token` is now type `Claims` instead of `map[string]interface{}`.  The default value is type `MapClaims`, which is an alias to `map[string]interface{}`.  This makes it possible to use a custom type when decoding claims.
+* Other Additions and Changes
+	* Added `Claims` interface type to allow users to decode the claims into a custom type
+	* Added `ParseWithClaims`, which takes a third argument of type `Claims`.  Use this function instead of `Parse` if you have a custom type you'd like to decode into.
+	* Dramatically improved the functionality and flexibility of `ParseFromRequest`, which is now in the `request` subpackage
+	* Added `ParseFromRequestWithClaims` which is the `FromRequest` equivalent of `ParseWithClaims`
+	* Added new interface type `Extractor`, which is used for extracting JWT strings from http requests.  Used with `ParseFromRequest` and `ParseFromRequestWithClaims`.
+	* Added several new, more specific, validation errors to error type bitmask
+	* Moved examples from README to executable example files
+	* Signing method registry is now thread safe
+	* Added new property to `ValidationError`, which contains the raw error returned by calls made by parse/verify (such as those returned by keyfunc or json parser)
+
+#### 2.7.0
+
+This will likely be the last backwards compatible release before 3.0.0, excluding essential bug fixes.
+
+* Added new option `-show` to the `jwt` command that will just output the decoded token without verifying
+* Error text for expired tokens includes how long it's been expired
+* Fixed incorrect error returned from `ParseRSAPublicKeyFromPEM`
+* Documentation updates
+
+#### 2.6.0
+
+* Exposed inner error within ValidationError
+* Fixed validation errors when using UseJSONNumber flag
+* Added several unit tests
+
+#### 2.5.0
+
+* Added support for signing method none.  You shouldn't use this.  The API tries to make this clear.
+* Updated/fixed some documentation
+* Added more helpful error message when trying to parse tokens that begin with `BEARER `
+
+#### 2.4.0
+
+* Added new type, Parser, to allow for configuration of various parsing parameters
+	* You can now specify a list of valid signing methods.  Anything outside this set will be rejected.
+	* You can now opt to use the `json.Number` type instead of `float64` when parsing token JSON
+* Added support for [Travis CI](https://travis-ci.org/dgrijalva/jwt-go)
+* Fixed some bugs with ECDSA parsing
+
+#### 2.3.0
+
+* Added support for ECDSA signing methods
+* Added support for RSA PSS signing methods (requires go v1.4)
+
+#### 2.2.0
+
+* Gracefully handle a `nil` `Keyfunc` being passed to `Parse`.  Result will now be the parsed token and an error, instead of a panic.
+
+#### 2.1.0
+
+Backwards compatible API change that was missed in 2.0.0.
+
+* The `SignedString` method on `Token` now takes `interface{}` instead of `[]byte`
+
+#### 2.0.0
+
+There were two major reasons for breaking backwards compatibility with this update.  The first was a refactor required to expand the width of the RSA and HMAC-SHA signing implementations.  There will likely be no required code changes to support this change.
+
+The second update, while unfortunately requiring a small change in integration, is required to open up this library to other signing methods.  Not all keys used for all signing methods have a single standard on-disk representation.  Requiring `[]byte` as the type for all keys proved too limiting.  Additionally, this implementation allows for pre-parsed tokens to be reused, which might matter in an application that parses a high volume of tokens with a small set of keys.  Backwards compatibilty has been maintained for passing `[]byte` to the RSA signing methods, but they will also accept `*rsa.PublicKey` and `*rsa.PrivateKey`.
+
+It is likely the only integration change required here will be to change `func(t *jwt.Token) ([]byte, error)` to `func(t *jwt.Token) (interface{}, error)` when calling `Parse`.
+
+* **Compatibility Breaking Changes**
+	* `SigningMethodHS256` is now `*SigningMethodHMAC` instead of `type struct`
+	* `SigningMethodRS256` is now `*SigningMethodRSA` instead of `type struct`
+	* `KeyFunc` now returns `interface{}` instead of `[]byte`
+	* `SigningMethod.Sign` now takes `interface{}` instead of `[]byte` for the key
+	* `SigningMethod.Verify` now takes `interface{}` instead of `[]byte` for the key
+* Renamed type `SigningMethodHS256` to `SigningMethodHMAC`.  Specific sizes are now just instances of this type.
+    * Added public package global `SigningMethodHS256`
+    * Added public package global `SigningMethodHS384`
+    * Added public package global `SigningMethodHS512`
+* Renamed type `SigningMethodRS256` to `SigningMethodRSA`.  Specific sizes are now just instances of this type.
+    * Added public package global `SigningMethodRS256`
+    * Added public package global `SigningMethodRS384`
+    * Added public package global `SigningMethodRS512`
+* Moved sample private key for HMAC tests from an inline value to a file on disk.  Value is unchanged.
+* Refactored the RSA implementation to be easier to read
+* Exposed helper methods `ParseRSAPrivateKeyFromPEM` and `ParseRSAPublicKeyFromPEM`
+
+#### 1.0.2
+
+* Fixed bug in parsing public keys from certificates
+* Added more tests around the parsing of keys for RS256
+* Code refactoring in RS256 implementation.  No functional changes
+
+#### 1.0.1
+
+* Fixed panic if RS256 signing method was passed an invalid key
+
+#### 1.0.0
+
+* First versioned release
+* API stabilized
+* Supports creating, signing, parsing, and validating JWT tokens
+* Supports RS256 and HS256 signing methods
\ No newline at end of file
diff --git a/vendor/github.com/dgrijalva/jwt-go/claims.go b/vendor/github.com/dgrijalva/jwt-go/claims.go
new file mode 100644
index 0000000..f0228f0
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/claims.go
@@ -0,0 +1,134 @@
+package jwt
+
+import (
+	"crypto/subtle"
+	"fmt"
+	"time"
+)
+
+// For a type to be a Claims object, it must just have a Valid method that determines
+// if the token is invalid for any supported reason
+type Claims interface {
+	Valid() error
+}
+
+// Structured version of Claims Section, as referenced at
+// https://tools.ietf.org/html/rfc7519#section-4.1
+// See examples for how to use this with your own claim types
+type StandardClaims struct {
+	Audience  string `json:"aud,omitempty"`
+	ExpiresAt int64  `json:"exp,omitempty"`
+	Id        string `json:"jti,omitempty"`
+	IssuedAt  int64  `json:"iat,omitempty"`
+	Issuer    string `json:"iss,omitempty"`
+	NotBefore int64  `json:"nbf,omitempty"`
+	Subject   string `json:"sub,omitempty"`
+}
+
+// Validates time based claims "exp, iat, nbf".
+// There is no accounting for clock skew.
+// As well, if any of the above claims are not in the token, it will still
+// be considered a valid claim.
+func (c StandardClaims) Valid() error {
+	vErr := new(ValidationError)
+	now := TimeFunc().Unix()
+
+	// The claims below are optional, by default, so if they are set to the
+	// default value in Go, let's not fail the verification for them.
+	if c.VerifyExpiresAt(now, false) == false {
+		delta := time.Unix(now, 0).Sub(time.Unix(c.ExpiresAt, 0))
+		vErr.Inner = fmt.Errorf("token is expired by %v", delta)
+		vErr.Errors |= ValidationErrorExpired
+	}
+
+	if c.VerifyIssuedAt(now, false) == false {
+		vErr.Inner = fmt.Errorf("Token used before issued")
+		vErr.Errors |= ValidationErrorIssuedAt
+	}
+
+	if c.VerifyNotBefore(now, false) == false {
+		vErr.Inner = fmt.Errorf("token is not valid yet")
+		vErr.Errors |= ValidationErrorNotValidYet
+	}
+
+	if vErr.valid() {
+		return nil
+	}
+
+	return vErr
+}
+
+// Compares the aud claim against cmp.
+// If required is false, this method will return true if the value matches or is unset
+func (c *StandardClaims) VerifyAudience(cmp string, req bool) bool {
+	return verifyAud(c.Audience, cmp, req)
+}
+
+// Compares the exp claim against cmp.
+// If required is false, this method will return true if the value matches or is unset
+func (c *StandardClaims) VerifyExpiresAt(cmp int64, req bool) bool {
+	return verifyExp(c.ExpiresAt, cmp, req)
+}
+
+// Compares the iat claim against cmp.
+// If required is false, this method will return true if the value matches or is unset
+func (c *StandardClaims) VerifyIssuedAt(cmp int64, req bool) bool {
+	return verifyIat(c.IssuedAt, cmp, req)
+}
+
+// Compares the iss claim against cmp.
+// If required is false, this method will return true if the value matches or is unset
+func (c *StandardClaims) VerifyIssuer(cmp string, req bool) bool {
+	return verifyIss(c.Issuer, cmp, req)
+}
+
+// Compares the nbf claim against cmp.
+// If required is false, this method will return true if the value matches or is unset
+func (c *StandardClaims) VerifyNotBefore(cmp int64, req bool) bool {
+	return verifyNbf(c.NotBefore, cmp, req)
+}
+
+// ----- helpers
+
+func verifyAud(aud string, cmp string, required bool) bool {
+	if aud == "" {
+		return !required
+	}
+	if subtle.ConstantTimeCompare([]byte(aud), []byte(cmp)) != 0 {
+		return true
+	} else {
+		return false
+	}
+}
+
+func verifyExp(exp int64, now int64, required bool) bool {
+	if exp == 0 {
+		return !required
+	}
+	return now <= exp
+}
+
+func verifyIat(iat int64, now int64, required bool) bool {
+	if iat == 0 {
+		return !required
+	}
+	return now >= iat
+}
+
+func verifyIss(iss string, cmp string, required bool) bool {
+	if iss == "" {
+		return !required
+	}
+	if subtle.ConstantTimeCompare([]byte(iss), []byte(cmp)) != 0 {
+		return true
+	} else {
+		return false
+	}
+}
+
+func verifyNbf(nbf int64, now int64, required bool) bool {
+	if nbf == 0 {
+		return !required
+	}
+	return now >= nbf
+}
diff --git a/vendor/github.com/dgrijalva/jwt-go/doc.go b/vendor/github.com/dgrijalva/jwt-go/doc.go
new file mode 100644
index 0000000..a86dc1a
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/doc.go
@@ -0,0 +1,4 @@
+// Package jwt is a Go implementation of JSON Web Tokens: http://self-issued.info/docs/draft-jones-json-web-token.html
+//
+// See README.md for more info.
+package jwt
diff --git a/vendor/github.com/dgrijalva/jwt-go/ecdsa.go b/vendor/github.com/dgrijalva/jwt-go/ecdsa.go
new file mode 100644
index 0000000..f977381
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/ecdsa.go
@@ -0,0 +1,148 @@
+package jwt
+
+import (
+	"crypto"
+	"crypto/ecdsa"
+	"crypto/rand"
+	"errors"
+	"math/big"
+)
+
+var (
+	// Sadly this is missing from crypto/ecdsa compared to crypto/rsa
+	ErrECDSAVerification = errors.New("crypto/ecdsa: verification error")
+)
+
+// Implements the ECDSA family of signing methods signing methods
+// Expects *ecdsa.PrivateKey for signing and *ecdsa.PublicKey for verification
+type SigningMethodECDSA struct {
+	Name      string
+	Hash      crypto.Hash
+	KeySize   int
+	CurveBits int
+}
+
+// Specific instances for EC256 and company
+var (
+	SigningMethodES256 *SigningMethodECDSA
+	SigningMethodES384 *SigningMethodECDSA
+	SigningMethodES512 *SigningMethodECDSA
+)
+
+func init() {
+	// ES256
+	SigningMethodES256 = &SigningMethodECDSA{"ES256", crypto.SHA256, 32, 256}
+	RegisterSigningMethod(SigningMethodES256.Alg(), func() SigningMethod {
+		return SigningMethodES256
+	})
+
+	// ES384
+	SigningMethodES384 = &SigningMethodECDSA{"ES384", crypto.SHA384, 48, 384}
+	RegisterSigningMethod(SigningMethodES384.Alg(), func() SigningMethod {
+		return SigningMethodES384
+	})
+
+	// ES512
+	SigningMethodES512 = &SigningMethodECDSA{"ES512", crypto.SHA512, 66, 521}
+	RegisterSigningMethod(SigningMethodES512.Alg(), func() SigningMethod {
+		return SigningMethodES512
+	})
+}
+
+func (m *SigningMethodECDSA) Alg() string {
+	return m.Name
+}
+
+// Implements the Verify method from SigningMethod
+// For this verify method, key must be an ecdsa.PublicKey struct
+func (m *SigningMethodECDSA) Verify(signingString, signature string, key interface{}) error {
+	var err error
+
+	// Decode the signature
+	var sig []byte
+	if sig, err = DecodeSegment(signature); err != nil {
+		return err
+	}
+
+	// Get the key
+	var ecdsaKey *ecdsa.PublicKey
+	switch k := key.(type) {
+	case *ecdsa.PublicKey:
+		ecdsaKey = k
+	default:
+		return ErrInvalidKeyType
+	}
+
+	if len(sig) != 2*m.KeySize {
+		return ErrECDSAVerification
+	}
+
+	r := big.NewInt(0).SetBytes(sig[:m.KeySize])
+	s := big.NewInt(0).SetBytes(sig[m.KeySize:])
+
+	// Create hasher
+	if !m.Hash.Available() {
+		return ErrHashUnavailable
+	}
+	hasher := m.Hash.New()
+	hasher.Write([]byte(signingString))
+
+	// Verify the signature
+	if verifystatus := ecdsa.Verify(ecdsaKey, hasher.Sum(nil), r, s); verifystatus == true {
+		return nil
+	} else {
+		return ErrECDSAVerification
+	}
+}
+
+// Implements the Sign method from SigningMethod
+// For this signing method, key must be an ecdsa.PrivateKey struct
+func (m *SigningMethodECDSA) Sign(signingString string, key interface{}) (string, error) {
+	// Get the key
+	var ecdsaKey *ecdsa.PrivateKey
+	switch k := key.(type) {
+	case *ecdsa.PrivateKey:
+		ecdsaKey = k
+	default:
+		return "", ErrInvalidKeyType
+	}
+
+	// Create the hasher
+	if !m.Hash.Available() {
+		return "", ErrHashUnavailable
+	}
+
+	hasher := m.Hash.New()
+	hasher.Write([]byte(signingString))
+
+	// Sign the string and return r, s
+	if r, s, err := ecdsa.Sign(rand.Reader, ecdsaKey, hasher.Sum(nil)); err == nil {
+		curveBits := ecdsaKey.Curve.Params().BitSize
+
+		if m.CurveBits != curveBits {
+			return "", ErrInvalidKey
+		}
+
+		keyBytes := curveBits / 8
+		if curveBits%8 > 0 {
+			keyBytes += 1
+		}
+
+		// We serialize the outpus (r and s) into big-endian byte arrays and pad
+		// them with zeros on the left to make sure the sizes work out. Both arrays
+		// must be keyBytes long, and the output must be 2*keyBytes long.
+		rBytes := r.Bytes()
+		rBytesPadded := make([]byte, keyBytes)
+		copy(rBytesPadded[keyBytes-len(rBytes):], rBytes)
+
+		sBytes := s.Bytes()
+		sBytesPadded := make([]byte, keyBytes)
+		copy(sBytesPadded[keyBytes-len(sBytes):], sBytes)
+
+		out := append(rBytesPadded, sBytesPadded...)
+
+		return EncodeSegment(out), nil
+	} else {
+		return "", err
+	}
+}
diff --git a/vendor/github.com/dgrijalva/jwt-go/ecdsa_utils.go b/vendor/github.com/dgrijalva/jwt-go/ecdsa_utils.go
new file mode 100644
index 0000000..d19624b
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/ecdsa_utils.go
@@ -0,0 +1,67 @@
+package jwt
+
+import (
+	"crypto/ecdsa"
+	"crypto/x509"
+	"encoding/pem"
+	"errors"
+)
+
+var (
+	ErrNotECPublicKey  = errors.New("Key is not a valid ECDSA public key")
+	ErrNotECPrivateKey = errors.New("Key is not a valid ECDSA private key")
+)
+
+// Parse PEM encoded Elliptic Curve Private Key Structure
+func ParseECPrivateKeyFromPEM(key []byte) (*ecdsa.PrivateKey, error) {
+	var err error
+
+	// Parse PEM block
+	var block *pem.Block
+	if block, _ = pem.Decode(key); block == nil {
+		return nil, ErrKeyMustBePEMEncoded
+	}
+
+	// Parse the key
+	var parsedKey interface{}
+	if parsedKey, err = x509.ParseECPrivateKey(block.Bytes); err != nil {
+		return nil, err
+	}
+
+	var pkey *ecdsa.PrivateKey
+	var ok bool
+	if pkey, ok = parsedKey.(*ecdsa.PrivateKey); !ok {
+		return nil, ErrNotECPrivateKey
+	}
+
+	return pkey, nil
+}
+
+// Parse PEM encoded PKCS1 or PKCS8 public key
+func ParseECPublicKeyFromPEM(key []byte) (*ecdsa.PublicKey, error) {
+	var err error
+
+	// Parse PEM block
+	var block *pem.Block
+	if block, _ = pem.Decode(key); block == nil {
+		return nil, ErrKeyMustBePEMEncoded
+	}
+
+	// Parse the key
+	var parsedKey interface{}
+	if parsedKey, err = x509.ParsePKIXPublicKey(block.Bytes); err != nil {
+		if cert, err := x509.ParseCertificate(block.Bytes); err == nil {
+			parsedKey = cert.PublicKey
+		} else {
+			return nil, err
+		}
+	}
+
+	var pkey *ecdsa.PublicKey
+	var ok bool
+	if pkey, ok = parsedKey.(*ecdsa.PublicKey); !ok {
+		return nil, ErrNotECPublicKey
+	}
+
+	return pkey, nil
+}
diff --git a/vendor/github.com/dgrijalva/jwt-go/errors.go b/vendor/github.com/dgrijalva/jwt-go/errors.go
new file mode 100644
index 0000000..1c93024
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/errors.go
@@ -0,0 +1,59 @@
+package jwt
+
+import (
+	"errors"
+)
+
+// Error constants
+var (
+	ErrInvalidKey      = errors.New("key is invalid")
+	ErrInvalidKeyType  = errors.New("key is of invalid type")
+	ErrHashUnavailable = errors.New("the requested hash function is unavailable")
+)
+
+// The errors that might occur when parsing and validating a token
+const (
+	ValidationErrorMalformed        uint32 = 1 << iota // Token is malformed
+	ValidationErrorUnverifiable                        // Token could not be verified because of signing problems
+	ValidationErrorSignatureInvalid                    // Signature validation failed
+
+	// Standard Claim validation errors
+	ValidationErrorAudience      // AUD validation failed
+	ValidationErrorExpired       // EXP validation failed
+	ValidationErrorIssuedAt      // IAT validation failed
+	ValidationErrorIssuer        // ISS validation failed
+	ValidationErrorNotValidYet   // NBF validation failed
+	ValidationErrorId            // JTI validation failed
+	ValidationErrorClaimsInvalid // Generic claims validation error
+)
+
+// Helper for constructing a ValidationError with a string error message
+func NewValidationError(errorText string, errorFlags uint32) *ValidationError {
+	return &ValidationError{
+		text:   errorText,
+		Errors: errorFlags,
+	}
+}
+
+// The error from Parse if token is not valid
+type ValidationError struct {
+	Inner  error  // stores the error returned by external dependencies, i.e.: KeyFunc
+	Errors uint32 // bitfield.  see ValidationError... constants
+	text   string // errors that do not have a valid error just have text
+}
+
+// Validation error is an error type
+func (e ValidationError) Error() string {
+	if e.Inner != nil {
+		return e.Inner.Error()
+	} else if e.text != "" {
+		return e.text
+	} else {
+		return "token is invalid"
+	}
+}
+
+// No errors
+func (e *ValidationError) valid() bool {
+	return e.Errors == 0
+}
diff --git a/vendor/github.com/dgrijalva/jwt-go/hmac.go b/vendor/github.com/dgrijalva/jwt-go/hmac.go
new file mode 100644
index 0000000..addbe5d
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/hmac.go
@@ -0,0 +1,95 @@
+package jwt
+
+import (
+	"crypto"
+	"crypto/hmac"
+	"errors"
+)
+
+// Implements the HMAC-SHA family of signing methods signing methods
+// Expects key type of []byte for both signing and validation
+type SigningMethodHMAC struct {
+	Name string
+	Hash crypto.Hash
+}
+
+// Specific instances for HS256 and company
+var (
+	SigningMethodHS256  *SigningMethodHMAC
+	SigningMethodHS384  *SigningMethodHMAC
+	SigningMethodHS512  *SigningMethodHMAC
+	ErrSignatureInvalid = errors.New("signature is invalid")
+)
+
+func init() {
+	// HS256
+	SigningMethodHS256 = &SigningMethodHMAC{"HS256", crypto.SHA256}
+	RegisterSigningMethod(SigningMethodHS256.Alg(), func() SigningMethod {
+		return SigningMethodHS256
+	})
+
+	// HS384
+	SigningMethodHS384 = &SigningMethodHMAC{"HS384", crypto.SHA384}
+	RegisterSigningMethod(SigningMethodHS384.Alg(), func() SigningMethod {
+		return SigningMethodHS384
+	})
+
+	// HS512
+	SigningMethodHS512 = &SigningMethodHMAC{"HS512", crypto.SHA512}
+	RegisterSigningMethod(SigningMethodHS512.Alg(), func() SigningMethod {
+		return SigningMethodHS512
+	})
+}
+
+func (m *SigningMethodHMAC) Alg() string {
+	return m.Name
+}
+
+// Verify the signature of HSXXX tokens.  Returns nil if the signature is valid.
+func (m *SigningMethodHMAC) Verify(signingString, signature string, key interface{}) error {
+	// Verify the key is the right type
+	keyBytes, ok := key.([]byte)
+	if !ok {
+		return ErrInvalidKeyType
+	}
+
+	// Decode signature, for comparison
+	sig, err := DecodeSegment(signature)
+	if err != nil {
+		return err
+	}
+
+	// Can we use the specified hashing method?
+	if !m.Hash.Available() {
+		return ErrHashUnavailable
+	}
+
+	// This signing method is symmetric, so we validate the signature
+	// by reproducing the signature from the signing string and key, then
+	// comparing that against the provided signature.
+	hasher := hmac.New(m.Hash.New, keyBytes)
+	hasher.Write([]byte(signingString))
+	if !hmac.Equal(sig, hasher.Sum(nil)) {
+		return ErrSignatureInvalid
+	}
+
+	// No validation errors.  Signature is good.
+	return nil
+}
+
+// Implements the Sign method from SigningMethod for this signing method.
+// Key must be []byte
+func (m *SigningMethodHMAC) Sign(signingString string, key interface{}) (string, error) {
+	if keyBytes, ok := key.([]byte); ok {
+		if !m.Hash.Available() {
+			return "", ErrHashUnavailable
+		}
+
+		hasher := hmac.New(m.Hash.New, keyBytes)
+		hasher.Write([]byte(signingString))
+
+		return EncodeSegment(hasher.Sum(nil)), nil
+	}
+
+	return "", ErrInvalidKeyType
+}
diff --git a/vendor/github.com/dgrijalva/jwt-go/map_claims.go b/vendor/github.com/dgrijalva/jwt-go/map_claims.go
new file mode 100644
index 0000000..291213c
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/map_claims.go
@@ -0,0 +1,94 @@
+package jwt
+
+import (
+	"encoding/json"
+	"errors"
+	// "fmt"
+)
+
+// Claims type that uses the map[string]interface{} for JSON decoding
+// This is the default claims type if you don't supply one
+type MapClaims map[string]interface{}
+
+// Compares the aud claim against cmp.
+// If required is false, this method will return true if the value matches or is unset
+func (m MapClaims) VerifyAudience(cmp string, req bool) bool {
+	aud, _ := m["aud"].(string)
+	return verifyAud(aud, cmp, req)
+}
+
+// Compares the exp claim against cmp.
+// If required is false, this method will return true if the value matches or is unset
+func (m MapClaims) VerifyExpiresAt(cmp int64, req bool) bool {
+	switch exp := m["exp"].(type) {
+	case float64:
+		return verifyExp(int64(exp), cmp, req)
+	case json.Number:
+		v, _ := exp.Int64()
+		return verifyExp(v, cmp, req)
+	}
+	return req == false
+}
+
+// Compares the iat claim against cmp.
+// If required is false, this method will return true if the value matches or is unset
+func (m MapClaims) VerifyIssuedAt(cmp int64, req bool) bool {
+	switch iat := m["iat"].(type) {
+	case float64:
+		return verifyIat(int64(iat), cmp, req)
+	case json.Number:
+		v, _ := iat.Int64()
+		return verifyIat(v, cmp, req)
+	}
+	return req == false
+}
+
+// Compares the iss claim against cmp.
+// If required is false, this method will return true if the value matches or is unset
+func (m MapClaims) VerifyIssuer(cmp string, req bool) bool {
+	iss, _ := m["iss"].(string)
+	return verifyIss(iss, cmp, req)
+}
+
+// Compares the nbf claim against cmp.
+// If required is false, this method will return true if the value matches or is unset
+func (m MapClaims) VerifyNotBefore(cmp int64, req bool) bool {
+	switch nbf := m["nbf"].(type) {
+	case float64:
+		return verifyNbf(int64(nbf), cmp, req)
+	case json.Number:
+		v, _ := nbf.Int64()
+		return verifyNbf(v, cmp, req)
+	}
+	return req == false
+}
+
+// Validates time based claims "exp, iat, nbf".
+// There is no accounting for clock skew.
+// As well, if any of the above claims are not in the token, it will still
+// be considered a valid claim.
+func (m MapClaims) Valid() error {
+	vErr := new(ValidationError)
+	now := TimeFunc().Unix()
+
+	if m.VerifyExpiresAt(now, false) == false {
+		vErr.Inner = errors.New("Token is expired")
+		vErr.Errors |= ValidationErrorExpired
+	}
+
+	if m.VerifyIssuedAt(now, false) == false {
+		vErr.Inner = errors.New("Token used before issued")
+		vErr.Errors |= ValidationErrorIssuedAt
+	}
+
+	if m.VerifyNotBefore(now, false) == false {
+		vErr.Inner = errors.New("Token is not valid yet")
+		vErr.Errors |= ValidationErrorNotValidYet
+	}
+
+	if vErr.valid() {
+		return nil
+	}
+
+	return vErr
+}
diff --git a/vendor/github.com/dgrijalva/jwt-go/none.go b/vendor/github.com/dgrijalva/jwt-go/none.go
new file mode 100644
index 0000000..f04d189
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/none.go
@@ -0,0 +1,52 @@
+package jwt
+
+// Implements the none signing method.  This is required by the spec
+// but you probably should never use it.
+var SigningMethodNone *signingMethodNone
+
+const UnsafeAllowNoneSignatureType unsafeNoneMagicConstant = "none signing method allowed"
+
+var NoneSignatureTypeDisallowedError error
+
+type signingMethodNone struct{}
+type unsafeNoneMagicConstant string
+
+func init() {
+	SigningMethodNone = &signingMethodNone{}
+	NoneSignatureTypeDisallowedError = NewValidationError("'none' signature type is not allowed", ValidationErrorSignatureInvalid)
+
+	RegisterSigningMethod(SigningMethodNone.Alg(), func() SigningMethod {
+		return SigningMethodNone
+	})
+}
+
+func (m *signingMethodNone) Alg() string {
+	return "none"
+}
+
+// Only allow 'none' alg type if UnsafeAllowNoneSignatureType is specified as the key
+func (m *signingMethodNone) Verify(signingString, signature string, key interface{}) (err error) {
+	// Key must be UnsafeAllowNoneSignatureType to prevent accidentally
+	// accepting 'none' signing method
+	if _, ok := key.(unsafeNoneMagicConstant); !ok {
+		return NoneSignatureTypeDisallowedError
+	}
+	// If signing method is none, signature must be an empty string
+	if signature != "" {
+		return NewValidationError(
+			"'none' signing method with non-empty signature",
+			ValidationErrorSignatureInvalid,
+		)
+	}
+
+	// Accept 'none' signing method.
+	return nil
+}
+
+// Only allow 'none' signing if UnsafeAllowNoneSignatureType is specified as the key
+func (m *signingMethodNone) Sign(signingString string, key interface{}) (string, error) {
+	if _, ok := key.(unsafeNoneMagicConstant); ok {
+		return "", nil
+	}
+	return "", NoneSignatureTypeDisallowedError
+}
diff --git a/vendor/github.com/dgrijalva/jwt-go/parser.go b/vendor/github.com/dgrijalva/jwt-go/parser.go
new file mode 100644
index 0000000..d6901d9
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/parser.go
@@ -0,0 +1,148 @@
+package jwt
+
+import (
+	"bytes"
+	"encoding/json"
+	"fmt"
+	"strings"
+)
+
+type Parser struct {
+	ValidMethods         []string // If populated, only these methods will be considered valid
+	UseJSONNumber        bool     // Use JSON Number format in JSON decoder
+	SkipClaimsValidation bool     // Skip claims validation during token parsing
+}
+
+// Parse, validate, and return a token.
+// keyFunc will receive the parsed token and should return the key for validating.
+// If everything is kosher, err will be nil
+func (p *Parser) Parse(tokenString string, keyFunc Keyfunc) (*Token, error) {
+	return p.ParseWithClaims(tokenString, MapClaims{}, keyFunc)
+}
+
+func (p *Parser) ParseWithClaims(tokenString string, claims Claims, keyFunc Keyfunc) (*Token, error) {
+	token, parts, err := p.ParseUnverified(tokenString, claims)
+	if err != nil {
+		return token, err
+	}
+
+	// Verify signing method is in the required set
+	if p.ValidMethods != nil {
+		var signingMethodValid = false
+		var alg = token.Method.Alg()
+		for _, m := range p.ValidMethods {
+			if m == alg {
+				signingMethodValid = true
+				break
+			}
+		}
+		if !signingMethodValid {
+			// signing method is not in the listed set
+			return token, NewValidationError(fmt.Sprintf("signing method %v is invalid", alg), ValidationErrorSignatureInvalid)
+		}
+	}
+
+	// Lookup key
+	var key interface{}
+	if keyFunc == nil {
+		// keyFunc was not provided.  short circuiting validation
+		return token, NewValidationError("no Keyfunc was provided.", ValidationErrorUnverifiable)
+	}
+	if key, err = keyFunc(token); err != nil {
+		// keyFunc returned an error
+		if ve, ok := err.(*ValidationError); ok {
+			return token, ve
+		}
+		return token, &ValidationError{Inner: err, Errors: ValidationErrorUnverifiable}
+	}
+
+	vErr := &ValidationError{}
+
+	// Validate Claims
+	if !p.SkipClaimsValidation {
+		if err := token.Claims.Valid(); err != nil {
+
+			// If the Claims Valid returned an error, check if it is a validation error,
+			// If it was another error type, create a ValidationError with a generic ClaimsInvalid flag set
+			if e, ok := err.(*ValidationError); !ok {
+				vErr = &ValidationError{Inner: err, Errors: ValidationErrorClaimsInvalid}
+			} else {
+				vErr = e
+			}
+		}
+	}
+
+	// Perform validation
+	token.Signature = parts[2]
+	if err = token.Method.Verify(strings.Join(parts[0:2], "."), token.Signature, key); err != nil {
+		vErr.Inner = err
+		vErr.Errors |= ValidationErrorSignatureInvalid
+	}
+
+	if vErr.valid() {
+		token.Valid = true
+		return token, nil
+	}
+
+	return token, vErr
+}
+
+// WARNING: Don't use this method unless you know what you're doing
+//
+// This method parses the token but doesn't validate the signature. It's only
+// ever useful in cases where you know the signature is valid (because it has
+// been checked previously in the stack) and you want to extract values from
+// it.
+func (p *Parser) ParseUnverified(tokenString string, claims Claims) (token *Token, parts []string, err error) {
+	parts = strings.Split(tokenString, ".")
+	if len(parts) != 3 {
+		return nil, parts, NewValidationError("token contains an invalid number of segments", ValidationErrorMalformed)
+	}
+
+	token = &Token{Raw: tokenString}
+
+	// parse Header
+	var headerBytes []byte
+	if headerBytes, err = DecodeSegment(parts[0]); err != nil {
+		if strings.HasPrefix(strings.ToLower(tokenString), "bearer ") {
+			return token, parts, NewValidationError("tokenstring should not contain 'bearer '", ValidationErrorMalformed)
+		}
+		return token, parts, &ValidationError{Inner: err, Errors: ValidationErrorMalformed}
+	}
+	if err = json.Unmarshal(headerBytes, &token.Header); err != nil {
+		return token, parts, &ValidationError{Inner: err, Errors: ValidationErrorMalformed}
+	}
+
+	// parse Claims
+	var claimBytes []byte
+	token.Claims = claims
+
+	if claimBytes, err = DecodeSegment(parts[1]); err != nil {
+		return token, parts, &ValidationError{Inner: err, Errors: ValidationErrorMalformed}
+	}
+	dec := json.NewDecoder(bytes.NewBuffer(claimBytes))
+	if p.UseJSONNumber {
+		dec.UseNumber()
+	}
+	// JSON Decode.  Special case for map type to avoid weird pointer behavior
+	if c, ok := token.Claims.(MapClaims); ok {
+		err = dec.Decode(&c)
+	} else {
+		err = dec.Decode(&claims)
+	}
+	// Handle decode error
+	if err != nil {
+		return token, parts, &ValidationError{Inner: err, Errors: ValidationErrorMalformed}
+	}
+
+	// Lookup signature method
+	if method, ok := token.Header["alg"].(string); ok {
+		if token.Method = GetSigningMethod(method); token.Method == nil {
+			return token, parts, NewValidationError("signing method (alg) is unavailable.", ValidationErrorUnverifiable)
+		}
+	} else {
+		return token, parts, NewValidationError("signing method (alg) is unspecified.", ValidationErrorUnverifiable)
+	}
+
+	return token, parts, nil
+}
diff --git a/vendor/github.com/dgrijalva/jwt-go/rsa.go b/vendor/github.com/dgrijalva/jwt-go/rsa.go
new file mode 100644
index 0000000..e4caf1c
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/rsa.go
@@ -0,0 +1,101 @@
+package jwt
+
+import (
+	"crypto"
+	"crypto/rand"
+	"crypto/rsa"
+)
+
+// Implements the RSA family of signing methods signing methods
+// Expects *rsa.PrivateKey for signing and *rsa.PublicKey for validation
+type SigningMethodRSA struct {
+	Name string
+	Hash crypto.Hash
+}
+
+// Specific instances for RS256 and company
+var (
+	SigningMethodRS256 *SigningMethodRSA
+	SigningMethodRS384 *SigningMethodRSA
+	SigningMethodRS512 *SigningMethodRSA
+)
+
+func init() {
+	// RS256
+	SigningMethodRS256 = &SigningMethodRSA{"RS256", crypto.SHA256}
+	RegisterSigningMethod(SigningMethodRS256.Alg(), func() SigningMethod {
+		return SigningMethodRS256
+	})
+
+	// RS384
+	SigningMethodRS384 = &SigningMethodRSA{"RS384", crypto.SHA384}
+	RegisterSigningMethod(SigningMethodRS384.Alg(), func() SigningMethod {
+		return SigningMethodRS384
+	})
+
+	// RS512
+	SigningMethodRS512 = &SigningMethodRSA{"RS512", crypto.SHA512}
+	RegisterSigningMethod(SigningMethodRS512.Alg(), func() SigningMethod {
+		return SigningMethodRS512
+	})
+}
+
+func (m *SigningMethodRSA) Alg() string {
+	return m.Name
+}
+
+// Implements the Verify method from SigningMethod
+// For this signing method, must be an *rsa.PublicKey structure.
+func (m *SigningMethodRSA) Verify(signingString, signature string, key interface{}) error {
+	var err error
+
+	// Decode the signature
+	var sig []byte
+	if sig, err = DecodeSegment(signature); err != nil {
+		return err
+	}
+
+	var rsaKey *rsa.PublicKey
+	var ok bool
+
+	if rsaKey, ok = key.(*rsa.PublicKey); !ok {
+		return ErrInvalidKeyType
+	}
+
+	// Create hasher
+	if !m.Hash.Available() {
+		return ErrHashUnavailable
+	}
+	hasher := m.Hash.New()
+	hasher.Write([]byte(signingString))
+
+	// Verify the signature
+	return rsa.VerifyPKCS1v15(rsaKey, m.Hash, hasher.Sum(nil), sig)
+}
+
+// Implements the Sign method from SigningMethod
+// For this signing method, must be an *rsa.PrivateKey structure.
+func (m *SigningMethodRSA) Sign(signingString string, key interface{}) (string, error) {
+	var rsaKey *rsa.PrivateKey
+	var ok bool
+
+	// Validate type of key
+	if rsaKey, ok = key.(*rsa.PrivateKey); !ok {
+		return "", ErrInvalidKey
+	}
+
+	// Create the hasher
+	if !m.Hash.Available() {
+		return "", ErrHashUnavailable
+	}
+
+	hasher := m.Hash.New()
+	hasher.Write([]byte(signingString))
+
+	// Sign the string and return the encoded bytes
+	if sigBytes, err := rsa.SignPKCS1v15(rand.Reader, rsaKey, m.Hash, hasher.Sum(nil)); err == nil {
+		return EncodeSegment(sigBytes), nil
+	} else {
+		return "", err
+	}
+}
diff --git a/vendor/github.com/dgrijalva/jwt-go/rsa_pss.go b/vendor/github.com/dgrijalva/jwt-go/rsa_pss.go
new file mode 100644
index 0000000..10ee9db
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/rsa_pss.go
@@ -0,0 +1,126 @@
+// +build go1.4
+
+package jwt
+
+import (
+	"crypto"
+	"crypto/rand"
+	"crypto/rsa"
+)
+
+// Implements the RSAPSS family of signing methods signing methods
+type SigningMethodRSAPSS struct {
+	*SigningMethodRSA
+	Options *rsa.PSSOptions
+}
+
+// Specific instances for RS/PS and company
+var (
+	SigningMethodPS256 *SigningMethodRSAPSS
+	SigningMethodPS384 *SigningMethodRSAPSS
+	SigningMethodPS512 *SigningMethodRSAPSS
+)
+
+func init() {
+	// PS256
+	SigningMethodPS256 = &SigningMethodRSAPSS{
+		&SigningMethodRSA{
+			Name: "PS256",
+			Hash: crypto.SHA256,
+		},
+		&rsa.PSSOptions{
+			SaltLength: rsa.PSSSaltLengthAuto,
+			Hash:       crypto.SHA256,
+		},
+	}
+	RegisterSigningMethod(SigningMethodPS256.Alg(), func() SigningMethod {
+		return SigningMethodPS256
+	})
+
+	// PS384
+	SigningMethodPS384 = &SigningMethodRSAPSS{
+		&SigningMethodRSA{
+			Name: "PS384",
+			Hash: crypto.SHA384,
+		},
+		&rsa.PSSOptions{
+			SaltLength: rsa.PSSSaltLengthAuto,
+			Hash:       crypto.SHA384,
+		},
+	}
+	RegisterSigningMethod(SigningMethodPS384.Alg(), func() SigningMethod {
+		return SigningMethodPS384
+	})
+
+	// PS512
+	SigningMethodPS512 = &SigningMethodRSAPSS{
+		&SigningMethodRSA{
+			Name: "PS512",
+			Hash: crypto.SHA512,
+		},
+		&rsa.PSSOptions{
+			SaltLength: rsa.PSSSaltLengthAuto,
+			Hash:       crypto.SHA512,
+		},
+	}
+	RegisterSigningMethod(SigningMethodPS512.Alg(), func() SigningMethod {
+		return SigningMethodPS512
+	})
+}
+
+// Implements the Verify method from SigningMethod
+// For this verify method, key must be an rsa.PublicKey struct
+func (m *SigningMethodRSAPSS) Verify(signingString, signature string, key interface{}) error {
+	var err error
+
+	// Decode the signature
+	var sig []byte
+	if sig, err = DecodeSegment(signature); err != nil {
+		return err
+	}
+
+	var rsaKey *rsa.PublicKey
+	switch k := key.(type) {
+	case *rsa.PublicKey:
+		rsaKey = k
+	default:
+		return ErrInvalidKey
+	}
+
+	// Create hasher
+	if !m.Hash.Available() {
+		return ErrHashUnavailable
+	}
+	hasher := m.Hash.New()
+	hasher.Write([]byte(signingString))
+
+	return rsa.VerifyPSS(rsaKey, m.Hash, hasher.Sum(nil), sig, m.Options)
+}
+
+// Implements the Sign method from SigningMethod
+// For this signing method, key must be an rsa.PrivateKey struct
+func (m *SigningMethodRSAPSS) Sign(signingString string, key interface{}) (string, error) {
+	var rsaKey *rsa.PrivateKey
+
+	switch k := key.(type) {
+	case *rsa.PrivateKey:
+		rsaKey = k
+	default:
+		return "", ErrInvalidKeyType
+	}
+
+	// Create the hasher
+	if !m.Hash.Available() {
+		return "", ErrHashUnavailable
+	}
+
+	hasher := m.Hash.New()
+	hasher.Write([]byte(signingString))
+
+	// Sign the string and return the encoded bytes
+	if sigBytes, err := rsa.SignPSS(rand.Reader, rsaKey, m.Hash, hasher.Sum(nil), m.Options); err == nil {
+		return EncodeSegment(sigBytes), nil
+	} else {
+		return "", err
+	}
+}
diff --git a/vendor/github.com/dgrijalva/jwt-go/rsa_utils.go b/vendor/github.com/dgrijalva/jwt-go/rsa_utils.go
new file mode 100644
index 0000000..a5ababf
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/rsa_utils.go
@@ -0,0 +1,101 @@
+package jwt
+
+import (
+	"crypto/rsa"
+	"crypto/x509"
+	"encoding/pem"
+	"errors"
+)
+
+var (
+	ErrKeyMustBePEMEncoded = errors.New("Invalid Key: Key must be PEM encoded PKCS1 or PKCS8 private key")
+	ErrNotRSAPrivateKey    = errors.New("Key is not a valid RSA private key")
+	ErrNotRSAPublicKey     = errors.New("Key is not a valid RSA public key")
+)
+
+// Parse PEM encoded PKCS1 or PKCS8 private key
+func ParseRSAPrivateKeyFromPEM(key []byte) (*rsa.PrivateKey, error) {
+	var err error
+
+	// Parse PEM block
+	var block *pem.Block
+	if block, _ = pem.Decode(key); block == nil {
+		return nil, ErrKeyMustBePEMEncoded
+	}
+
+	var parsedKey interface{}
+	if parsedKey, err = x509.ParsePKCS1PrivateKey(block.Bytes); err != nil {
+		if parsedKey, err = x509.ParsePKCS8PrivateKey(block.Bytes); err != nil {
+			return nil, err
+		}
+	}
+
+	var pkey *rsa.PrivateKey
+	var ok bool
+	if pkey, ok = parsedKey.(*rsa.PrivateKey); !ok {
+		return nil, ErrNotRSAPrivateKey
+	}
+
+	return pkey, nil
+}
+
+// Parse PEM encoded PKCS1 or PKCS8 private key protected with password
+func ParseRSAPrivateKeyFromPEMWithPassword(key []byte, password string) (*rsa.PrivateKey, error) {
+	var err error
+
+	// Parse PEM block
+	var block *pem.Block
+	if block, _ = pem.Decode(key); block == nil {
+		return nil, ErrKeyMustBePEMEncoded
+	}
+
+	var parsedKey interface{}
+
+	var blockDecrypted []byte
+	if blockDecrypted, err = x509.DecryptPEMBlock(block, []byte(password)); err != nil {
+		return nil, err
+	}
+
+	if parsedKey, err = x509.ParsePKCS1PrivateKey(blockDecrypted); err != nil {
+		if parsedKey, err = x509.ParsePKCS8PrivateKey(blockDecrypted); err != nil {
+			return nil, err
+		}
+	}
+
+	var pkey *rsa.PrivateKey
+	var ok bool
+	if pkey, ok = parsedKey.(*rsa.PrivateKey); !ok {
+		return nil, ErrNotRSAPrivateKey
+	}
+
+	return pkey, nil
+}
+
+// Parse PEM encoded PKCS1 or PKCS8 public key
+func ParseRSAPublicKeyFromPEM(key []byte) (*rsa.PublicKey, error) {
+	var err error
+
+	// Parse PEM block
+	var block *pem.Block
+	if block, _ = pem.Decode(key); block == nil {
+		return nil, ErrKeyMustBePEMEncoded
+	}
+
+	// Parse the key
+	var parsedKey interface{}
+	if parsedKey, err = x509.ParsePKIXPublicKey(block.Bytes); err != nil {
+		if cert, err := x509.ParseCertificate(block.Bytes); err == nil {
+			parsedKey = cert.PublicKey
+		} else {
+			return nil, err
+		}
+	}
+
+	var pkey *rsa.PublicKey
+	var ok bool
+	if pkey, ok = parsedKey.(*rsa.PublicKey); !ok {
+		return nil, ErrNotRSAPublicKey
+	}
+
+	return pkey, nil
+}
diff --git a/vendor/github.com/dgrijalva/jwt-go/signing_method.go b/vendor/github.com/dgrijalva/jwt-go/signing_method.go
new file mode 100644
index 0000000..ed1f212
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/signing_method.go
@@ -0,0 +1,35 @@
+package jwt
+
+import (
+	"sync"
+)
+
+var signingMethods = map[string]func() SigningMethod{}
+var signingMethodLock = new(sync.RWMutex)
+
+// Implement SigningMethod to add new methods for signing or verifying tokens.
+type SigningMethod interface {
+	Verify(signingString, signature string, key interface{}) error // Returns nil if signature is valid
+	Sign(signingString string, key interface{}) (string, error)    // Returns encoded signature or error
+	Alg() string                                                   // returns the alg identifier for this method (example: 'HS256')
+}
+
+// Register the "alg" name and a factory function for signing method.
+// This is typically done during init() in the method's implementation
+func RegisterSigningMethod(alg string, f func() SigningMethod) {
+	signingMethodLock.Lock()
+	defer signingMethodLock.Unlock()
+
+	signingMethods[alg] = f
+}
+
+// Get a signing method from an "alg" string
+func GetSigningMethod(alg string) (method SigningMethod) {
+	signingMethodLock.RLock()
+	defer signingMethodLock.RUnlock()
+
+	if methodF, ok := signingMethods[alg]; ok {
+		method = methodF()
+	}
+	return
+}
diff --git a/vendor/github.com/dgrijalva/jwt-go/token.go b/vendor/github.com/dgrijalva/jwt-go/token.go
new file mode 100644
index 0000000..d637e08
--- /dev/null
+++ b/vendor/github.com/dgrijalva/jwt-go/token.go
@@ -0,0 +1,108 @@
+package jwt
+
+import (
+	"encoding/base64"
+	"encoding/json"
+	"strings"
+	"time"
+)
+
+// TimeFunc provides the current time when parsing token to validate "exp" claim (expiration time).
+// You can override it to use another time value.  This is useful for testing or if your
+// server uses a different time zone than your tokens.
+var TimeFunc = time.Now
+
+// Parse methods use this callback function to supply
+// the key for verification.  The function receives the parsed,
+// but unverified Token.  This allows you to use properties in the
+// Header of the token (such as `kid`) to identify which key to use.
+type Keyfunc func(*Token) (interface{}, error)
+
+// A JWT Token.  Different fields will be used depending on whether you're
+// creating or parsing/verifying a token.
+type Token struct {
+	Raw       string                 // The raw token.  Populated when you Parse a token
+	Method    SigningMethod          // The signing method used or to be used
+	Header    map[string]interface{} // The first segment of the token
+	Claims    Claims                 // The second segment of the token
+	Signature string                 // The third segment of the token.  Populated when you Parse a token
+	Valid     bool                   // Is the token valid?  Populated when you Parse/Verify a token
+}
+
+// Create a new Token.  Takes a signing method
+func New(method SigningMethod) *Token {
+	return NewWithClaims(method, MapClaims{})
+}
+
+func NewWithClaims(method SigningMethod, claims Claims) *Token {
+	return &Token{
+		Header: map[string]interface{}{
+			"typ": "JWT",
+			"alg": method.Alg(),
+		},
+		Claims: claims,
+		Method: method,
+	}
+}
+
+// Get the complete, signed token
+func (t *Token) SignedString(key interface{}) (string, error) {
+	var sig, sstr string
+	var err error
+	if sstr, err = t.SigningString(); err != nil {
+		return "", err
+	}
+	if sig, err = t.Method.Sign(sstr, key); err != nil {
+		return "", err
+	}
+	return strings.Join([]string{sstr, sig}, "."), nil
+}
+
+// Generate the signing string.  This is the
+// most expensive part of the whole deal.  Unless you
+// need this for something special, just go straight for
+// the SignedString.
+func (t *Token) SigningString() (string, error) {
+	var err error
+	parts := make([]string, 2)
+	for i, _ := range parts {
+		var jsonValue []byte
+		if i == 0 {
+			if jsonValue, err = json.Marshal(t.Header); err != nil {
+				return "", err
+			}
+		} else {
+			if jsonValue, err = json.Marshal(t.Claims); err != nil {
+				return "", err
+			}
+		}
+
+		parts[i] = EncodeSegment(jsonValue)
+	}
+	return strings.Join(parts, "."), nil
+}
+
+// Parse, validate, and return a token.
+// keyFunc will receive the parsed token and should return the key for validating.
+// If everything is kosher, err will be nil
+func Parse(tokenString string, keyFunc Keyfunc) (*Token, error) {
+	return new(Parser).Parse(tokenString, keyFunc)
+}
+
+func ParseWithClaims(tokenString string, claims Claims, keyFunc Keyfunc) (*Token, error) {
+	return new(Parser).ParseWithClaims(tokenString, claims, keyFunc)
+}
+
+// Encode JWT specific base64url encoding with padding stripped
+func EncodeSegment(seg []byte) string {
+	return strings.TrimRight(base64.URLEncoding.EncodeToString(seg), "=")
+}
+
+// Decode JWT specific base64url encoding with padding stripped
+func DecodeSegment(seg string) ([]byte, error) {
+	if l := len(seg) % 4; l > 0 {
+		seg += strings.Repeat("=", 4-l)
+	}
+
+	return base64.URLEncoding.DecodeString(seg)
+}
diff --git a/vendor/github.com/dustin/go-humanize/.travis.yml b/vendor/github.com/dustin/go-humanize/.travis.yml
new file mode 100644
index 0000000..ba95cdd
--- /dev/null
+++ b/vendor/github.com/dustin/go-humanize/.travis.yml
@@ -0,0 +1,21 @@
+sudo: false
+language: go
+go:
+  - 1.3.x
+  - 1.5.x
+  - 1.6.x
+  - 1.7.x
+  - 1.8.x
+  - 1.9.x
+  - master
+matrix:
+  allow_failures:
+    - go: master
+  fast_finish: true
+install:
+  - # Do nothing. This is needed to prevent default install action "go get -t -v ./..." from happening here (we want it to happen inside script step).
+script:
+  - go get -t -v ./...
+  - diff -u <(echo -n) <(gofmt -d -s .)
+  - go tool vet .
+  - go test -v -race ./...
diff --git a/vendor/github.com/dustin/go-humanize/LICENSE b/vendor/github.com/dustin/go-humanize/LICENSE
new file mode 100644
index 0000000..8d9a94a
--- /dev/null
+++ b/vendor/github.com/dustin/go-humanize/LICENSE
@@ -0,0 +1,21 @@
+Copyright (c) 2005-2008  Dustin Sallings <dustin@spy.net>
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+<http://www.opensource.org/licenses/mit-license.php>
diff --git a/vendor/github.com/dustin/go-humanize/README.markdown b/vendor/github.com/dustin/go-humanize/README.markdown
new file mode 100644
index 0000000..91b4ae5
--- /dev/null
+++ b/vendor/github.com/dustin/go-humanize/README.markdown
@@ -0,0 +1,124 @@
+# Humane Units [![Build Status](https://travis-ci.org/dustin/go-humanize.svg?branch=master)](https://travis-ci.org/dustin/go-humanize) [![GoDoc](https://godoc.org/github.com/dustin/go-humanize?status.svg)](https://godoc.org/github.com/dustin/go-humanize)
+
+Just a few functions for helping humanize times and sizes.
+
+`go get` it as `github.com/dustin/go-humanize`, import it as
+`"github.com/dustin/go-humanize"`, use it as `humanize`.
+
+See [godoc](https://godoc.org/github.com/dustin/go-humanize) for
+complete documentation.
+
+## Sizes
+
+This lets you take numbers like `82854982` and convert them to useful
+strings like, `83 MB` or `79 MiB` (whichever you prefer).
+
+Example:
+
+```go
+fmt.Printf("That file is %s.", humanize.Bytes(82854982)) // That file is 83 MB.
+```
+
+## Times
+
+This lets you take a `time.Time` and spit it out in relative terms.
+For example, `12 seconds ago` or `3 days from now`.
+
+Example:
+
+```go
+fmt.Printf("This was touched %s.", humanize.Time(someTimeInstance)) // This was touched 7 hours ago.
+```
+
+Thanks to Kyle Lemons for the time implementation from an IRC
+conversation one day. It's pretty neat.
+
+## Ordinals
+
+From a [mailing list discussion][odisc] where a user wanted to be able
+to label ordinals.
+
+    0 -> 0th
+    1 -> 1st
+    2 -> 2nd
+    3 -> 3rd
+    4 -> 4th
+    [...]
+
+Example:
+
+```go
+fmt.Printf("You're my %s best friend.", humanize.Ordinal(193)) // You are my 193rd best friend.
+```
+
+## Commas
+
+Want to shove commas into numbers? Be my guest.
+
+    0 -> 0
+    100 -> 100
+    1000 -> 1,000
+    1000000000 -> 1,000,000,000
+    -100000 -> -100,000
+
+Example:
+
+```go
+fmt.Printf("You owe $%s.\n", humanize.Comma(6582491)) // You owe $6,582,491.
+```
+
+## Ftoa
+
+Nicer float64 formatter that removes trailing zeros.
+
+```go
+fmt.Printf("%f", 2.24)                // 2.240000
+fmt.Printf("%s", humanize.Ftoa(2.24)) // 2.24
+fmt.Printf("%f", 2.0)                 // 2.000000
+fmt.Printf("%s", humanize.Ftoa(2.0))  // 2
+```
+
+## SI notation
+
+Format numbers with [SI notation][sinotation].
+
+Example:
+
+```go
+humanize.SI(0.00000000223, "M") // 2.23 nM
+```
+
+## English-specific functions
+
+The following functions are in the `humanize/english` subpackage.
+
+### Plurals
+
+Simple English pluralization
+
+```go
+english.PluralWord(1, "object", "") // object
+english.PluralWord(42, "object", "") // objects
+english.PluralWord(2, "bus", "") // buses
+english.PluralWord(99, "locus", "loci") // loci
+
+english.Plural(1, "object", "") // 1 object
+english.Plural(42, "object", "") // 42 objects
+english.Plural(2, "bus", "") // 2 buses
+english.Plural(99, "locus", "loci") // 99 loci
+```
+
+### Word series
+
+Format comma-separated words lists with conjuctions:
+
+```go
+english.WordSeries([]string{"foo"}, "and") // foo
+english.WordSeries([]string{"foo", "bar"}, "and") // foo and bar
+english.WordSeries([]string{"foo", "bar", "baz"}, "and") // foo, bar and baz
+
+english.OxfordWordSeries([]string{"foo", "bar", "baz"}, "and") // foo, bar, and baz
+```
+
+[odisc]: https://groups.google.com/d/topic/golang-nuts/l8NhI74jl-4/discussion
+[sinotation]: http://en.wikipedia.org/wiki/Metric_prefix
diff --git a/vendor/github.com/dustin/go-humanize/big.go b/vendor/github.com/dustin/go-humanize/big.go
new file mode 100644
index 0000000..f49dc33
--- /dev/null
+++ b/vendor/github.com/dustin/go-humanize/big.go
@@ -0,0 +1,31 @@
+package humanize
+
+import (
+	"math/big"
+)
+
+// order of magnitude (to a max order)
+func oomm(n, b *big.Int, maxmag int) (float64, int) {
+	mag := 0
+	m := &big.Int{}
+	for n.Cmp(b) >= 0 {
+		n.DivMod(n, b, m)
+		mag++
+		if mag == maxmag && maxmag >= 0 {
+			break
+		}
+	}
+	return float64(n.Int64()) + (float64(m.Int64()) / float64(b.Int64())), mag
+}
+
+// total order of magnitude
+// (same as above, but with no upper limit)
+func oom(n, b *big.Int) (float64, int) {
+	mag := 0
+	m := &big.Int{}
+	for n.Cmp(b) >= 0 {
+		n.DivMod(n, b, m)
+		mag++
+	}
+	return float64(n.Int64()) + (float64(m.Int64()) / float64(b.Int64())), mag
+}
diff --git a/vendor/github.com/dustin/go-humanize/bigbytes.go b/vendor/github.com/dustin/go-humanize/bigbytes.go
new file mode 100644
index 0000000..1a2bf61
--- /dev/null
+++ b/vendor/github.com/dustin/go-humanize/bigbytes.go
@@ -0,0 +1,173 @@
+package humanize
+
+import (
+	"fmt"
+	"math/big"
+	"strings"
+	"unicode"
+)
+
+var (
+	bigIECExp = big.NewInt(1024)
+
+	// BigByte is one byte in bit.Ints
+	BigByte = big.NewInt(1)
+	// BigKiByte is 1,024 bytes in bit.Ints
+	BigKiByte = (&big.Int{}).Mul(BigByte, bigIECExp)
+	// BigMiByte is 1,024 k bytes in bit.Ints
+	BigMiByte = (&big.Int{}).Mul(BigKiByte, bigIECExp)
+	// BigGiByte is 1,024 m bytes in bit.Ints
+	BigGiByte = (&big.Int{}).Mul(BigMiByte, bigIECExp)
+	// BigTiByte is 1,024 g bytes in bit.Ints
+	BigTiByte = (&big.Int{}).Mul(BigGiByte, bigIECExp)
+	// BigPiByte is 1,024 t bytes in bit.Ints
+	BigPiByte = (&big.Int{}).Mul(BigTiByte, bigIECExp)
+	// BigEiByte is 1,024 p bytes in bit.Ints
+	BigEiByte = (&big.Int{}).Mul(BigPiByte, bigIECExp)
+	// BigZiByte is 1,024 e bytes in bit.Ints
+	BigZiByte = (&big.Int{}).Mul(BigEiByte, bigIECExp)
+	// BigYiByte is 1,024 z bytes in bit.Ints
+	BigYiByte = (&big.Int{}).Mul(BigZiByte, bigIECExp)
+)
+
+var (
+	bigSIExp = big.NewInt(1000)
+
+	// BigSIByte is one SI byte in big.Ints
+	BigSIByte = big.NewInt(1)
+	// BigKByte is 1,000 SI bytes in big.Ints
+	BigKByte = (&big.Int{}).Mul(BigSIByte, bigSIExp)
+	// BigMByte is 1,000 SI k bytes in big.Ints
+	BigMByte = (&big.Int{}).Mul(BigKByte, bigSIExp)
+	// BigGByte is 1,000 SI m bytes in big.Ints
+	BigGByte = (&big.Int{}).Mul(BigMByte, bigSIExp)
+	// BigTByte is 1,000 SI g bytes in big.Ints
+	BigTByte = (&big.Int{}).Mul(BigGByte, bigSIExp)
+	// BigPByte is 1,000 SI t bytes in big.Ints
+	BigPByte = (&big.Int{}).Mul(BigTByte, bigSIExp)
+	// BigEByte is 1,000 SI p bytes in big.Ints
+	BigEByte = (&big.Int{}).Mul(BigPByte, bigSIExp)
+	// BigZByte is 1,000 SI e bytes in big.Ints
+	BigZByte = (&big.Int{}).Mul(BigEByte, bigSIExp)
+	// BigYByte is 1,000 SI z bytes in big.Ints
+	BigYByte = (&big.Int{}).Mul(BigZByte, bigSIExp)
+)
+
+var bigBytesSizeTable = map[string]*big.Int{
+	"b":   BigByte,
+	"kib": BigKiByte,
+	"kb":  BigKByte,
+	"mib": BigMiByte,
+	"mb":  BigMByte,
+	"gib": BigGiByte,
+	"gb":  BigGByte,
+	"tib": BigTiByte,
+	"tb":  BigTByte,
+	"pib": BigPiByte,
+	"pb":  BigPByte,
+	"eib": BigEiByte,
+	"eb":  BigEByte,
+	"zib": BigZiByte,
+	"zb":  BigZByte,
+	"yib": BigYiByte,
+	"yb":  BigYByte,
+	// Without suffix
+	"":   BigByte,
+	"ki": BigKiByte,
+	"k":  BigKByte,
+	"mi": BigMiByte,
+	"m":  BigMByte,
+	"gi": BigGiByte,
+	"g":  BigGByte,
+	"ti": BigTiByte,
+	"t":  BigTByte,
+	"pi": BigPiByte,
+	"p":  BigPByte,
+	"ei": BigEiByte,
+	"e":  BigEByte,
+	"z":  BigZByte,
+	"zi": BigZiByte,
+	"y":  BigYByte,
+	"yi": BigYiByte,
+}
+
+var ten = big.NewInt(10)
+
+func humanateBigBytes(s, base *big.Int, sizes []string) string {
+	if s.Cmp(ten) < 0 {
+		return fmt.Sprintf("%d B", s)
+	}
+	c := (&big.Int{}).Set(s)
+	val, mag := oomm(c, base, len(sizes)-1)
+	suffix := sizes[mag]
+	f := "%.0f %s"
+	if val < 10 {
+		f = "%.1f %s"
+	}
+
+	return fmt.Sprintf(f, val, suffix)
+
+}
+
+// BigBytes produces a human readable representation of an SI size.
+//
+// See also: ParseBigBytes.
+//
+// BigBytes(82854982) -> 83 MB
+func BigBytes(s *big.Int) string {
+	sizes := []string{"B", "kB", "MB", "GB", "TB", "PB", "EB", "ZB", "YB"}
+	return humanateBigBytes(s, bigSIExp, sizes)
+}
+
+// BigIBytes produces a human readable representation of an IEC size.
+//
+// See also: ParseBigBytes.
+//
+// BigIBytes(82854982) -> 79 MiB
+func BigIBytes(s *big.Int) string {
+	sizes := []string{"B", "KiB", "MiB", "GiB", "TiB", "PiB", "EiB", "ZiB", "YiB"}
+	return humanateBigBytes(s, bigIECExp, sizes)
+}
+
+// ParseBigBytes parses a string representation of bytes into the number
+// of bytes it represents.
+//
+// See also: BigBytes, BigIBytes.
+//
+// ParseBigBytes("42 MB") -> 42000000, nil
+// ParseBigBytes("42 mib") -> 44040192, nil
+func ParseBigBytes(s string) (*big.Int, error) {
+	lastDigit := 0
+	hasComma := false
+	for _, r := range s {
+		if !(unicode.IsDigit(r) || r == '.' || r == ',') {
+			break
+		}
+		if r == ',' {
+			hasComma = true
+		}
+		lastDigit++
+	}
+
+	num := s[:lastDigit]
+	if hasComma {
+		num = strings.Replace(num, ",", "", -1)
+	}
+
+	val := &big.Rat{}
+	_, err := fmt.Sscanf(num, "%f", val)
+	if err != nil {
+		return nil, err
+	}
+
+	extra := strings.ToLower(strings.TrimSpace(s[lastDigit:]))
+	if m, ok := bigBytesSizeTable[extra]; ok {
+		mv := (&big.Rat{}).SetInt(m)
+		val.Mul(val, mv)
+		rv := &big.Int{}
+		rv.Div(val.Num(), val.Denom())
+		return rv, nil
+	}
+
+	return nil, fmt.Errorf("unhandled size name: %v", extra)
+}
diff --git a/vendor/github.com/dustin/go-humanize/bytes.go b/vendor/github.com/dustin/go-humanize/bytes.go
new file mode 100644
index 0000000..0b498f4
--- /dev/null
+++ b/vendor/github.com/dustin/go-humanize/bytes.go
@@ -0,0 +1,143 @@
+package humanize
+
+import (
+	"fmt"
+	"math"
+	"strconv"
+	"strings"
+	"unicode"
+)
+
+// IEC Sizes.
+// kibis of bits
+const (
+	Byte = 1 << (iota * 10)
+	KiByte
+	MiByte
+	GiByte
+	TiByte
+	PiByte
+	EiByte
+)
+
+// SI Sizes.
+const (
+	IByte = 1
+	KByte = IByte * 1000
+	MByte = KByte * 1000
+	GByte = MByte * 1000
+	TByte = GByte * 1000
+	PByte = TByte * 1000
+	EByte = PByte * 1000
+)
+
+var bytesSizeTable = map[string]uint64{
+	"b":   Byte,
+	"kib": KiByte,
+	"kb":  KByte,
+	"mib": MiByte,
+	"mb":  MByte,
+	"gib": GiByte,
+	"gb":  GByte,
+	"tib": TiByte,
+	"tb":  TByte,
+	"pib": PiByte,
+	"pb":  PByte,
+	"eib": EiByte,
+	"eb":  EByte,
+	// Without suffix
+	"":   Byte,
+	"ki": KiByte,
+	"k":  KByte,
+	"mi": MiByte,
+	"m":  MByte,
+	"gi": GiByte,
+	"g":  GByte,
+	"ti": TiByte,
+	"t":  TByte,
+	"pi": PiByte,
+	"p":  PByte,
+	"ei": EiByte,
+	"e":  EByte,
+}
+
+func logn(n, b float64) float64 {
+	return math.Log(n) / math.Log(b)
+}
+
+func humanateBytes(s uint64, base float64, sizes []string) string {
+	if s < 10 {
+		return fmt.Sprintf("%d B", s)
+	}
+	e := math.Floor(logn(float64(s), base))
+	suffix := sizes[int(e)]
+	val := math.Floor(float64(s)/math.Pow(base, e)*10+0.5) / 10
+	f := "%.0f %s"
+	if val < 10 {
+		f = "%.1f %s"
+	}
+
+	return fmt.Sprintf(f, val, suffix)
+}
+
+// Bytes produces a human readable representation of an SI size.
+//
+// See also: ParseBytes.
+//
+// Bytes(82854982) -> 83 MB
+func Bytes(s uint64) string {
+	sizes := []string{"B", "kB", "MB", "GB", "TB", "PB", "EB"}
+	return humanateBytes(s, 1000, sizes)
+}
+
+// IBytes produces a human readable representation of an IEC size.
+//
+// See also: ParseBytes.
+//
+// IBytes(82854982) -> 79 MiB
+func IBytes(s uint64) string {
+	sizes := []string{"B", "KiB", "MiB", "GiB", "TiB", "PiB", "EiB"}
+	return humanateBytes(s, 1024, sizes)
+}
+
+// ParseBytes parses a string representation of bytes into the number
+// of bytes it represents.
+//
+// See Also: Bytes, IBytes.
+//
+// ParseBytes("42 MB") -> 42000000, nil
+// ParseBytes("42 mib") -> 44040192, nil
+func ParseBytes(s string) (uint64, error) {
+	lastDigit := 0
+	hasComma := false
+	for _, r := range s {
+		if !(unicode.IsDigit(r) || r == '.' || r == ',') {
+			break
+		}
+		if r == ',' {
+			hasComma = true
+		}
+		lastDigit++
+	}
+
+	num := s[:lastDigit]
+	if hasComma {
+		num = strings.Replace(num, ",", "", -1)
+	}
+
+	f, err := strconv.ParseFloat(num, 64)
+	if err != nil {
+		return 0, err
+	}
+
+	extra := strings.ToLower(strings.TrimSpace(s[lastDigit:]))
+	if m, ok := bytesSizeTable[extra]; ok {
+		f *= float64(m)
+		if f >= math.MaxUint64 {
+			return 0, fmt.Errorf("too large: %v", s)
+		}
+		return uint64(f), nil
+	}
+
+	return 0, fmt.Errorf("unhandled size name: %v", extra)
+}
diff --git a/vendor/github.com/dustin/go-humanize/comma.go b/vendor/github.com/dustin/go-humanize/comma.go
new file mode 100644
index 0000000..13611aa
--- /dev/null
+++ b/vendor/github.com/dustin/go-humanize/comma.go
@@ -0,0 +1,108 @@
+package humanize
+
+import (
+	"bytes"
+	"math"
+	"math/big"
+	"strconv"
+	"strings"
+)
+
+// Comma produces a string form of the given number in base 10 with
+// commas after every three orders of magnitude.
+//
+// e.g. Comma(834142) -> 834,142
+func Comma(v int64) string {
+	sign := ""
+
+	// Min int64 can't be negated to a usable value, so it has to be special cased.
+	if v == math.MinInt64 {
+		return "-9,223,372,036,854,775,808"
+	}
+
+	if v < 0 {
+		sign = "-"
+		v = 0 - v
+	}
+
+	parts := []string{"", "", "", "", "", "", ""}
+	j := len(parts) - 1
+
+	for v > 999 {
+		parts[j] = strconv.FormatInt(v%1000, 10)
+		switch len(parts[j]) {
+		case 2:
+			parts[j] = "0" + parts[j]
+		case 1:
+			parts[j] = "00" + parts[j]
+		}
+		v = v / 1000
+		j--
+	}
+	parts[j] = strconv.Itoa(int(v))
+	return sign + strings.Join(parts[j:], ",")
+}
+
+// Commaf produces a string form of the given number in base 10 with
+// commas after every three orders of magnitude.
+//
+// e.g. Commaf(834142.32) -> 834,142.32
+func Commaf(v float64) string {
+	buf := &bytes.Buffer{}
+	if v < 0 {
+		buf.Write([]byte{'-'})
+		v = 0 - v
+	}
+
+	comma := []byte{','}
+
+	parts := strings.Split(strconv.FormatFloat(v, 'f', -1, 64), ".")
+	pos := 0
+	if len(parts[0])%3 != 0 {
+		pos += len(parts[0]) % 3
+		buf.WriteString(parts[0][:pos])
+		buf.Write(comma)
+	}
+	for ; pos < len(parts[0]); pos += 3 {
+		buf.WriteString(parts[0][pos : pos+3])
+		buf.Write(comma)
+	}
+	buf.Truncate(buf.Len() - 1)
+
+	if len(parts) > 1 {
+		buf.Write([]byte{'.'})
+		buf.WriteString(parts[1])
+	}
+	return buf.String()
+}
+
+// BigComma produces a string form of the given big.Int in base 10
+// with commas after every three orders of magnitude.
+func BigComma(b *big.Int) string {
+	sign := ""
+	if b.Sign() < 0 {
+		sign = "-"
+		b.Abs(b)
+	}
+
+	athousand := big.NewInt(1000)
+	c := (&big.Int{}).Set(b)
+	_, m := oom(c, athousand)
+	parts := make([]string, m+1)
+	j := len(parts) - 1
+
+	mod := &big.Int{}
+	for b.Cmp(athousand) >= 0 {
+		b.DivMod(b, athousand, mod)
+		parts[j] = strconv.FormatInt(mod.Int64(), 10)
+		switch len(parts[j]) {
+		case 2:
+			parts[j] = "0" + parts[j]
+		case 1:
+			parts[j] = "00" + parts[j]
+		}
+		j--
+	}
+	parts[j] = strconv.Itoa(int(b.Int64()))
+	return sign + strings.Join(parts[j:], ",")
+}
diff --git a/vendor/github.com/dustin/go-humanize/commaf.go b/vendor/github.com/dustin/go-humanize/commaf.go
new file mode 100644
index 0000000..620690d
--- /dev/null
+++ b/vendor/github.com/dustin/go-humanize/commaf.go
@@ -0,0 +1,40 @@
+// +build go1.6
+
+package humanize
+
+import (
+	"bytes"
+	"math/big"
+	"strings"
+)
+
+// BigCommaf produces a string form of the given big.Float in base 10
+// with commas after every three orders of magnitude.
+func BigCommaf(v *big.Float) string {
+	buf := &bytes.Buffer{}
+	if v.Sign() < 0 {
+		buf.Write([]byte{'-'})
+		v.Abs(v)
+	}
+
+	comma := []byte{','}
+
+	parts := strings.Split(v.Text('f', -1), ".")
+	pos := 0
+	if len(parts[0])%3 != 0 {
+		pos += len(parts[0]) % 3
+		buf.WriteString(parts[0][:pos])
+		buf.Write(comma)
+	}
+	for ; pos < len(parts[0]); pos += 3 {
+		buf.WriteString(parts[0][pos : pos+3])
+		buf.Write(comma)
+	}
+	buf.Truncate(buf.Len() - 1)
+
+	if len(parts) > 1 {
+		buf.Write([]byte{'.'})
+		buf.WriteString(parts[1])
+	}
+	return buf.String()
+}
diff --git a/vendor/github.com/dustin/go-humanize/ftoa.go b/vendor/github.com/dustin/go-humanize/ftoa.go
new file mode 100644
index 0000000..c76190b
--- /dev/null
+++ b/vendor/github.com/dustin/go-humanize/ftoa.go
@@ -0,0 +1,23 @@
+package humanize
+
+import "strconv"
+
+func stripTrailingZeros(s string) string {
+	offset := len(s) - 1
+	for offset > 0 {
+		if s[offset] == '.' {
+			offset--
+			break
+		}
+		if s[offset] != '0' {
+			break
+		}
+		offset--
+	}
+	return s[:offset+1]
+}
+
+// Ftoa converts a float to a string with no trailing zeros.
+func Ftoa(num float64) string {
+	return stripTrailingZeros(strconv.FormatFloat(num, 'f', 6, 64))
+}
diff --git a/vendor/github.com/dustin/go-humanize/humanize.go b/vendor/github.com/dustin/go-humanize/humanize.go
new file mode 100644
index 0000000..a2c2da3
--- /dev/null
+++ b/vendor/github.com/dustin/go-humanize/humanize.go
@@ -0,0 +1,8 @@
+/*
+Package humanize converts boring ugly numbers to human-friendly strings and back.
+
+Durations can be turned into strings such as "3 days ago", numbers
+representing sizes like 82854982 into useful strings like, "83 MB" or
+"79 MiB" (whichever you prefer).
+*/
+package humanize
diff --git a/vendor/github.com/dustin/go-humanize/number.go b/vendor/github.com/dustin/go-humanize/number.go
new file mode 100644
index 0000000..dec6186
--- /dev/null
+++ b/vendor/github.com/dustin/go-humanize/number.go
@@ -0,0 +1,192 @@
+package humanize
+
+/*
+Slightly adapted from the source to fit go-humanize.
+
+Author: https://github.com/gorhill
+Source: https://gist.github.com/gorhill/5285193
+
+*/
+
+import (
+	"math"
+	"strconv"
+)
+
+var (
+	renderFloatPrecisionMultipliers = [...]float64{
+		1,
+		10,
+		100,
+		1000,
+		10000,
+		100000,
+		1000000,
+		10000000,
+		100000000,
+		1000000000,
+	}
+
+	renderFloatPrecisionRounders = [...]float64{
+		0.5,
+		0.05,
+		0.005,
+		0.0005,
+		0.00005,
+		0.000005,
+		0.0000005,
+		0.00000005,
+		0.000000005,
+		0.0000000005,
+	}
+)
+
+// FormatFloat produces a formatted number as string based on the following user-specified criteria:
+// * thousands separator
+// * decimal separator
+// * decimal precision
+//
+// Usage: s := RenderFloat(format, n)
+// The format parameter tells how to render the number n.
+//
+// See examples: http://play.golang.org/p/LXc1Ddm1lJ
+//
+// Examples of format strings, given n = 12345.6789:
+// "#,###.##" => "12,345.67"
+// "#,###." => "12,345"
+// "#,###" => "12345,678"
+// "#\u202F###,##" => "12 345,68"
+// "#.###,###### => 12.345,678900
+// "" (aka default format) => 12,345.67
+//
+// The highest precision allowed is 9 digits after the decimal symbol.
+// There is also a version for integer number, FormatInteger(),
+// which is convenient for calls within template.
+func FormatFloat(format string, n float64) string {
+	// Special cases:
+	//   NaN = "NaN"
+	//   +Inf = "+Infinity"
+	//   -Inf = "-Infinity"
+	if math.IsNaN(n) {
+		return "NaN"
+	}
+	if n > math.MaxFloat64 {
+		return "Infinity"
+	}
+	if n < -math.MaxFloat64 {
+		return "-Infinity"
+	}
+
+	// default format
+	precision := 2
+	decimalStr := "."
+	thousandStr := ","
+	positiveStr := ""
+	negativeStr := "-"
+
+	if len(format) > 0 {
+		format := []rune(format)
+
+		// If there is an explicit format directive,
+		// then default values are these:
+		precision = 9
+		thousandStr = ""
+
+		// collect indices of meaningful formatting directives
+		formatIndx := []int{}
+		for i, char := range format {
+			if char != '#' && char != '0' {
+				formatIndx = append(formatIndx, i)
+			}
+		}
+
+		if len(formatIndx) > 0 {
+			// Directive at index 0:
+			//   Must be a '+'
+			//   Raise an error if not the case
+			// index: 0123456789
+			//        +0.000,000
+			//        +000,000.0
+			//        +0000.00
+			//        +0000
+			if formatIndx[0] == 0 {
+				if format[formatIndx[0]] != '+' {
+					panic("RenderFloat(): invalid positive sign directive")
+				}
+				positiveStr = "+"
+				formatIndx = formatIndx[1:]
+			}
+
+			// Two directives:
+			//   First is thousands separator
+			//   Raise an error if not followed by 3-digit
+			// 0123456789
+			// 0.000,000
+			// 000,000.00
+			if len(formatIndx) == 2 {
+				if (formatIndx[1] - formatIndx[0]) != 4 {
+					panic("RenderFloat(): thousands separator directive must be followed by 3 digit-specifiers")
+				}
+				thousandStr = string(format[formatIndx[0]])
+				formatIndx = formatIndx[1:]
+			}
+
+			// One directive:
+			//   Directive is decimal separator
+			//   The number of digit-specifier following the separator indicates wanted precision
+			// 0123456789
+			// 0.00
+			// 000,0000
+			if len(formatIndx) == 1 {
+				decimalStr = string(format[formatIndx[0]])
+				precision = len(format) - formatIndx[0] - 1
+			}
+		}
+	}
+
+	// generate sign part
+	var signStr string
+	if n >= 0.000000001 {
+		signStr = positiveStr
+	} else if n <= -0.000000001 {
+		signStr = negativeStr
+		n = -n
+	} else {
+		signStr = ""
+		n = 0.0
+	}
+
+	// split number into integer and fractional parts
+	intf, fracf := math.Modf(n + renderFloatPrecisionRounders[precision])
+
+	// generate integer part string
+	intStr := strconv.FormatInt(int64(intf), 10)
+
+	// add thousand separator if required
+	if len(thousandStr) > 0 {
+		for i := len(intStr); i > 3; {
+			i -= 3
+			intStr = intStr[:i] + thousandStr + intStr[i:]
+		}
+	}
+
+	// no fractional part, we can leave now
+	if precision == 0 {
+		return signStr + intStr
+	}
+
+	// generate fractional part
+	fracStr := strconv.Itoa(int(fracf * renderFloatPrecisionMultipliers[precision]))
+	// may need padding
+	if len(fracStr) < precision {
+		fracStr = "000000000000000"[:precision-len(fracStr)] + fracStr
+	}
+
+	return signStr + intStr + decimalStr + fracStr
+}
+
+// FormatInteger produces a formatted number as string.
+// See FormatFloat.
+func FormatInteger(format string, n int) string {
+	return FormatFloat(format, float64(n))
+}
diff --git a/vendor/github.com/dustin/go-humanize/ordinals.go b/vendor/github.com/dustin/go-humanize/ordinals.go
new file mode 100644
index 0000000..43d88a8
--- /dev/null
+++ b/vendor/github.com/dustin/go-humanize/ordinals.go
@@ -0,0 +1,25 @@
+package humanize
+
+import "strconv"
+
+// Ordinal gives you the input number in a rank/ordinal format.
+//
+// Ordinal(3) -> 3rd
+func Ordinal(x int) string {
+	suffix := "th"
+	switch x % 10 {
+	case 1:
+		if x%100 != 11 {
+			suffix = "st"
+		}
+	case 2:
+		if x%100 != 12 {
+			suffix = "nd"
+		}
+	case 3:
+		if x%100 != 13 {
+			suffix = "rd"
+		}
+	}
+	return strconv.Itoa(x) + suffix
+}
diff --git a/vendor/github.com/dustin/go-humanize/si.go b/vendor/github.com/dustin/go-humanize/si.go
new file mode 100644
index 0000000..b24e481
--- /dev/null
+++ b/vendor/github.com/dustin/go-humanize/si.go
@@ -0,0 +1,113 @@
+package humanize
+
+import (
+	"errors"
+	"math"
+	"regexp"
+	"strconv"
+)
+
+var siPrefixTable = map[float64]string{
+	-24: "y", // yocto
+	-21: "z", // zepto
+	-18: "a", // atto
+	-15: "f", // femto
+	-12: "p", // pico
+	-9:  "n", // nano
+	-6:  "µ", // micro
+	-3:  "m", // milli
+	0:   "",
+	3:   "k", // kilo
+	6:   "M", // mega
+	9:   "G", // giga
+	12:  "T", // tera
+	15:  "P", // peta
+	18:  "E", // exa
+	21:  "Z", // zetta
+	24:  "Y", // yotta
+}
+
+var revSIPrefixTable = revfmap(siPrefixTable)
+
+// revfmap reverses the map and precomputes the power multiplier
+func revfmap(in map[float64]string) map[string]float64 {
+	rv := map[string]float64{}
+	for k, v := range in {
+		rv[v] = math.Pow(10, k)
+	}
+	return rv
+}
+
+var riParseRegex *regexp.Regexp
+
+func init() {
+	ri := `^([\-0-9.]+)\s?([`
+	for _, v := range siPrefixTable {
+		ri += v
+	}
+	ri += `]?)(.*)`
+
+	riParseRegex = regexp.MustCompile(ri)
+}
+
+// ComputeSI finds the most appropriate SI prefix for the given number
+// and returns the prefix along with the value adjusted to be within
+// that prefix.
+//
+// See also: SI, ParseSI.
+//
+// e.g. ComputeSI(2.2345e-12) -> (2.2345, "p")
+func ComputeSI(input float64) (float64, string) {
+	if input == 0 {
+		return 0, ""
+	}
+	mag := math.Abs(input)
+	exponent := math.Floor(logn(mag, 10))
+	exponent = math.Floor(exponent/3) * 3
+
+	value := mag / math.Pow(10, exponent)
+
+	// Handle special case where value is exactly 1000.0
+	// Should return 1 M instead of 1000 k
+	if value == 1000.0 {
+		exponent += 3
+		value = mag / math.Pow(10, exponent)
+	}
+
+	value = math.Copysign(value, input)
+
+	prefix := siPrefixTable[exponent]
+	return value, prefix
+}
+
+// SI returns a string with default formatting.
+//
+// SI uses Ftoa to format float value, removing trailing zeros.
+//
+// See also: ComputeSI, ParseSI.
+//
+// e.g. SI(1000000, "B") -> 1 MB
+// e.g. SI(2.2345e-12, "F") -> 2.2345 pF
+func SI(input float64, unit string) string {
+	value, prefix := ComputeSI(input)
+	return Ftoa(value) + " " + prefix + unit
+}
+
+var errInvalid = errors.New("invalid input")
+
+// ParseSI parses an SI string back into the number and unit.
+//
+// See also: SI, ComputeSI.
+//
+// e.g. ParseSI("2.2345 pF") -> (2.2345e-12, "F", nil)
+func ParseSI(input string) (float64, string, error) {
+	found := riParseRegex.FindStringSubmatch(input)
+	if len(found) != 4 {
+		return 0, "", errInvalid
+	}
+	mag := revSIPrefixTable[found[2]]
+	unit := found[3]
+
+	base, err := strconv.ParseFloat(found[1], 64)
+	return base * mag, unit, err
+}
diff --git a/vendor/github.com/dustin/go-humanize/times.go b/vendor/github.com/dustin/go-humanize/times.go
new file mode 100644
index 0000000..dd3fbf5
--- /dev/null
+++ b/vendor/github.com/dustin/go-humanize/times.go
@@ -0,0 +1,117 @@
+package humanize
+
+import (
+	"fmt"
+	"math"
+	"sort"
+	"time"
+)
+
+// Seconds-based time units
+const (
+	Day      = 24 * time.Hour
+	Week     = 7 * Day
+	Month    = 30 * Day
+	Year     = 12 * Month
+	LongTime = 37 * Year
+)
+
+// Time formats a time into a relative string.
+//
+// Time(someT) -> "3 weeks ago"
+func Time(then time.Time) string {
+	return RelTime(then, time.Now(), "ago", "from now")
+}
+
+// A RelTimeMagnitude struct contains a relative time point at which
+// the relative format of time will switch to a new format string.  A
+// slice of these in ascending order by their "D" field is passed to
+// CustomRelTime to format durations.
+//
+// The Format field is a string that may contain a "%s" which will be
+// replaced with the appropriate signed label (e.g. "ago" or "from
+// now") and a "%d" that will be replaced by the quantity.
+//
+// The DivBy field is the amount of time the time difference must be
+// divided by in order to display correctly.
+//
+// e.g. if D is 2*time.Minute and you want to display "%d minutes %s"
+// DivBy should be time.Minute so whatever the duration is will be
+// expressed in minutes.
+type RelTimeMagnitude struct {
+	D      time.Duration
+	Format string
+	DivBy  time.Duration
+}
+
+var defaultMagnitudes = []RelTimeMagnitude{
+	{time.Second, "now", time.Second},
+	{2 * time.Second, "1 second %s", 1},
+	{time.Minute, "%d seconds %s", time.Second},
+	{2 * time.Minute, "1 minute %s", 1},
+	{time.Hour, "%d minutes %s", time.Minute},
+	{2 * time.Hour, "1 hour %s", 1},
+	{Day, "%d hours %s", time.Hour},
+	{2 * Day, "1 day %s", 1},
+	{Week, "%d days %s", Day},
+	{2 * Week, "1 week %s", 1},
+	{Month, "%d weeks %s", Week},
+	{2 * Month, "1 month %s", 1},
+	{Year, "%d months %s", Month},
+	{18 * Month, "1 year %s", 1},
+	{2 * Year, "2 years %s", 1},
+	{LongTime, "%d years %s", Year},
+	{math.MaxInt64, "a long while %s", 1},
+}
+
+// RelTime formats a time into a relative string.
+//
+// It takes two times and two labels.  In addition to the generic time
+// delta string (e.g. 5 minutes), the labels are used applied so that
+// the label corresponding to the smaller time is applied.
+//
+// RelTime(timeInPast, timeInFuture, "earlier", "later") -> "3 weeks earlier"
+func RelTime(a, b time.Time, albl, blbl string) string {
+	return CustomRelTime(a, b, albl, blbl, defaultMagnitudes)
+}
+
+// CustomRelTime formats a time into a relative string.
+//
+// It takes two times two labels and a table of relative time formats.
+// In addition to the generic time delta string (e.g. 5 minutes), the
+// labels are used applied so that the label corresponding to the
+// smaller time is applied.
+func CustomRelTime(a, b time.Time, albl, blbl string, magnitudes []RelTimeMagnitude) string {
+	lbl := albl
+	diff := b.Sub(a)
+
+	if a.After(b) {
+		lbl = blbl
+		diff = a.Sub(b)
+	}
+
+	n := sort.Search(len(magnitudes), func(i int) bool {
+		return magnitudes[i].D > diff
+	})
+
+	if n >= len(magnitudes) {
+		n = len(magnitudes) - 1
+	}
+	mag := magnitudes[n]
+	args := []interface{}{}
+	escaped := false
+	for _, ch := range mag.Format {
+		if escaped {
+			switch ch {
+			case 's':
+				args = append(args, lbl)
+			case 'd':
+				args = append(args, diff/mag.DivBy)
+			}
+			escaped = false
+		} else {
+			escaped = ch == '%'
+		}
+	}
+	return fmt.Sprintf(mag.Format, args...)
+}
diff --git a/vendor/github.com/golang/protobuf/jsonpb/jsonpb.go b/vendor/github.com/golang/protobuf/jsonpb/jsonpb.go
new file mode 100644
index 0000000..e9cc202
--- /dev/null
+++ b/vendor/github.com/golang/protobuf/jsonpb/jsonpb.go
@@ -0,0 +1,1284 @@
+// Go support for Protocol Buffers - Google's data interchange format
+//
+// Copyright 2015 The Go Authors.  All rights reserved.
+// https://github.com/golang/protobuf
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+/*
+Package jsonpb provides marshaling and unmarshaling between protocol buffers and JSON.
+It follows the specification at https://developers.google.com/protocol-buffers/docs/proto3#json.
+
+This package produces a different output than the standard "encoding/json" package,
+which does not operate correctly on protocol buffers.
+*/
+package jsonpb
+
+import (
+	"bytes"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"io"
+	"math"
+	"reflect"
+	"sort"
+	"strconv"
+	"strings"
+	"time"
+
+	"github.com/golang/protobuf/proto"
+
+	stpb "github.com/golang/protobuf/ptypes/struct"
+)
+
+const secondInNanos = int64(time.Second / time.Nanosecond)
+const maxSecondsInDuration = 315576000000
+
+// Marshaler is a configurable object for converting between
+// protocol buffer objects and a JSON representation for them.
+type Marshaler struct {
+	// Whether to render enum values as integers, as opposed to string values.
+	EnumsAsInts bool
+
+	// Whether to render fields with zero values.
+	EmitDefaults bool
+
+	// A string to indent each level by. The presence of this field will
+	// also cause a space to appear between the field separator and
+	// value, and for newlines to be appear between fields and array
+	// elements.
+	Indent string
+
+	// Whether to use the original (.proto) name for fields.
+	OrigName bool
+
+	// A custom URL resolver to use when marshaling Any messages to JSON.
+	// If unset, the default resolution strategy is to extract the
+	// fully-qualified type name from the type URL and pass that to
+	// proto.MessageType(string).
+	AnyResolver AnyResolver
+}
+
+// AnyResolver takes a type URL, present in an Any message, and resolves it into
+// an instance of the associated message.
+type AnyResolver interface {
+	Resolve(typeUrl string) (proto.Message, error)
+}
+
+func defaultResolveAny(typeUrl string) (proto.Message, error) {
+	// Only the part of typeUrl after the last slash is relevant.
+	mname := typeUrl
+	if slash := strings.LastIndex(mname, "/"); slash >= 0 {
+		mname = mname[slash+1:]
+	}
+	mt := proto.MessageType(mname)
+	if mt == nil {
+		return nil, fmt.Errorf("unknown message type %q", mname)
+	}
+	return reflect.New(mt.Elem()).Interface().(proto.Message), nil
+}
+
+// JSONPBMarshaler is implemented by protobuf messages that customize the
+// way they are marshaled to JSON. Messages that implement this should
+// also implement JSONPBUnmarshaler so that the custom format can be
+// parsed.
+//
+// The JSON marshaling must follow the proto to JSON specification:
+//	https://developers.google.com/protocol-buffers/docs/proto3#json
+type JSONPBMarshaler interface {
+	MarshalJSONPB(*Marshaler) ([]byte, error)
+}
+
+// JSONPBUnmarshaler is implemented by protobuf messages that customize
+// the way they are unmarshaled from JSON. Messages that implement this
+// should also implement JSONPBMarshaler so that the custom format can be
+// produced.
+//
+// The JSON unmarshaling must follow the JSON to proto specification:
+//	https://developers.google.com/protocol-buffers/docs/proto3#json
+type JSONPBUnmarshaler interface {
+	UnmarshalJSONPB(*Unmarshaler, []byte) error
+}
+
+// Marshal marshals a protocol buffer into JSON.
+func (m *Marshaler) Marshal(out io.Writer, pb proto.Message) error {
+	v := reflect.ValueOf(pb)
+	if pb == nil || (v.Kind() == reflect.Ptr && v.IsNil()) {
+		return errors.New("Marshal called with nil")
+	}
+	// Check for unset required fields first.
+	if err := checkRequiredFields(pb); err != nil {
+		return err
+	}
+	writer := &errWriter{writer: out}
+	return m.marshalObject(writer, pb, "", "")
+}
+
+// MarshalToString converts a protocol buffer object to JSON string.
+func (m *Marshaler) MarshalToString(pb proto.Message) (string, error) {
+	var buf bytes.Buffer
+	if err := m.Marshal(&buf, pb); err != nil {
+		return "", err
+	}
+	return buf.String(), nil
+}
+
+type int32Slice []int32
+
+var nonFinite = map[string]float64{
+	`"NaN"`:       math.NaN(),
+	`"Infinity"`:  math.Inf(1),
+	`"-Infinity"`: math.Inf(-1),
+}
+
+// For sorting extensions ids to ensure stable output.
+func (s int32Slice) Len() int           { return len(s) }
+func (s int32Slice) Less(i, j int) bool { return s[i] < s[j] }
+func (s int32Slice) Swap(i, j int)      { s[i], s[j] = s[j], s[i] }
+
+type wkt interface {
+	XXX_WellKnownType() string
+}
+
+// marshalObject writes a struct to the Writer.
+func (m *Marshaler) marshalObject(out *errWriter, v proto.Message, indent, typeURL string) error {
+	if jsm, ok := v.(JSONPBMarshaler); ok {
+		b, err := jsm.MarshalJSONPB(m)
+		if err != nil {
+			return err
+		}
+		if typeURL != "" {
+			// we are marshaling this object to an Any type
+			var js map[string]*json.RawMessage
+			if err = json.Unmarshal(b, &js); err != nil {
+				return fmt.Errorf("type %T produced invalid JSON: %v", v, err)
+			}
+			turl, err := json.Marshal(typeURL)
+			if err != nil {
+				return fmt.Errorf("failed to marshal type URL %q to JSON: %v", typeURL, err)
+			}
+			js["@type"] = (*json.RawMessage)(&turl)
+			if m.Indent != "" {
+				b, err = json.MarshalIndent(js, indent, m.Indent)
+			} else {
+				b, err = json.Marshal(js)
+			}
+			if err != nil {
+				return err
+			}
+		}
+
+		out.write(string(b))
+		return out.err
+	}
+
+	s := reflect.ValueOf(v).Elem()
+
+	// Handle well-known types.
+	if wkt, ok := v.(wkt); ok {
+		switch wkt.XXX_WellKnownType() {
+		case "DoubleValue", "FloatValue", "Int64Value", "UInt64Value",
+			"Int32Value", "UInt32Value", "BoolValue", "StringValue", "BytesValue":
+			// "Wrappers use the same representation in JSON
+			//  as the wrapped primitive type, ..."
+			sprop := proto.GetProperties(s.Type())
+			return m.marshalValue(out, sprop.Prop[0], s.Field(0), indent)
+		case "Any":
+			// Any is a bit more involved.
+			return m.marshalAny(out, v, indent)
+		case "Duration":
+			s, ns := s.Field(0).Int(), s.Field(1).Int()
+			if s < -maxSecondsInDuration || s > maxSecondsInDuration {
+				return fmt.Errorf("seconds out of range %v", s)
+			}
+			if ns <= -secondInNanos || ns >= secondInNanos {
+				return fmt.Errorf("ns out of range (%v, %v)", -secondInNanos, secondInNanos)
+			}
+			if (s > 0 && ns < 0) || (s < 0 && ns > 0) {
+				return errors.New("signs of seconds and nanos do not match")
+			}
+			// Generated output always contains 0, 3, 6, or 9 fractional digits,
+			// depending on required precision, followed by the suffix "s".
+			f := "%d.%09d"
+			if ns < 0 {
+				ns = -ns
+				if s == 0 {
+					f = "-%d.%09d"
+				}
+			}
+			x := fmt.Sprintf(f, s, ns)
+			x = strings.TrimSuffix(x, "000")
+			x = strings.TrimSuffix(x, "000")
+			x = strings.TrimSuffix(x, ".000")
+			out.write(`"`)
+			out.write(x)
+			out.write(`s"`)
+			return out.err
+		case "Struct", "ListValue":
+			// Let marshalValue handle the `Struct.fields` map or the `ListValue.values` slice.
+			// TODO: pass the correct Properties if needed.
+			return m.marshalValue(out, &proto.Properties{}, s.Field(0), indent)
+		case "Timestamp":
+			// "RFC 3339, where generated output will always be Z-normalized
+			//  and uses 0, 3, 6 or 9 fractional digits."
+			s, ns := s.Field(0).Int(), s.Field(1).Int()
+			if ns < 0 || ns >= secondInNanos {
+				return fmt.Errorf("ns out of range [0, %v)", secondInNanos)
+			}
+			t := time.Unix(s, ns).UTC()
+			// time.RFC3339Nano isn't exactly right (we need to get 3/6/9 fractional digits).
+			x := t.Format("2006-01-02T15:04:05.000000000")
+			x = strings.TrimSuffix(x, "000")
+			x = strings.TrimSuffix(x, "000")
+			x = strings.TrimSuffix(x, ".000")
+			out.write(`"`)
+			out.write(x)
+			out.write(`Z"`)
+			return out.err
+		case "Value":
+			// Value has a single oneof.
+			kind := s.Field(0)
+			if kind.IsNil() {
+				// "absence of any variant indicates an error"
+				return errors.New("nil Value")
+			}
+			// oneof -> *T -> T -> T.F
+			x := kind.Elem().Elem().Field(0)
+			// TODO: pass the correct Properties if needed.
+			return m.marshalValue(out, &proto.Properties{}, x, indent)
+		}
+	}
+
+	out.write("{")
+	if m.Indent != "" {
+		out.write("\n")
+	}
+
+	firstField := true
+
+	if typeURL != "" {
+		if err := m.marshalTypeURL(out, indent, typeURL); err != nil {
+			return err
+		}
+		firstField = false
+	}
+
+	for i := 0; i < s.NumField(); i++ {
+		value := s.Field(i)
+		valueField := s.Type().Field(i)
+		if strings.HasPrefix(valueField.Name, "XXX_") {
+			continue
+		}
+
+		// IsNil will panic on most value kinds.
+		switch value.Kind() {
+		case reflect.Chan, reflect.Func, reflect.Interface:
+			if value.IsNil() {
+				continue
+			}
+		}
+
+		if !m.EmitDefaults {
+			switch value.Kind() {
+			case reflect.Bool:
+				if !value.Bool() {
+					continue
+				}
+			case reflect.Int32, reflect.Int64:
+				if value.Int() == 0 {
+					continue
+				}
+			case reflect.Uint32, reflect.Uint64:
+				if value.Uint() == 0 {
+					continue
+				}
+			case reflect.Float32, reflect.Float64:
+				if value.Float() == 0 {
+					continue
+				}
+			case reflect.String:
+				if value.Len() == 0 {
+					continue
+				}
+			case reflect.Map, reflect.Ptr, reflect.Slice:
+				if value.IsNil() {
+					continue
+				}
+			}
+		}
+
+		// Oneof fields need special handling.
+		if valueField.Tag.Get("protobuf_oneof") != "" {
+			// value is an interface containing &T{real_value}.
+			sv := value.Elem().Elem() // interface -> *T -> T
+			value = sv.Field(0)
+			valueField = sv.Type().Field(0)
+		}
+		prop := jsonProperties(valueField, m.OrigName)
+		if !firstField {
+			m.writeSep(out)
+		}
+		if err := m.marshalField(out, prop, value, indent); err != nil {
+			return err
+		}
+		firstField = false
+	}
+
+	// Handle proto2 extensions.
+	if ep, ok := v.(proto.Message); ok {
+		extensions := proto.RegisteredExtensions(v)
+		// Sort extensions for stable output.
+		ids := make([]int32, 0, len(extensions))
+		for id, desc := range extensions {
+			if !proto.HasExtension(ep, desc) {
+				continue
+			}
+			ids = append(ids, id)
+		}
+		sort.Sort(int32Slice(ids))
+		for _, id := range ids {
+			desc := extensions[id]
+			if desc == nil {
+				// unknown extension
+				continue
+			}
+			ext, extErr := proto.GetExtension(ep, desc)
+			if extErr != nil {
+				return extErr
+			}
+			value := reflect.ValueOf(ext)
+			var prop proto.Properties
+			prop.Parse(desc.Tag)
+			prop.JSONName = fmt.Sprintf("[%s]", desc.Name)
+			if !firstField {
+				m.writeSep(out)
+			}
+			if err := m.marshalField(out, &prop, value, indent); err != nil {
+				return err
+			}
+			firstField = false
+		}
+
+	}
+
+	if m.Indent != "" {
+		out.write("\n")
+		out.write(indent)
+	}
+	out.write("}")
+	return out.err
+}
+
+func (m *Marshaler) writeSep(out *errWriter) {
+	if m.Indent != "" {
+		out.write(",\n")
+	} else {
+		out.write(",")
+	}
+}
+
+func (m *Marshaler) marshalAny(out *errWriter, any proto.Message, indent string) error {
+	// "If the Any contains a value that has a special JSON mapping,
+	//  it will be converted as follows: {"@type": xxx, "value": yyy}.
+	//  Otherwise, the value will be converted into a JSON object,
+	//  and the "@type" field will be inserted to indicate the actual data type."
+	v := reflect.ValueOf(any).Elem()
+	turl := v.Field(0).String()
+	val := v.Field(1).Bytes()
+
+	var msg proto.Message
+	var err error
+	if m.AnyResolver != nil {
+		msg, err = m.AnyResolver.Resolve(turl)
+	} else {
+		msg, err = defaultResolveAny(turl)
+	}
+	if err != nil {
+		return err
+	}
+
+	if err := proto.Unmarshal(val, msg); err != nil {
+		return err
+	}
+
+	if _, ok := msg.(wkt); ok {
+		out.write("{")
+		if m.Indent != "" {
+			out.write("\n")
+		}
+		if err := m.marshalTypeURL(out, indent, turl); err != nil {
+			return err
+		}
+		m.writeSep(out)
+		if m.Indent != "" {
+			out.write(indent)
+			out.write(m.Indent)
+			out.write(`"value": `)
+		} else {
+			out.write(`"value":`)
+		}
+		if err := m.marshalObject(out, msg, indent+m.Indent, ""); err != nil {
+			return err
+		}
+		if m.Indent != "" {
+			out.write("\n")
+			out.write(indent)
+		}
+		out.write("}")
+		return out.err
+	}
+
+	return m.marshalObject(out, msg, indent, turl)
+}
+
+func (m *Marshaler) marshalTypeURL(out *errWriter, indent, typeURL string) error {
+	if m.Indent != "" {
+		out.write(indent)
+		out.write(m.Indent)
+	}
+	out.write(`"@type":`)
+	if m.Indent != "" {
+		out.write(" ")
+	}
+	b, err := json.Marshal(typeURL)
+	if err != nil {
+		return err
+	}
+	out.write(string(b))
+	return out.err
+}
+
+// marshalField writes field description and value to the Writer.
+func (m *Marshaler) marshalField(out *errWriter, prop *proto.Properties, v reflect.Value, indent string) error {
+	if m.Indent != "" {
+		out.write(indent)
+		out.write(m.Indent)
+	}
+	out.write(`"`)
+	out.write(prop.JSONName)
+	out.write(`":`)
+	if m.Indent != "" {
+		out.write(" ")
+	}
+	if err := m.marshalValue(out, prop, v, indent); err != nil {
+		return err
+	}
+	return nil
+}
+
+// marshalValue writes the value to the Writer.
+func (m *Marshaler) marshalValue(out *errWriter, prop *proto.Properties, v reflect.Value, indent string) error {
+	var err error
+	v = reflect.Indirect(v)
+
+	// Handle nil pointer
+	if v.Kind() == reflect.Invalid {
+		out.write("null")
+		return out.err
+	}
+
+	// Handle repeated elements.
+	if v.Kind() == reflect.Slice && v.Type().Elem().Kind() != reflect.Uint8 {
+		out.write("[")
+		comma := ""
+		for i := 0; i < v.Len(); i++ {
+			sliceVal := v.Index(i)
+			out.write(comma)
+			if m.Indent != "" {
+				out.write("\n")
+				out.write(indent)
+				out.write(m.Indent)
+				out.write(m.Indent)
+			}
+			if err := m.marshalValue(out, prop, sliceVal, indent+m.Indent); err != nil {
+				return err
+			}
+			comma = ","
+		}
+		if m.Indent != "" {
+			out.write("\n")
+			out.write(indent)
+			out.write(m.Indent)
+		}
+		out.write("]")
+		return out.err
+	}
+
+	// Handle well-known types.
+	// Most are handled up in marshalObject (because 99% are messages).
+	if wkt, ok := v.Interface().(wkt); ok {
+		switch wkt.XXX_WellKnownType() {
+		case "NullValue":
+			out.write("null")
+			return out.err
+		}
+	}
+
+	// Handle enumerations.
+	if !m.EnumsAsInts && prop.Enum != "" {
+		// Unknown enum values will are stringified by the proto library as their
+		// value. Such values should _not_ be quoted or they will be interpreted
+		// as an enum string instead of their value.
+		enumStr := v.Interface().(fmt.Stringer).String()
+		var valStr string
+		if v.Kind() == reflect.Ptr {
+			valStr = strconv.Itoa(int(v.Elem().Int()))
+		} else {
+			valStr = strconv.Itoa(int(v.Int()))
+		}
+		isKnownEnum := enumStr != valStr
+		if isKnownEnum {
+			out.write(`"`)
+		}
+		out.write(enumStr)
+		if isKnownEnum {
+			out.write(`"`)
+		}
+		return out.err
+	}
+
+	// Handle nested messages.
+	if v.Kind() == reflect.Struct {
+		return m.marshalObject(out, v.Addr().Interface().(proto.Message), indent+m.Indent, "")
+	}
+
+	// Handle maps.
+	// Since Go randomizes map iteration, we sort keys for stable output.
+	if v.Kind() == reflect.Map {
+		out.write(`{`)
+		keys := v.MapKeys()
+		sort.Sort(mapKeys(keys))
+		for i, k := range keys {
+			if i > 0 {
+				out.write(`,`)
+			}
+			if m.Indent != "" {
+				out.write("\n")
+				out.write(indent)
+				out.write(m.Indent)
+				out.write(m.Indent)
+			}
+
+			// TODO handle map key prop properly
+			b, err := json.Marshal(k.Interface())
+			if err != nil {
+				return err
+			}
+			s := string(b)
+
+			// If the JSON is not a string value, encode it again to make it one.
+			if !strings.HasPrefix(s, `"`) {
+				b, err := json.Marshal(s)
+				if err != nil {
+					return err
+				}
+				s = string(b)
+			}
+
+			out.write(s)
+			out.write(`:`)
+			if m.Indent != "" {
+				out.write(` `)
+			}
+
+			vprop := prop
+			if prop != nil && prop.MapValProp != nil {
+				vprop = prop.MapValProp
+			}
+			if err := m.marshalValue(out, vprop, v.MapIndex(k), indent+m.Indent); err != nil {
+				return err
+			}
+		}
+		if m.Indent != "" {
+			out.write("\n")
+			out.write(indent)
+			out.write(m.Indent)
+		}
+		out.write(`}`)
+		return out.err
+	}
+
+	// Handle non-finite floats, e.g. NaN, Infinity and -Infinity.
+	if v.Kind() == reflect.Float32 || v.Kind() == reflect.Float64 {
+		f := v.Float()
+		var sval string
+		switch {
+		case math.IsInf(f, 1):
+			sval = `"Infinity"`
+		case math.IsInf(f, -1):
+			sval = `"-Infinity"`
+		case math.IsNaN(f):
+			sval = `"NaN"`
+		}
+		if sval != "" {
+			out.write(sval)
+			return out.err
+		}
+	}
+
+	// Default handling defers to the encoding/json library.
+	b, err := json.Marshal(v.Interface())
+	if err != nil {
+		return err
+	}
+	needToQuote := string(b[0]) != `"` && (v.Kind() == reflect.Int64 || v.Kind() == reflect.Uint64)
+	if needToQuote {
+		out.write(`"`)
+	}
+	out.write(string(b))
+	if needToQuote {
+		out.write(`"`)
+	}
+	return out.err
+}
+
+// Unmarshaler is a configurable object for converting from a JSON
+// representation to a protocol buffer object.
+type Unmarshaler struct {
+	// Whether to allow messages to contain unknown fields, as opposed to
+	// failing to unmarshal.
+	AllowUnknownFields bool
+
+	// A custom URL resolver to use when unmarshaling Any messages from JSON.
+	// If unset, the default resolution strategy is to extract the
+	// fully-qualified type name from the type URL and pass that to
+	// proto.MessageType(string).
+	AnyResolver AnyResolver
+}
+
+// UnmarshalNext unmarshals the next protocol buffer from a JSON object stream.
+// This function is lenient and will decode any options permutations of the
+// related Marshaler.
+func (u *Unmarshaler) UnmarshalNext(dec *json.Decoder, pb proto.Message) error {
+	inputValue := json.RawMessage{}
+	if err := dec.Decode(&inputValue); err != nil {
+		return err
+	}
+	if err := u.unmarshalValue(reflect.ValueOf(pb).Elem(), inputValue, nil); err != nil {
+		return err
+	}
+	return checkRequiredFields(pb)
+}
+
+// Unmarshal unmarshals a JSON object stream into a protocol
+// buffer. This function is lenient and will decode any options
+// permutations of the related Marshaler.
+func (u *Unmarshaler) Unmarshal(r io.Reader, pb proto.Message) error {
+	dec := json.NewDecoder(r)
+	return u.UnmarshalNext(dec, pb)
+}
+
+// UnmarshalNext unmarshals the next protocol buffer from a JSON object stream.
+// This function is lenient and will decode any options permutations of the
+// related Marshaler.
+func UnmarshalNext(dec *json.Decoder, pb proto.Message) error {
+	return new(Unmarshaler).UnmarshalNext(dec, pb)
+}
+
+// Unmarshal unmarshals a JSON object stream into a protocol
+// buffer. This function is lenient and will decode any options
+// permutations of the related Marshaler.
+func Unmarshal(r io.Reader, pb proto.Message) error {
+	return new(Unmarshaler).Unmarshal(r, pb)
+}
+
+// UnmarshalString will populate the fields of a protocol buffer based
+// on a JSON string. This function is lenient and will decode any options
+// permutations of the related Marshaler.
+func UnmarshalString(str string, pb proto.Message) error {
+	return new(Unmarshaler).Unmarshal(strings.NewReader(str), pb)
+}
+
+// unmarshalValue converts/copies a value into the target.
+// prop may be nil.
+func (u *Unmarshaler) unmarshalValue(target reflect.Value, inputValue json.RawMessage, prop *proto.Properties) error {
+	targetType := target.Type()
+
+	// Allocate memory for pointer fields.
+	if targetType.Kind() == reflect.Ptr {
+		// If input value is "null" and target is a pointer type, then the field should be treated as not set
+		// UNLESS the target is structpb.Value, in which case it should be set to structpb.NullValue.
+		_, isJSONPBUnmarshaler := target.Interface().(JSONPBUnmarshaler)
+		if string(inputValue) == "null" && targetType != reflect.TypeOf(&stpb.Value{}) && !isJSONPBUnmarshaler {
+			return nil
+		}
+		target.Set(reflect.New(targetType.Elem()))
+
+		return u.unmarshalValue(target.Elem(), inputValue, prop)
+	}
+
+	if jsu, ok := target.Addr().Interface().(JSONPBUnmarshaler); ok {
+		return jsu.UnmarshalJSONPB(u, []byte(inputValue))
+	}
+
+	// Handle well-known types that are not pointers.
+	if w, ok := target.Addr().Interface().(wkt); ok {
+		switch w.XXX_WellKnownType() {
+		case "DoubleValue", "FloatValue", "Int64Value", "UInt64Value",
+			"Int32Value", "UInt32Value", "BoolValue", "StringValue", "BytesValue":
+			return u.unmarshalValue(target.Field(0), inputValue, prop)
+		case "Any":
+			// Use json.RawMessage pointer type instead of value to support pre-1.8 version.
+			// 1.8 changed RawMessage.MarshalJSON from pointer type to value type, see
+			// https://github.com/golang/go/issues/14493
+			var jsonFields map[string]*json.RawMessage
+			if err := json.Unmarshal(inputValue, &jsonFields); err != nil {
+				return err
+			}
+
+			val, ok := jsonFields["@type"]
+			if !ok || val == nil {
+				return errors.New("Any JSON doesn't have '@type'")
+			}
+
+			var turl string
+			if err := json.Unmarshal([]byte(*val), &turl); err != nil {
+				return fmt.Errorf("can't unmarshal Any's '@type': %q", *val)
+			}
+			target.Field(0).SetString(turl)
+
+			var m proto.Message
+			var err error
+			if u.AnyResolver != nil {
+				m, err = u.AnyResolver.Resolve(turl)
+			} else {
+				m, err = defaultResolveAny(turl)
+			}
+			if err != nil {
+				return err
+			}
+
+			if _, ok := m.(wkt); ok {
+				val, ok := jsonFields["value"]
+				if !ok {
+					return errors.New("Any JSON doesn't have 'value'")
+				}
+
+				if err := u.unmarshalValue(reflect.ValueOf(m).Elem(), *val, nil); err != nil {
+					return fmt.Errorf("can't unmarshal Any nested proto %T: %v", m, err)
+				}
+			} else {
+				delete(jsonFields, "@type")
+				nestedProto, err := json.Marshal(jsonFields)
+				if err != nil {
+					return fmt.Errorf("can't generate JSON for Any's nested proto to be unmarshaled: %v", err)
+				}
+
+				if err = u.unmarshalValue(reflect.ValueOf(m).Elem(), nestedProto, nil); err != nil {
+					return fmt.Errorf("can't unmarshal Any nested proto %T: %v", m, err)
+				}
+			}
+
+			b, err := proto.Marshal(m)
+			if err != nil {
+				return fmt.Errorf("can't marshal proto %T into Any.Value: %v", m, err)
+			}
+			target.Field(1).SetBytes(b)
+
+			return nil
+		case "Duration":
+			unq, err := unquote(string(inputValue))
+			if err != nil {
+				return err
+			}
+
+			d, err := time.ParseDuration(unq)
+			if err != nil {
+				return fmt.Errorf("bad Duration: %v", err)
+			}
+
+			ns := d.Nanoseconds()
+			s := ns / 1e9
+			ns %= 1e9
+			target.Field(0).SetInt(s)
+			target.Field(1).SetInt(ns)
+			return nil
+		case "Timestamp":
+			unq, err := unquote(string(inputValue))
+			if err != nil {
+				return err
+			}
+
+			t, err := time.Parse(time.RFC3339Nano, unq)
+			if err != nil {
+				return fmt.Errorf("bad Timestamp: %v", err)
+			}
+
+			target.Field(0).SetInt(t.Unix())
+			target.Field(1).SetInt(int64(t.Nanosecond()))
+			return nil
+		case "Struct":
+			var m map[string]json.RawMessage
+			if err := json.Unmarshal(inputValue, &m); err != nil {
+				return fmt.Errorf("bad StructValue: %v", err)
+			}
+
+			target.Field(0).Set(reflect.ValueOf(map[string]*stpb.Value{}))
+			for k, jv := range m {
+				pv := &stpb.Value{}
+				if err := u.unmarshalValue(reflect.ValueOf(pv).Elem(), jv, prop); err != nil {
+					return fmt.Errorf("bad value in StructValue for key %q: %v", k, err)
+				}
+				target.Field(0).SetMapIndex(reflect.ValueOf(k), reflect.ValueOf(pv))
+			}
+			return nil
+		case "ListValue":
+			var s []json.RawMessage
+			if err := json.Unmarshal(inputValue, &s); err != nil {
+				return fmt.Errorf("bad ListValue: %v", err)
+			}
+
+			target.Field(0).Set(reflect.ValueOf(make([]*stpb.Value, len(s))))
+			for i, sv := range s {
+				if err := u.unmarshalValue(target.Field(0).Index(i), sv, prop); err != nil {
+					return err
+				}
+			}
+			return nil
+		case "Value":
+			ivStr := string(inputValue)
+			if ivStr == "null" {
+				target.Field(0).Set(reflect.ValueOf(&stpb.Value_NullValue{}))
+			} else if v, err := strconv.ParseFloat(ivStr, 0); err == nil {
+				target.Field(0).Set(reflect.ValueOf(&stpb.Value_NumberValue{v}))
+			} else if v, err := unquote(ivStr); err == nil {
+				target.Field(0).Set(reflect.ValueOf(&stpb.Value_StringValue{v}))
+			} else if v, err := strconv.ParseBool(ivStr); err == nil {
+				target.Field(0).Set(reflect.ValueOf(&stpb.Value_BoolValue{v}))
+			} else if err := json.Unmarshal(inputValue, &[]json.RawMessage{}); err == nil {
+				lv := &stpb.ListValue{}
+				target.Field(0).Set(reflect.ValueOf(&stpb.Value_ListValue{lv}))
+				return u.unmarshalValue(reflect.ValueOf(lv).Elem(), inputValue, prop)
+			} else if err := json.Unmarshal(inputValue, &map[string]json.RawMessage{}); err == nil {
+				sv := &stpb.Struct{}
+				target.Field(0).Set(reflect.ValueOf(&stpb.Value_StructValue{sv}))
+				return u.unmarshalValue(reflect.ValueOf(sv).Elem(), inputValue, prop)
+			} else {
+				return fmt.Errorf("unrecognized type for Value %q", ivStr)
+			}
+			return nil
+		}
+	}
+
+	// Handle enums, which have an underlying type of int32,
+	// and may appear as strings.
+	// The case of an enum appearing as a number is handled
+	// at the bottom of this function.
+	if inputValue[0] == '"' && prop != nil && prop.Enum != "" {
+		vmap := proto.EnumValueMap(prop.Enum)
+		// Don't need to do unquoting; valid enum names
+		// are from a limited character set.
+		s := inputValue[1 : len(inputValue)-1]
+		n, ok := vmap[string(s)]
+		if !ok {
+			return fmt.Errorf("unknown value %q for enum %s", s, prop.Enum)
+		}
+		if target.Kind() == reflect.Ptr { // proto2
+			target.Set(reflect.New(targetType.Elem()))
+			target = target.Elem()
+		}
+		if targetType.Kind() != reflect.Int32 {
+			return fmt.Errorf("invalid target %q for enum %s", targetType.Kind(), prop.Enum)
+		}
+		target.SetInt(int64(n))
+		return nil
+	}
+
+	// Handle nested messages.
+	if targetType.Kind() == reflect.Struct {
+		var jsonFields map[string]json.RawMessage
+		if err := json.Unmarshal(inputValue, &jsonFields); err != nil {
+			return err
+		}
+
+		consumeField := func(prop *proto.Properties) (json.RawMessage, bool) {
+			// Be liberal in what names we accept; both orig_name and camelName are okay.
+			fieldNames := acceptedJSONFieldNames(prop)
+
+			vOrig, okOrig := jsonFields[fieldNames.orig]
+			vCamel, okCamel := jsonFields[fieldNames.camel]
+			if !okOrig && !okCamel {
+				return nil, false
+			}
+			// If, for some reason, both are present in the data, favour the camelName.
+			var raw json.RawMessage
+			if okOrig {
+				raw = vOrig
+				delete(jsonFields, fieldNames.orig)
+			}
+			if okCamel {
+				raw = vCamel
+				delete(jsonFields, fieldNames.camel)
+			}
+			return raw, true
+		}
+
+		sprops := proto.GetProperties(targetType)
+		for i := 0; i < target.NumField(); i++ {
+			ft := target.Type().Field(i)
+			if strings.HasPrefix(ft.Name, "XXX_") {
+				continue
+			}
+
+			valueForField, ok := consumeField(sprops.Prop[i])
+			if !ok {
+				continue
+			}
+
+			if err := u.unmarshalValue(target.Field(i), valueForField, sprops.Prop[i]); err != nil {
+				return err
+			}
+		}
+		// Check for any oneof fields.
+		if len(jsonFields) > 0 {
+			for _, oop := range sprops.OneofTypes {
+				raw, ok := consumeField(oop.Prop)
+				if !ok {
+					continue
+				}
+				nv := reflect.New(oop.Type.Elem())
+				target.Field(oop.Field).Set(nv)
+				if err := u.unmarshalValue(nv.Elem().Field(0), raw, oop.Prop); err != nil {
+					return err
+				}
+			}
+		}
+		// Handle proto2 extensions.
+		if len(jsonFields) > 0 {
+			if ep, ok := target.Addr().Interface().(proto.Message); ok {
+				for _, ext := range proto.RegisteredExtensions(ep) {
+					name := fmt.Sprintf("[%s]", ext.Name)
+					raw, ok := jsonFields[name]
+					if !ok {
+						continue
+					}
+					delete(jsonFields, name)
+					nv := reflect.New(reflect.TypeOf(ext.ExtensionType).Elem())
+					if err := u.unmarshalValue(nv.Elem(), raw, nil); err != nil {
+						return err
+					}
+					if err := proto.SetExtension(ep, ext, nv.Interface()); err != nil {
+						return err
+					}
+				}
+			}
+		}
+		if !u.AllowUnknownFields && len(jsonFields) > 0 {
+			// Pick any field to be the scapegoat.
+			var f string
+			for fname := range jsonFields {
+				f = fname
+				break
+			}
+			return fmt.Errorf("unknown field %q in %v", f, targetType)
+		}
+		return nil
+	}
+
+	// Handle arrays (which aren't encoded bytes)
+	if targetType.Kind() == reflect.Slice && targetType.Elem().Kind() != reflect.Uint8 {
+		var slc []json.RawMessage
+		if err := json.Unmarshal(inputValue, &slc); err != nil {
+			return err
+		}
+		if slc != nil {
+			l := len(slc)
+			target.Set(reflect.MakeSlice(targetType, l, l))
+			for i := 0; i < l; i++ {
+				if err := u.unmarshalValue(target.Index(i), slc[i], prop); err != nil {
+					return err
+				}
+			}
+		}
+		return nil
+	}
+
+	// Handle maps (whose keys are always strings)
+	if targetType.Kind() == reflect.Map {
+		var mp map[string]json.RawMessage
+		if err := json.Unmarshal(inputValue, &mp); err != nil {
+			return err
+		}
+		if mp != nil {
+			target.Set(reflect.MakeMap(targetType))
+			for ks, raw := range mp {
+				// Unmarshal map key. The core json library already decoded the key into a
+				// string, so we handle that specially. Other types were quoted post-serialization.
+				var k reflect.Value
+				if targetType.Key().Kind() == reflect.String {
+					k = reflect.ValueOf(ks)
+				} else {
+					k = reflect.New(targetType.Key()).Elem()
+					var kprop *proto.Properties
+					if prop != nil && prop.MapKeyProp != nil {
+						kprop = prop.MapKeyProp
+					}
+					if err := u.unmarshalValue(k, json.RawMessage(ks), kprop); err != nil {
+						return err
+					}
+				}
+
+				// Unmarshal map value.
+				v := reflect.New(targetType.Elem()).Elem()
+				var vprop *proto.Properties
+				if prop != nil && prop.MapValProp != nil {
+					vprop = prop.MapValProp
+				}
+				if err := u.unmarshalValue(v, raw, vprop); err != nil {
+					return err
+				}
+				target.SetMapIndex(k, v)
+			}
+		}
+		return nil
+	}
+
+	// Non-finite numbers can be encoded as strings.
+	isFloat := targetType.Kind() == reflect.Float32 || targetType.Kind() == reflect.Float64
+	if isFloat {
+		if num, ok := nonFinite[string(inputValue)]; ok {
+			target.SetFloat(num)
+			return nil
+		}
+	}
+
+	// integers & floats can be encoded as strings. In this case we drop
+	// the quotes and proceed as normal.
+	isNum := targetType.Kind() == reflect.Int64 || targetType.Kind() == reflect.Uint64 ||
+		targetType.Kind() == reflect.Int32 || targetType.Kind() == reflect.Uint32 ||
+		targetType.Kind() == reflect.Float32 || targetType.Kind() == reflect.Float64
+	if isNum && strings.HasPrefix(string(inputValue), `"`) {
+		inputValue = inputValue[1 : len(inputValue)-1]
+	}
+
+	// Use the encoding/json for parsing other value types.
+	return json.Unmarshal(inputValue, target.Addr().Interface())
+}
+
+func unquote(s string) (string, error) {
+	var ret string
+	err := json.Unmarshal([]byte(s), &ret)
+	return ret, err
+}
+
+// jsonProperties returns parsed proto.Properties for the field and corrects JSONName attribute.
+func jsonProperties(f reflect.StructField, origName bool) *proto.Properties {
+	var prop proto.Properties
+	prop.Init(f.Type, f.Name, f.Tag.Get("protobuf"), &f)
+	if origName || prop.JSONName == "" {
+		prop.JSONName = prop.OrigName
+	}
+	return &prop
+}
+
+type fieldNames struct {
+	orig, camel string
+}
+
+func acceptedJSONFieldNames(prop *proto.Properties) fieldNames {
+	opts := fieldNames{orig: prop.OrigName, camel: prop.OrigName}
+	if prop.JSONName != "" {
+		opts.camel = prop.JSONName
+	}
+	return opts
+}
+
+// Writer wrapper inspired by https://blog.golang.org/errors-are-values
+type errWriter struct {
+	writer io.Writer
+	err    error
+}
+
+func (w *errWriter) write(str string) {
+	if w.err != nil {
+		return
+	}
+	_, w.err = w.writer.Write([]byte(str))
+}
+
+// Map fields may have key types of non-float scalars, strings and enums.
+// The easiest way to sort them in some deterministic order is to use fmt.
+// If this turns out to be inefficient we can always consider other options,
+// such as doing a Schwartzian transform.
+//
+// Numeric keys are sorted in numeric order per
+// https://developers.google.com/protocol-buffers/docs/proto#maps.
+type mapKeys []reflect.Value
+
+func (s mapKeys) Len() int      { return len(s) }
+func (s mapKeys) Swap(i, j int) { s[i], s[j] = s[j], s[i] }
+func (s mapKeys) Less(i, j int) bool {
+	if k := s[i].Kind(); k == s[j].Kind() {
+		switch k {
+		case reflect.String:
+			return s[i].String() < s[j].String()
+		case reflect.Int32, reflect.Int64:
+			return s[i].Int() < s[j].Int()
+		case reflect.Uint32, reflect.Uint64:
+			return s[i].Uint() < s[j].Uint()
+		}
+	}
+	return fmt.Sprint(s[i].Interface()) < fmt.Sprint(s[j].Interface())
+}
+
+// checkRequiredFields returns an error if any required field in the given proto message is not set.
+// This function is used by both Marshal and Unmarshal.  While required fields only exist in a
+// proto2 message, a proto3 message can contain proto2 message(s).
+func checkRequiredFields(pb proto.Message) error {
+	// Most well-known type messages do not contain required fields.  The "Any" type may contain
+	// a message that has required fields.
+	//
+	// When an Any message is being marshaled, the code will invoked proto.Unmarshal on Any.Value
+	// field in order to transform that into JSON, and that should have returned an error if a
+	// required field is not set in the embedded message.
+	//
+	// When an Any message is being unmarshaled, the code will have invoked proto.Marshal on the
+	// embedded message to store the serialized message in Any.Value field, and that should have
+	// returned an error if a required field is not set.
+	if _, ok := pb.(wkt); ok {
+		return nil
+	}
+
+	v := reflect.ValueOf(pb)
+	// Skip message if it is not a struct pointer.
+	if v.Kind() != reflect.Ptr {
+		return nil
+	}
+	v = v.Elem()
+	if v.Kind() != reflect.Struct {
+		return nil
+	}
+
+	for i := 0; i < v.NumField(); i++ {
+		field := v.Field(i)
+		sfield := v.Type().Field(i)
+
+		if sfield.PkgPath != "" {
+			// blank PkgPath means the field is exported; skip if not exported
+			continue
+		}
+
+		if strings.HasPrefix(sfield.Name, "XXX_") {
+			continue
+		}
+
+		// Oneof field is an interface implemented by wrapper structs containing the actual oneof
+		// field, i.e. an interface containing &T{real_value}.
+		if sfield.Tag.Get("protobuf_oneof") != "" {
+			if field.Kind() != reflect.Interface {
+				continue
+			}
+			v := field.Elem()
+			if v.Kind() != reflect.Ptr || v.IsNil() {
+				continue
+			}
+			v = v.Elem()
+			if v.Kind() != reflect.Struct || v.NumField() < 1 {
+				continue
+			}
+			field = v.Field(0)
+			sfield = v.Type().Field(0)
+		}
+
+		protoTag := sfield.Tag.Get("protobuf")
+		if protoTag == "" {
+			continue
+		}
+		var prop proto.Properties
+		prop.Init(sfield.Type, sfield.Name, protoTag, &sfield)
+
+		switch field.Kind() {
+		case reflect.Map:
+			if field.IsNil() {
+				continue
+			}
+			// Check each map value.
+			keys := field.MapKeys()
+			for _, k := range keys {
+				v := field.MapIndex(k)
+				if err := checkRequiredFieldsInValue(v); err != nil {
+					return err
+				}
+			}
+		case reflect.Slice:
+			// Handle non-repeated type, e.g. bytes.
+			if !prop.Repeated {
+				if prop.Required && field.IsNil() {
+					return fmt.Errorf("required field %q is not set", prop.Name)
+				}
+				continue
+			}
+
+			// Handle repeated type.
+			if field.IsNil() {
+				continue
+			}
+			// Check each slice item.
+			for i := 0; i < field.Len(); i++ {
+				v := field.Index(i)
+				if err := checkRequiredFieldsInValue(v); err != nil {
+					return err
+				}
+			}
+		case reflect.Ptr:
+			if field.IsNil() {
+				if prop.Required {
+					return fmt.Errorf("required field %q is not set", prop.Name)
+				}
+				continue
+			}
+			if err := checkRequiredFieldsInValue(field); err != nil {
+				return err
+			}
+		}
+	}
+
+	// Handle proto2 extensions.
+	for _, ext := range proto.RegisteredExtensions(pb) {
+		if !proto.HasExtension(pb, ext) {
+			continue
+		}
+		ep, err := proto.GetExtension(pb, ext)
+		if err != nil {
+			return err
+		}
+		err = checkRequiredFieldsInValue(reflect.ValueOf(ep))
+		if err != nil {
+			return err
+		}
+	}
+
+	return nil
+}
+
+func checkRequiredFieldsInValue(v reflect.Value) error {
+	if pm, ok := v.Interface().(proto.Message); ok {
+		return checkRequiredFields(pm)
+	}
+	return nil
+}
diff --git a/vendor/github.com/golang/protobuf/protoc-gen-go/generator/generator.go b/vendor/github.com/golang/protobuf/protoc-gen-go/generator/generator.go
new file mode 100644
index 0000000..6f4a902
--- /dev/null
+++ b/vendor/github.com/golang/protobuf/protoc-gen-go/generator/generator.go
@@ -0,0 +1,2806 @@
+// Go support for Protocol Buffers - Google's data interchange format
+//
+// Copyright 2010 The Go Authors.  All rights reserved.
+// https://github.com/golang/protobuf
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+/*
+	The code generator for the plugin for the Google protocol buffer compiler.
+	It generates Go code from the protocol buffer description files read by the
+	main routine.
+*/
+package generator
+
+import (
+	"bufio"
+	"bytes"
+	"compress/gzip"
+	"crypto/sha256"
+	"encoding/hex"
+	"fmt"
+	"go/ast"
+	"go/build"
+	"go/parser"
+	"go/printer"
+	"go/token"
+	"log"
+	"os"
+	"path"
+	"sort"
+	"strconv"
+	"strings"
+	"unicode"
+	"unicode/utf8"
+
+	"github.com/golang/protobuf/proto"
+	"github.com/golang/protobuf/protoc-gen-go/generator/internal/remap"
+
+	"github.com/golang/protobuf/protoc-gen-go/descriptor"
+	plugin "github.com/golang/protobuf/protoc-gen-go/plugin"
+)
+
+// generatedCodeVersion indicates a version of the generated code.
+// It is incremented whenever an incompatibility between the generated code and
+// proto package is introduced; the generated code references
+// a constant, proto.ProtoPackageIsVersionN (where N is generatedCodeVersion).
+const generatedCodeVersion = 3
+
+// A Plugin provides functionality to add to the output during Go code generation,
+// such as to produce RPC stubs.
+type Plugin interface {
+	// Name identifies the plugin.
+	Name() string
+	// Init is called once after data structures are built but before
+	// code generation begins.
+	Init(g *Generator)
+	// Generate produces the code generated by the plugin for this file,
+	// except for the imports, by calling the generator's methods P, In, and Out.
+	Generate(file *FileDescriptor)
+	// GenerateImports produces the import declarations for this file.
+	// It is called after Generate.
+	GenerateImports(file *FileDescriptor)
+}
+
+var plugins []Plugin
+
+// RegisterPlugin installs a (second-order) plugin to be run when the Go output is generated.
+// It is typically called during initialization.
+func RegisterPlugin(p Plugin) {
+	plugins = append(plugins, p)
+}
+
+// A GoImportPath is the import path of a Go package. e.g., "google.golang.org/genproto/protobuf".
+type GoImportPath string
+
+func (p GoImportPath) String() string { return strconv.Quote(string(p)) }
+
+// A GoPackageName is the name of a Go package. e.g., "protobuf".
+type GoPackageName string
+
+// Each type we import as a protocol buffer (other than FileDescriptorProto) needs
+// a pointer to the FileDescriptorProto that represents it.  These types achieve that
+// wrapping by placing each Proto inside a struct with the pointer to its File. The
+// structs have the same names as their contents, with "Proto" removed.
+// FileDescriptor is used to store the things that it points to.
+
+// The file and package name method are common to messages and enums.
+type common struct {
+	file *FileDescriptor // File this object comes from.
+}
+
+// GoImportPath is the import path of the Go package containing the type.
+func (c *common) GoImportPath() GoImportPath {
+	return c.file.importPath
+}
+
+func (c *common) File() *FileDescriptor { return c.file }
+
+func fileIsProto3(file *descriptor.FileDescriptorProto) bool {
+	return file.GetSyntax() == "proto3"
+}
+
+func (c *common) proto3() bool { return fileIsProto3(c.file.FileDescriptorProto) }
+
+// Descriptor represents a protocol buffer message.
+type Descriptor struct {
+	common
+	*descriptor.DescriptorProto
+	parent   *Descriptor            // The containing message, if any.
+	nested   []*Descriptor          // Inner messages, if any.
+	enums    []*EnumDescriptor      // Inner enums, if any.
+	ext      []*ExtensionDescriptor // Extensions, if any.
+	typename []string               // Cached typename vector.
+	index    int                    // The index into the container, whether the file or another message.
+	path     string                 // The SourceCodeInfo path as comma-separated integers.
+	group    bool
+}
+
+// TypeName returns the elements of the dotted type name.
+// The package name is not part of this name.
+func (d *Descriptor) TypeName() []string {
+	if d.typename != nil {
+		return d.typename
+	}
+	n := 0
+	for parent := d; parent != nil; parent = parent.parent {
+		n++
+	}
+	s := make([]string, n)
+	for parent := d; parent != nil; parent = parent.parent {
+		n--
+		s[n] = parent.GetName()
+	}
+	d.typename = s
+	return s
+}
+
+// EnumDescriptor describes an enum. If it's at top level, its parent will be nil.
+// Otherwise it will be the descriptor of the message in which it is defined.
+type EnumDescriptor struct {
+	common
+	*descriptor.EnumDescriptorProto
+	parent   *Descriptor // The containing message, if any.
+	typename []string    // Cached typename vector.
+	index    int         // The index into the container, whether the file or a message.
+	path     string      // The SourceCodeInfo path as comma-separated integers.
+}
+
+// TypeName returns the elements of the dotted type name.
+// The package name is not part of this name.
+func (e *EnumDescriptor) TypeName() (s []string) {
+	if e.typename != nil {
+		return e.typename
+	}
+	name := e.GetName()
+	if e.parent == nil {
+		s = make([]string, 1)
+	} else {
+		pname := e.parent.TypeName()
+		s = make([]string, len(pname)+1)
+		copy(s, pname)
+	}
+	s[len(s)-1] = name
+	e.typename = s
+	return s
+}
+
+// Everything but the last element of the full type name, CamelCased.
+// The values of type Foo.Bar are call Foo_value1... not Foo_Bar_value1... .
+func (e *EnumDescriptor) prefix() string {
+	if e.parent == nil {
+		// If the enum is not part of a message, the prefix is just the type name.
+		return CamelCase(*e.Name) + "_"
+	}
+	typeName := e.TypeName()
+	return CamelCaseSlice(typeName[0:len(typeName)-1]) + "_"
+}
+
+// The integer value of the named constant in this enumerated type.
+func (e *EnumDescriptor) integerValueAsString(name string) string {
+	for _, c := range e.Value {
+		if c.GetName() == name {
+			return fmt.Sprint(c.GetNumber())
+		}
+	}
+	log.Fatal("cannot find value for enum constant")
+	return ""
+}
+
+// ExtensionDescriptor describes an extension. If it's at top level, its parent will be nil.
+// Otherwise it will be the descriptor of the message in which it is defined.
+type ExtensionDescriptor struct {
+	common
+	*descriptor.FieldDescriptorProto
+	parent *Descriptor // The containing message, if any.
+}
+
+// TypeName returns the elements of the dotted type name.
+// The package name is not part of this name.
+func (e *ExtensionDescriptor) TypeName() (s []string) {
+	name := e.GetName()
+	if e.parent == nil {
+		// top-level extension
+		s = make([]string, 1)
+	} else {
+		pname := e.parent.TypeName()
+		s = make([]string, len(pname)+1)
+		copy(s, pname)
+	}
+	s[len(s)-1] = name
+	return s
+}
+
+// DescName returns the variable name used for the generated descriptor.
+func (e *ExtensionDescriptor) DescName() string {
+	// The full type name.
+	typeName := e.TypeName()
+	// Each scope of the extension is individually CamelCased, and all are joined with "_" with an "E_" prefix.
+	for i, s := range typeName {
+		typeName[i] = CamelCase(s)
+	}
+	return "E_" + strings.Join(typeName, "_")
+}
+
+// ImportedDescriptor describes a type that has been publicly imported from another file.
+type ImportedDescriptor struct {
+	common
+	o Object
+}
+
+func (id *ImportedDescriptor) TypeName() []string { return id.o.TypeName() }
+
+// FileDescriptor describes an protocol buffer descriptor file (.proto).
+// It includes slices of all the messages and enums defined within it.
+// Those slices are constructed by WrapTypes.
+type FileDescriptor struct {
+	*descriptor.FileDescriptorProto
+	desc []*Descriptor          // All the messages defined in this file.
+	enum []*EnumDescriptor      // All the enums defined in this file.
+	ext  []*ExtensionDescriptor // All the top-level extensions defined in this file.
+	imp  []*ImportedDescriptor  // All types defined in files publicly imported by this file.
+
+	// Comments, stored as a map of path (comma-separated integers) to the comment.
+	comments map[string]*descriptor.SourceCodeInfo_Location
+
+	// The full list of symbols that are exported,
+	// as a map from the exported object to its symbols.
+	// This is used for supporting public imports.
+	exported map[Object][]symbol
+
+	importPath  GoImportPath  // Import path of this file's package.
+	packageName GoPackageName // Name of this file's Go package.
+
+	proto3 bool // whether to generate proto3 code for this file
+}
+
+// VarName is the variable name we'll use in the generated code to refer
+// to the compressed bytes of this descriptor. It is not exported, so
+// it is only valid inside the generated package.
+func (d *FileDescriptor) VarName() string {
+	h := sha256.Sum256([]byte(d.GetName()))
+	return fmt.Sprintf("fileDescriptor_%s", hex.EncodeToString(h[:8]))
+}
+
+// goPackageOption interprets the file's go_package option.
+// If there is no go_package, it returns ("", "", false).
+// If there's a simple name, it returns ("", pkg, true).
+// If the option implies an import path, it returns (impPath, pkg, true).
+func (d *FileDescriptor) goPackageOption() (impPath GoImportPath, pkg GoPackageName, ok bool) {
+	opt := d.GetOptions().GetGoPackage()
+	if opt == "" {
+		return "", "", false
+	}
+	// A semicolon-delimited suffix delimits the import path and package name.
+	sc := strings.Index(opt, ";")
+	if sc >= 0 {
+		return GoImportPath(opt[:sc]), cleanPackageName(opt[sc+1:]), true
+	}
+	// The presence of a slash implies there's an import path.
+	slash := strings.LastIndex(opt, "/")
+	if slash >= 0 {
+		return GoImportPath(opt), cleanPackageName(opt[slash+1:]), true
+	}
+	return "", cleanPackageName(opt), true
+}
+
+// goFileName returns the output name for the generated Go file.
+func (d *FileDescriptor) goFileName(pathType pathType) string {
+	name := *d.Name
+	if ext := path.Ext(name); ext == ".proto" || ext == ".protodevel" {
+		name = name[:len(name)-len(ext)]
+	}
+	name += ".pb.go"
+
+	if pathType == pathTypeSourceRelative {
+		return name
+	}
+
+	// Does the file have a "go_package" option?
+	// If it does, it may override the filename.
+	if impPath, _, ok := d.goPackageOption(); ok && impPath != "" {
+		// Replace the existing dirname with the declared import path.
+		_, name = path.Split(name)
+		name = path.Join(string(impPath), name)
+		return name
+	}
+
+	return name
+}
+
+func (d *FileDescriptor) addExport(obj Object, sym symbol) {
+	d.exported[obj] = append(d.exported[obj], sym)
+}
+
+// symbol is an interface representing an exported Go symbol.
+type symbol interface {
+	// GenerateAlias should generate an appropriate alias
+	// for the symbol from the named package.
+	GenerateAlias(g *Generator, filename string, pkg GoPackageName)
+}
+
+type messageSymbol struct {
+	sym                         string
+	hasExtensions, isMessageSet bool
+	oneofTypes                  []string
+}
+
+type getterSymbol struct {
+	name     string
+	typ      string
+	typeName string // canonical name in proto world; empty for proto.Message and similar
+	genType  bool   // whether typ contains a generated type (message/group/enum)
+}
+
+func (ms *messageSymbol) GenerateAlias(g *Generator, filename string, pkg GoPackageName) {
+	g.P("// ", ms.sym, " from public import ", filename)
+	g.P("type ", ms.sym, " = ", pkg, ".", ms.sym)
+	for _, name := range ms.oneofTypes {
+		g.P("type ", name, " = ", pkg, ".", name)
+	}
+}
+
+type enumSymbol struct {
+	name   string
+	proto3 bool // Whether this came from a proto3 file.
+}
+
+func (es enumSymbol) GenerateAlias(g *Generator, filename string, pkg GoPackageName) {
+	s := es.name
+	g.P("// ", s, " from public import ", filename)
+	g.P("type ", s, " = ", pkg, ".", s)
+	g.P("var ", s, "_name = ", pkg, ".", s, "_name")
+	g.P("var ", s, "_value = ", pkg, ".", s, "_value")
+}
+
+type constOrVarSymbol struct {
+	sym  string
+	typ  string // either "const" or "var"
+	cast string // if non-empty, a type cast is required (used for enums)
+}
+
+func (cs constOrVarSymbol) GenerateAlias(g *Generator, filename string, pkg GoPackageName) {
+	v := string(pkg) + "." + cs.sym
+	if cs.cast != "" {
+		v = cs.cast + "(" + v + ")"
+	}
+	g.P(cs.typ, " ", cs.sym, " = ", v)
+}
+
+// Object is an interface abstracting the abilities shared by enums, messages, extensions and imported objects.
+type Object interface {
+	GoImportPath() GoImportPath
+	TypeName() []string
+	File() *FileDescriptor
+}
+
+// Generator is the type whose methods generate the output, stored in the associated response structure.
+type Generator struct {
+	*bytes.Buffer
+
+	Request  *plugin.CodeGeneratorRequest  // The input.
+	Response *plugin.CodeGeneratorResponse // The output.
+
+	Param             map[string]string // Command-line parameters.
+	PackageImportPath string            // Go import path of the package we're generating code for
+	ImportPrefix      string            // String to prefix to imported package file names.
+	ImportMap         map[string]string // Mapping from .proto file name to import path
+
+	Pkg map[string]string // The names under which we import support packages
+
+	outputImportPath GoImportPath                   // Package we're generating code for.
+	allFiles         []*FileDescriptor              // All files in the tree
+	allFilesByName   map[string]*FileDescriptor     // All files by filename.
+	genFiles         []*FileDescriptor              // Those files we will generate output for.
+	file             *FileDescriptor                // The file we are compiling now.
+	packageNames     map[GoImportPath]GoPackageName // Imported package names in the current file.
+	usedPackages     map[GoImportPath]bool          // Packages used in current file.
+	usedPackageNames map[GoPackageName]bool         // Package names used in the current file.
+	addedImports     map[GoImportPath]bool          // Additional imports to emit.
+	typeNameToObject map[string]Object              // Key is a fully-qualified name in input syntax.
+	init             []string                       // Lines to emit in the init function.
+	indent           string
+	pathType         pathType // How to generate output filenames.
+	writeOutput      bool
+	annotateCode     bool                                       // whether to store annotations
+	annotations      []*descriptor.GeneratedCodeInfo_Annotation // annotations to store
+}
+
+type pathType int
+
+const (
+	pathTypeImport pathType = iota
+	pathTypeSourceRelative
+)
+
+// New creates a new generator and allocates the request and response protobufs.
+func New() *Generator {
+	g := new(Generator)
+	g.Buffer = new(bytes.Buffer)
+	g.Request = new(plugin.CodeGeneratorRequest)
+	g.Response = new(plugin.CodeGeneratorResponse)
+	return g
+}
+
+// Error reports a problem, including an error, and exits the program.
+func (g *Generator) Error(err error, msgs ...string) {
+	s := strings.Join(msgs, " ") + ":" + err.Error()
+	log.Print("protoc-gen-go: error:", s)
+	os.Exit(1)
+}
+
+// Fail reports a problem and exits the program.
+func (g *Generator) Fail(msgs ...string) {
+	s := strings.Join(msgs, " ")
+	log.Print("protoc-gen-go: error:", s)
+	os.Exit(1)
+}
+
+// CommandLineParameters breaks the comma-separated list of key=value pairs
+// in the parameter (a member of the request protobuf) into a key/value map.
+// It then sets file name mappings defined by those entries.
+func (g *Generator) CommandLineParameters(parameter string) {
+	g.Param = make(map[string]string)
+	for _, p := range strings.Split(parameter, ",") {
+		if i := strings.Index(p, "="); i < 0 {
+			g.Param[p] = ""
+		} else {
+			g.Param[p[0:i]] = p[i+1:]
+		}
+	}
+
+	g.ImportMap = make(map[string]string)
+	pluginList := "none" // Default list of plugin names to enable (empty means all).
+	for k, v := range g.Param {
+		switch k {
+		case "import_prefix":
+			g.ImportPrefix = v
+		case "import_path":
+			g.PackageImportPath = v
+		case "paths":
+			switch v {
+			case "import":
+				g.pathType = pathTypeImport
+			case "source_relative":
+				g.pathType = pathTypeSourceRelative
+			default:
+				g.Fail(fmt.Sprintf(`Unknown path type %q: want "import" or "source_relative".`, v))
+			}
+		case "plugins":
+			pluginList = v
+		case "annotate_code":
+			if v == "true" {
+				g.annotateCode = true
+			}
+		default:
+			if len(k) > 0 && k[0] == 'M' {
+				g.ImportMap[k[1:]] = v
+			}
+		}
+	}
+	if pluginList != "" {
+		// Amend the set of plugins.
+		enabled := make(map[string]bool)
+		for _, name := range strings.Split(pluginList, "+") {
+			enabled[name] = true
+		}
+		var nplugins []Plugin
+		for _, p := range plugins {
+			if enabled[p.Name()] {
+				nplugins = append(nplugins, p)
+			}
+		}
+		plugins = nplugins
+	}
+}
+
+// DefaultPackageName returns the package name printed for the object.
+// If its file is in a different package, it returns the package name we're using for this file, plus ".".
+// Otherwise it returns the empty string.
+func (g *Generator) DefaultPackageName(obj Object) string {
+	importPath := obj.GoImportPath()
+	if importPath == g.outputImportPath {
+		return ""
+	}
+	return string(g.GoPackageName(importPath)) + "."
+}
+
+// GoPackageName returns the name used for a package.
+func (g *Generator) GoPackageName(importPath GoImportPath) GoPackageName {
+	if name, ok := g.packageNames[importPath]; ok {
+		return name
+	}
+	name := cleanPackageName(baseName(string(importPath)))
+	for i, orig := 1, name; g.usedPackageNames[name] || isGoPredeclaredIdentifier[string(name)]; i++ {
+		name = orig + GoPackageName(strconv.Itoa(i))
+	}
+	g.packageNames[importPath] = name
+	g.usedPackageNames[name] = true
+	return name
+}
+
+// AddImport adds a package to the generated file's import section.
+// It returns the name used for the package.
+func (g *Generator) AddImport(importPath GoImportPath) GoPackageName {
+	g.addedImports[importPath] = true
+	return g.GoPackageName(importPath)
+}
+
+var globalPackageNames = map[GoPackageName]bool{
+	"fmt":   true,
+	"math":  true,
+	"proto": true,
+}
+
+// Create and remember a guaranteed unique package name. Pkg is the candidate name.
+// The FileDescriptor parameter is unused.
+func RegisterUniquePackageName(pkg string, f *FileDescriptor) string {
+	name := cleanPackageName(pkg)
+	for i, orig := 1, name; globalPackageNames[name]; i++ {
+		name = orig + GoPackageName(strconv.Itoa(i))
+	}
+	globalPackageNames[name] = true
+	return string(name)
+}
+
+var isGoKeyword = map[string]bool{
+	"break":       true,
+	"case":        true,
+	"chan":        true,
+	"const":       true,
+	"continue":    true,
+	"default":     true,
+	"else":        true,
+	"defer":       true,
+	"fallthrough": true,
+	"for":         true,
+	"func":        true,
+	"go":          true,
+	"goto":        true,
+	"if":          true,
+	"import":      true,
+	"interface":   true,
+	"map":         true,
+	"package":     true,
+	"range":       true,
+	"return":      true,
+	"select":      true,
+	"struct":      true,
+	"switch":      true,
+	"type":        true,
+	"var":         true,
+}
+
+var isGoPredeclaredIdentifier = map[string]bool{
+	"append":     true,
+	"bool":       true,
+	"byte":       true,
+	"cap":        true,
+	"close":      true,
+	"complex":    true,
+	"complex128": true,
+	"complex64":  true,
+	"copy":       true,
+	"delete":     true,
+	"error":      true,
+	"false":      true,
+	"float32":    true,
+	"float64":    true,
+	"imag":       true,
+	"int":        true,
+	"int16":      true,
+	"int32":      true,
+	"int64":      true,
+	"int8":       true,
+	"iota":       true,
+	"len":        true,
+	"make":       true,
+	"new":        true,
+	"nil":        true,
+	"panic":      true,
+	"print":      true,
+	"println":    true,
+	"real":       true,
+	"recover":    true,
+	"rune":       true,
+	"string":     true,
+	"true":       true,
+	"uint":       true,
+	"uint16":     true,
+	"uint32":     true,
+	"uint64":     true,
+	"uint8":      true,
+	"uintptr":    true,
+}
+
+func cleanPackageName(name string) GoPackageName {
+	name = strings.Map(badToUnderscore, name)
+	// Identifier must not be keyword or predeclared identifier: insert _.
+	if isGoKeyword[name] {
+		name = "_" + name
+	}
+	// Identifier must not begin with digit: insert _.
+	if r, _ := utf8.DecodeRuneInString(name); unicode.IsDigit(r) {
+		name = "_" + name
+	}
+	return GoPackageName(name)
+}
+
+// defaultGoPackage returns the package name to use,
+// derived from the import path of the package we're building code for.
+func (g *Generator) defaultGoPackage() GoPackageName {
+	p := g.PackageImportPath
+	if i := strings.LastIndex(p, "/"); i >= 0 {
+		p = p[i+1:]
+	}
+	return cleanPackageName(p)
+}
+
+// SetPackageNames sets the package name for this run.
+// The package name must agree across all files being generated.
+// It also defines unique package names for all imported files.
+func (g *Generator) SetPackageNames() {
+	g.outputImportPath = g.genFiles[0].importPath
+
+	defaultPackageNames := make(map[GoImportPath]GoPackageName)
+	for _, f := range g.genFiles {
+		if _, p, ok := f.goPackageOption(); ok {
+			defaultPackageNames[f.importPath] = p
+		}
+	}
+	for _, f := range g.genFiles {
+		if _, p, ok := f.goPackageOption(); ok {
+			// Source file: option go_package = "quux/bar";
+			f.packageName = p
+		} else if p, ok := defaultPackageNames[f.importPath]; ok {
+			// A go_package option in another file in the same package.
+			//
+			// This is a poor choice in general, since every source file should
+			// contain a go_package option. Supported mainly for historical
+			// compatibility.
+			f.packageName = p
+		} else if p := g.defaultGoPackage(); p != "" {
+			// Command-line: import_path=quux/bar.
+			//
+			// The import_path flag sets a package name for files which don't
+			// contain a go_package option.
+			f.packageName = p
+		} else if p := f.GetPackage(); p != "" {
+			// Source file: package quux.bar;
+			f.packageName = cleanPackageName(p)
+		} else {
+			// Source filename.
+			f.packageName = cleanPackageName(baseName(f.GetName()))
+		}
+	}
+
+	// Check that all files have a consistent package name and import path.
+	for _, f := range g.genFiles[1:] {
+		if a, b := g.genFiles[0].importPath, f.importPath; a != b {
+			g.Fail(fmt.Sprintf("inconsistent package import paths: %v, %v", a, b))
+		}
+		if a, b := g.genFiles[0].packageName, f.packageName; a != b {
+			g.Fail(fmt.Sprintf("inconsistent package names: %v, %v", a, b))
+		}
+	}
+
+	// Names of support packages. These never vary (if there are conflicts,
+	// we rename the conflicting package), so this could be removed someday.
+	g.Pkg = map[string]string{
+		"fmt":   "fmt",
+		"math":  "math",
+		"proto": "proto",
+	}
+}
+
+// WrapTypes walks the incoming data, wrapping DescriptorProtos, EnumDescriptorProtos
+// and FileDescriptorProtos into file-referenced objects within the Generator.
+// It also creates the list of files to generate and so should be called before GenerateAllFiles.
+func (g *Generator) WrapTypes() {
+	g.allFiles = make([]*FileDescriptor, 0, len(g.Request.ProtoFile))
+	g.allFilesByName = make(map[string]*FileDescriptor, len(g.allFiles))
+	genFileNames := make(map[string]bool)
+	for _, n := range g.Request.FileToGenerate {
+		genFileNames[n] = true
+	}
+	for _, f := range g.Request.ProtoFile {
+		fd := &FileDescriptor{
+			FileDescriptorProto: f,
+			exported:            make(map[Object][]symbol),
+			proto3:              fileIsProto3(f),
+		}
+		// The import path may be set in a number of ways.
+		if substitution, ok := g.ImportMap[f.GetName()]; ok {
+			// Command-line: M=foo.proto=quux/bar.
+			//
+			// Explicit mapping of source file to import path.
+			fd.importPath = GoImportPath(substitution)
+		} else if genFileNames[f.GetName()] && g.PackageImportPath != "" {
+			// Command-line: import_path=quux/bar.
+			//
+			// The import_path flag sets the import path for every file that
+			// we generate code for.
+			fd.importPath = GoImportPath(g.PackageImportPath)
+		} else if p, _, _ := fd.goPackageOption(); p != "" {
+			// Source file: option go_package = "quux/bar";
+			//
+			// The go_package option sets the import path. Most users should use this.
+			fd.importPath = p
+		} else {
+			// Source filename.
+			//
+			// Last resort when nothing else is available.
+			fd.importPath = GoImportPath(path.Dir(f.GetName()))
+		}
+		// We must wrap the descriptors before we wrap the enums
+		fd.desc = wrapDescriptors(fd)
+		g.buildNestedDescriptors(fd.desc)
+		fd.enum = wrapEnumDescriptors(fd, fd.desc)
+		g.buildNestedEnums(fd.desc, fd.enum)
+		fd.ext = wrapExtensions(fd)
+		extractComments(fd)
+		g.allFiles = append(g.allFiles, fd)
+		g.allFilesByName[f.GetName()] = fd
+	}
+	for _, fd := range g.allFiles {
+		fd.imp = wrapImported(fd, g)
+	}
+
+	g.genFiles = make([]*FileDescriptor, 0, len(g.Request.FileToGenerate))
+	for _, fileName := range g.Request.FileToGenerate {
+		fd := g.allFilesByName[fileName]
+		if fd == nil {
+			g.Fail("could not find file named", fileName)
+		}
+		g.genFiles = append(g.genFiles, fd)
+	}
+}
+
+// Scan the descriptors in this file.  For each one, build the slice of nested descriptors
+func (g *Generator) buildNestedDescriptors(descs []*Descriptor) {
+	for _, desc := range descs {
+		if len(desc.NestedType) != 0 {
+			for _, nest := range descs {
+				if nest.parent == desc {
+					desc.nested = append(desc.nested, nest)
+				}
+			}
+			if len(desc.nested) != len(desc.NestedType) {
+				g.Fail("internal error: nesting failure for", desc.GetName())
+			}
+		}
+	}
+}
+
+func (g *Generator) buildNestedEnums(descs []*Descriptor, enums []*EnumDescriptor) {
+	for _, desc := range descs {
+		if len(desc.EnumType) != 0 {
+			for _, enum := range enums {
+				if enum.parent == desc {
+					desc.enums = append(desc.enums, enum)
+				}
+			}
+			if len(desc.enums) != len(desc.EnumType) {
+				g.Fail("internal error: enum nesting failure for", desc.GetName())
+			}
+		}
+	}
+}
+
+// Construct the Descriptor
+func newDescriptor(desc *descriptor.DescriptorProto, parent *Descriptor, file *FileDescriptor, index int) *Descriptor {
+	d := &Descriptor{
+		common:          common{file},
+		DescriptorProto: desc,
+		parent:          parent,
+		index:           index,
+	}
+	if parent == nil {
+		d.path = fmt.Sprintf("%d,%d", messagePath, index)
+	} else {
+		d.path = fmt.Sprintf("%s,%d,%d", parent.path, messageMessagePath, index)
+	}
+
+	// The only way to distinguish a group from a message is whether
+	// the containing message has a TYPE_GROUP field that matches.
+	if parent != nil {
+		parts := d.TypeName()
+		if file.Package != nil {
+			parts = append([]string{*file.Package}, parts...)
+		}
+		exp := "." + strings.Join(parts, ".")
+		for _, field := range parent.Field {
+			if field.GetType() == descriptor.FieldDescriptorProto_TYPE_GROUP && field.GetTypeName() == exp {
+				d.group = true
+				break
+			}
+		}
+	}
+
+	for _, field := range desc.Extension {
+		d.ext = append(d.ext, &ExtensionDescriptor{common{file}, field, d})
+	}
+
+	return d
+}
+
+// Return a slice of all the Descriptors defined within this file
+func wrapDescriptors(file *FileDescriptor) []*Descriptor {
+	sl := make([]*Descriptor, 0, len(file.MessageType)+10)
+	for i, desc := range file.MessageType {
+		sl = wrapThisDescriptor(sl, desc, nil, file, i)
+	}
+	return sl
+}
+
+// Wrap this Descriptor, recursively
+func wrapThisDescriptor(sl []*Descriptor, desc *descriptor.DescriptorProto, parent *Descriptor, file *FileDescriptor, index int) []*Descriptor {
+	sl = append(sl, newDescriptor(desc, parent, file, index))
+	me := sl[len(sl)-1]
+	for i, nested := range desc.NestedType {
+		sl = wrapThisDescriptor(sl, nested, me, file, i)
+	}
+	return sl
+}
+
+// Construct the EnumDescriptor
+func newEnumDescriptor(desc *descriptor.EnumDescriptorProto, parent *Descriptor, file *FileDescriptor, index int) *EnumDescriptor {
+	ed := &EnumDescriptor{
+		common:              common{file},
+		EnumDescriptorProto: desc,
+		parent:              parent,
+		index:               index,
+	}
+	if parent == nil {
+		ed.path = fmt.Sprintf("%d,%d", enumPath, index)
+	} else {
+		ed.path = fmt.Sprintf("%s,%d,%d", parent.path, messageEnumPath, index)
+	}
+	return ed
+}
+
+// Return a slice of all the EnumDescriptors defined within this file
+func wrapEnumDescriptors(file *FileDescriptor, descs []*Descriptor) []*EnumDescriptor {
+	sl := make([]*EnumDescriptor, 0, len(file.EnumType)+10)
+	// Top-level enums.
+	for i, enum := range file.EnumType {
+		sl = append(sl, newEnumDescriptor(enum, nil, file, i))
+	}
+	// Enums within messages. Enums within embedded messages appear in the outer-most message.
+	for _, nested := range descs {
+		for i, enum := range nested.EnumType {
+			sl = append(sl, newEnumDescriptor(enum, nested, file, i))
+		}
+	}
+	return sl
+}
+
+// Return a slice of all the top-level ExtensionDescriptors defined within this file.
+func wrapExtensions(file *FileDescriptor) []*ExtensionDescriptor {
+	var sl []*ExtensionDescriptor
+	for _, field := range file.Extension {
+		sl = append(sl, &ExtensionDescriptor{common{file}, field, nil})
+	}
+	return sl
+}
+
+// Return a slice of all the types that are publicly imported into this file.
+func wrapImported(file *FileDescriptor, g *Generator) (sl []*ImportedDescriptor) {
+	for _, index := range file.PublicDependency {
+		df := g.fileByName(file.Dependency[index])
+		for _, d := range df.desc {
+			if d.GetOptions().GetMapEntry() {
+				continue
+			}
+			sl = append(sl, &ImportedDescriptor{common{file}, d})
+		}
+		for _, e := range df.enum {
+			sl = append(sl, &ImportedDescriptor{common{file}, e})
+		}
+		for _, ext := range df.ext {
+			sl = append(sl, &ImportedDescriptor{common{file}, ext})
+		}
+	}
+	return
+}
+
+func extractComments(file *FileDescriptor) {
+	file.comments = make(map[string]*descriptor.SourceCodeInfo_Location)
+	for _, loc := range file.GetSourceCodeInfo().GetLocation() {
+		if loc.LeadingComments == nil {
+			continue
+		}
+		var p []string
+		for _, n := range loc.Path {
+			p = append(p, strconv.Itoa(int(n)))
+		}
+		file.comments[strings.Join(p, ",")] = loc
+	}
+}
+
+// BuildTypeNameMap builds the map from fully qualified type names to objects.
+// The key names for the map come from the input data, which puts a period at the beginning.
+// It should be called after SetPackageNames and before GenerateAllFiles.
+func (g *Generator) BuildTypeNameMap() {
+	g.typeNameToObject = make(map[string]Object)
+	for _, f := range g.allFiles {
+		// The names in this loop are defined by the proto world, not us, so the
+		// package name may be empty.  If so, the dotted package name of X will
+		// be ".X"; otherwise it will be ".pkg.X".
+		dottedPkg := "." + f.GetPackage()
+		if dottedPkg != "." {
+			dottedPkg += "."
+		}
+		for _, enum := range f.enum {
+			name := dottedPkg + dottedSlice(enum.TypeName())
+			g.typeNameToObject[name] = enum
+		}
+		for _, desc := range f.desc {
+			name := dottedPkg + dottedSlice(desc.TypeName())
+			g.typeNameToObject[name] = desc
+		}
+	}
+}
+
+// ObjectNamed, given a fully-qualified input type name as it appears in the input data,
+// returns the descriptor for the message or enum with that name.
+func (g *Generator) ObjectNamed(typeName string) Object {
+	o, ok := g.typeNameToObject[typeName]
+	if !ok {
+		g.Fail("can't find object with type", typeName)
+	}
+	return o
+}
+
+// AnnotatedAtoms is a list of atoms (as consumed by P) that records the file name and proto AST path from which they originated.
+type AnnotatedAtoms struct {
+	source string
+	path   string
+	atoms  []interface{}
+}
+
+// Annotate records the file name and proto AST path of a list of atoms
+// so that a later call to P can emit a link from each atom to its origin.
+func Annotate(file *FileDescriptor, path string, atoms ...interface{}) *AnnotatedAtoms {
+	return &AnnotatedAtoms{source: *file.Name, path: path, atoms: atoms}
+}
+
+// printAtom prints the (atomic, non-annotation) argument to the generated output.
+func (g *Generator) printAtom(v interface{}) {
+	switch v := v.(type) {
+	case string:
+		g.WriteString(v)
+	case *string:
+		g.WriteString(*v)
+	case bool:
+		fmt.Fprint(g, v)
+	case *bool:
+		fmt.Fprint(g, *v)
+	case int:
+		fmt.Fprint(g, v)
+	case *int32:
+		fmt.Fprint(g, *v)
+	case *int64:
+		fmt.Fprint(g, *v)
+	case float64:
+		fmt.Fprint(g, v)
+	case *float64:
+		fmt.Fprint(g, *v)
+	case GoPackageName:
+		g.WriteString(string(v))
+	case GoImportPath:
+		g.WriteString(strconv.Quote(string(v)))
+	default:
+		g.Fail(fmt.Sprintf("unknown type in printer: %T", v))
+	}
+}
+
+// P prints the arguments to the generated output.  It handles strings and int32s, plus
+// handling indirections because they may be *string, etc.  Any inputs of type AnnotatedAtoms may emit
+// annotations in a .meta file in addition to outputting the atoms themselves (if g.annotateCode
+// is true).
+func (g *Generator) P(str ...interface{}) {
+	if !g.writeOutput {
+		return
+	}
+	g.WriteString(g.indent)
+	for _, v := range str {
+		switch v := v.(type) {
+		case *AnnotatedAtoms:
+			begin := int32(g.Len())
+			for _, v := range v.atoms {
+				g.printAtom(v)
+			}
+			if g.annotateCode {
+				end := int32(g.Len())
+				var path []int32
+				for _, token := range strings.Split(v.path, ",") {
+					val, err := strconv.ParseInt(token, 10, 32)
+					if err != nil {
+						g.Fail("could not parse proto AST path: ", err.Error())
+					}
+					path = append(path, int32(val))
+				}
+				g.annotations = append(g.annotations, &descriptor.GeneratedCodeInfo_Annotation{
+					Path:       path,
+					SourceFile: &v.source,
+					Begin:      &begin,
+					End:        &end,
+				})
+			}
+		default:
+			g.printAtom(v)
+		}
+	}
+	g.WriteByte('\n')
+}
+
+// addInitf stores the given statement to be printed inside the file's init function.
+// The statement is given as a format specifier and arguments.
+func (g *Generator) addInitf(stmt string, a ...interface{}) {
+	g.init = append(g.init, fmt.Sprintf(stmt, a...))
+}
+
+// In Indents the output one tab stop.
+func (g *Generator) In() { g.indent += "\t" }
+
+// Out unindents the output one tab stop.
+func (g *Generator) Out() {
+	if len(g.indent) > 0 {
+		g.indent = g.indent[1:]
+	}
+}
+
+// GenerateAllFiles generates the output for all the files we're outputting.
+func (g *Generator) GenerateAllFiles() {
+	// Initialize the plugins
+	for _, p := range plugins {
+		p.Init(g)
+	}
+	// Generate the output. The generator runs for every file, even the files
+	// that we don't generate output for, so that we can collate the full list
+	// of exported symbols to support public imports.
+	genFileMap := make(map[*FileDescriptor]bool, len(g.genFiles))
+	for _, file := range g.genFiles {
+		genFileMap[file] = true
+	}
+	for _, file := range g.allFiles {
+		g.Reset()
+		g.annotations = nil
+		g.writeOutput = genFileMap[file]
+		g.generate(file)
+		if !g.writeOutput {
+			continue
+		}
+		fname := file.goFileName(g.pathType)
+		g.Response.File = append(g.Response.File, &plugin.CodeGeneratorResponse_File{
+			Name:    proto.String(fname),
+			Content: proto.String(g.String()),
+		})
+		if g.annotateCode {
+			// Store the generated code annotations in text, as the protoc plugin protocol requires that
+			// strings contain valid UTF-8.
+			g.Response.File = append(g.Response.File, &plugin.CodeGeneratorResponse_File{
+				Name:    proto.String(file.goFileName(g.pathType) + ".meta"),
+				Content: proto.String(proto.CompactTextString(&descriptor.GeneratedCodeInfo{Annotation: g.annotations})),
+			})
+		}
+	}
+}
+
+// Run all the plugins associated with the file.
+func (g *Generator) runPlugins(file *FileDescriptor) {
+	for _, p := range plugins {
+		p.Generate(file)
+	}
+}
+
+// Fill the response protocol buffer with the generated output for all the files we're
+// supposed to generate.
+func (g *Generator) generate(file *FileDescriptor) {
+	g.file = file
+	g.usedPackages = make(map[GoImportPath]bool)
+	g.packageNames = make(map[GoImportPath]GoPackageName)
+	g.usedPackageNames = make(map[GoPackageName]bool)
+	g.addedImports = make(map[GoImportPath]bool)
+	for name := range globalPackageNames {
+		g.usedPackageNames[name] = true
+	}
+
+	g.P("// This is a compile-time assertion to ensure that this generated file")
+	g.P("// is compatible with the proto package it is being compiled against.")
+	g.P("// A compilation error at this line likely means your copy of the")
+	g.P("// proto package needs to be updated.")
+	g.P("const _ = ", g.Pkg["proto"], ".ProtoPackageIsVersion", generatedCodeVersion, " // please upgrade the proto package")
+	g.P()
+
+	for _, td := range g.file.imp {
+		g.generateImported(td)
+	}
+	for _, enum := range g.file.enum {
+		g.generateEnum(enum)
+	}
+	for _, desc := range g.file.desc {
+		// Don't generate virtual messages for maps.
+		if desc.GetOptions().GetMapEntry() {
+			continue
+		}
+		g.generateMessage(desc)
+	}
+	for _, ext := range g.file.ext {
+		g.generateExtension(ext)
+	}
+	g.generateInitFunction()
+	g.generateFileDescriptor(file)
+
+	// Run the plugins before the imports so we know which imports are necessary.
+	g.runPlugins(file)
+
+	// Generate header and imports last, though they appear first in the output.
+	rem := g.Buffer
+	remAnno := g.annotations
+	g.Buffer = new(bytes.Buffer)
+	g.annotations = nil
+	g.generateHeader()
+	g.generateImports()
+	if !g.writeOutput {
+		return
+	}
+	// Adjust the offsets for annotations displaced by the header and imports.
+	for _, anno := range remAnno {
+		*anno.Begin += int32(g.Len())
+		*anno.End += int32(g.Len())
+		g.annotations = append(g.annotations, anno)
+	}
+	g.Write(rem.Bytes())
+
+	// Reformat generated code and patch annotation locations.
+	fset := token.NewFileSet()
+	original := g.Bytes()
+	if g.annotateCode {
+		// make a copy independent of g; we'll need it after Reset.
+		original = append([]byte(nil), original...)
+	}
+	fileAST, err := parser.ParseFile(fset, "", original, parser.ParseComments)
+	if err != nil {
+		// Print out the bad code with line numbers.
+		// This should never happen in practice, but it can while changing generated code,
+		// so consider this a debugging aid.
+		var src bytes.Buffer
+		s := bufio.NewScanner(bytes.NewReader(original))
+		for line := 1; s.Scan(); line++ {
+			fmt.Fprintf(&src, "%5d\t%s\n", line, s.Bytes())
+		}
+		g.Fail("bad Go source code was generated:", err.Error(), "\n"+src.String())
+	}
+	ast.SortImports(fset, fileAST)
+	g.Reset()
+	err = (&printer.Config{Mode: printer.TabIndent | printer.UseSpaces, Tabwidth: 8}).Fprint(g, fset, fileAST)
+	if err != nil {
+		g.Fail("generated Go source code could not be reformatted:", err.Error())
+	}
+	if g.annotateCode {
+		m, err := remap.Compute(original, g.Bytes())
+		if err != nil {
+			g.Fail("formatted generated Go source code could not be mapped back to the original code:", err.Error())
+		}
+		for _, anno := range g.annotations {
+			new, ok := m.Find(int(*anno.Begin), int(*anno.End))
+			if !ok {
+				g.Fail("span in formatted generated Go source code could not be mapped back to the original code")
+			}
+			*anno.Begin = int32(new.Pos)
+			*anno.End = int32(new.End)
+		}
+	}
+}
+
+// Generate the header, including package definition
+func (g *Generator) generateHeader() {
+	g.P("// Code generated by protoc-gen-go. DO NOT EDIT.")
+	if g.file.GetOptions().GetDeprecated() {
+		g.P("// ", g.file.Name, " is a deprecated file.")
+	} else {
+		g.P("// source: ", g.file.Name)
+	}
+	g.P()
+	g.PrintComments(strconv.Itoa(packagePath))
+	g.P()
+	g.P("package ", g.file.packageName)
+	g.P()
+}
+
+// deprecationComment is the standard comment added to deprecated
+// messages, fields, enums, and enum values.
+var deprecationComment = "// Deprecated: Do not use."
+
+// PrintComments prints any comments from the source .proto file.
+// The path is a comma-separated list of integers.
+// It returns an indication of whether any comments were printed.
+// See descriptor.proto for its format.
+func (g *Generator) PrintComments(path string) bool {
+	if !g.writeOutput {
+		return false
+	}
+	if c, ok := g.makeComments(path); ok {
+		g.P(c)
+		return true
+	}
+	return false
+}
+
+// makeComments generates the comment string for the field, no "\n" at the end
+func (g *Generator) makeComments(path string) (string, bool) {
+	loc, ok := g.file.comments[path]
+	if !ok {
+		return "", false
+	}
+	w := new(bytes.Buffer)
+	nl := ""
+	for _, line := range strings.Split(strings.TrimSuffix(loc.GetLeadingComments(), "\n"), "\n") {
+		fmt.Fprintf(w, "%s//%s", nl, line)
+		nl = "\n"
+	}
+	return w.String(), true
+}
+
+func (g *Generator) fileByName(filename string) *FileDescriptor {
+	return g.allFilesByName[filename]
+}
+
+// weak returns whether the ith import of the current file is a weak import.
+func (g *Generator) weak(i int32) bool {
+	for _, j := range g.file.WeakDependency {
+		if j == i {
+			return true
+		}
+	}
+	return false
+}
+
+// Generate the imports
+func (g *Generator) generateImports() {
+	imports := make(map[GoImportPath]GoPackageName)
+	for i, s := range g.file.Dependency {
+		fd := g.fileByName(s)
+		importPath := fd.importPath
+		// Do not import our own package.
+		if importPath == g.file.importPath {
+			continue
+		}
+		// Do not import weak imports.
+		if g.weak(int32(i)) {
+			continue
+		}
+		// Do not import a package twice.
+		if _, ok := imports[importPath]; ok {
+			continue
+		}
+		// We need to import all the dependencies, even if we don't reference them,
+		// because other code and tools depend on having the full transitive closure
+		// of protocol buffer types in the binary.
+		packageName := g.GoPackageName(importPath)
+		if _, ok := g.usedPackages[importPath]; !ok {
+			packageName = "_"
+		}
+		imports[importPath] = packageName
+	}
+	for importPath := range g.addedImports {
+		imports[importPath] = g.GoPackageName(importPath)
+	}
+	// We almost always need a proto import.  Rather than computing when we
+	// do, which is tricky when there's a plugin, just import it and
+	// reference it later. The same argument applies to the fmt and math packages.
+	g.P("import (")
+	g.P(g.Pkg["fmt"] + ` "fmt"`)
+	g.P(g.Pkg["math"] + ` "math"`)
+	g.P(g.Pkg["proto"]+" ", GoImportPath(g.ImportPrefix)+"github.com/golang/protobuf/proto")
+	for importPath, packageName := range imports {
+		g.P(packageName, " ", GoImportPath(g.ImportPrefix)+importPath)
+	}
+	g.P(")")
+	g.P()
+	// TODO: may need to worry about uniqueness across plugins
+	for _, p := range plugins {
+		p.GenerateImports(g.file)
+		g.P()
+	}
+	g.P("// Reference imports to suppress errors if they are not otherwise used.")
+	g.P("var _ = ", g.Pkg["proto"], ".Marshal")
+	g.P("var _ = ", g.Pkg["fmt"], ".Errorf")
+	g.P("var _ = ", g.Pkg["math"], ".Inf")
+	g.P()
+}
+
+func (g *Generator) generateImported(id *ImportedDescriptor) {
+	df := id.o.File()
+	filename := *df.Name
+	if df.importPath == g.file.importPath {
+		// Don't generate type aliases for files in the same Go package as this one.
+		return
+	}
+	if !supportTypeAliases {
+		g.Fail(fmt.Sprintf("%s: public imports require at least go1.9", filename))
+	}
+	g.usedPackages[df.importPath] = true
+
+	for _, sym := range df.exported[id.o] {
+		sym.GenerateAlias(g, filename, g.GoPackageName(df.importPath))
+	}
+
+	g.P()
+}
+
+// Generate the enum definitions for this EnumDescriptor.
+func (g *Generator) generateEnum(enum *EnumDescriptor) {
+	// The full type name
+	typeName := enum.TypeName()
+	// The full type name, CamelCased.
+	ccTypeName := CamelCaseSlice(typeName)
+	ccPrefix := enum.prefix()
+
+	deprecatedEnum := ""
+	if enum.GetOptions().GetDeprecated() {
+		deprecatedEnum = deprecationComment
+	}
+	g.PrintComments(enum.path)
+	g.P("type ", Annotate(enum.file, enum.path, ccTypeName), " int32", deprecatedEnum)
+	g.file.addExport(enum, enumSymbol{ccTypeName, enum.proto3()})
+	g.P("const (")
+	for i, e := range enum.Value {
+		etorPath := fmt.Sprintf("%s,%d,%d", enum.path, enumValuePath, i)
+		g.PrintComments(etorPath)
+
+		deprecatedValue := ""
+		if e.GetOptions().GetDeprecated() {
+			deprecatedValue = deprecationComment
+		}
+
+		name := ccPrefix + *e.Name
+		g.P(Annotate(enum.file, etorPath, name), " ", ccTypeName, " = ", e.Number, " ", deprecatedValue)
+		g.file.addExport(enum, constOrVarSymbol{name, "const", ccTypeName})
+	}
+	g.P(")")
+	g.P()
+	g.P("var ", ccTypeName, "_name = map[int32]string{")
+	generated := make(map[int32]bool) // avoid duplicate values
+	for _, e := range enum.Value {
+		duplicate := ""
+		if _, present := generated[*e.Number]; present {
+			duplicate = "// Duplicate value: "
+		}
+		g.P(duplicate, e.Number, ": ", strconv.Quote(*e.Name), ",")
+		generated[*e.Number] = true
+	}
+	g.P("}")
+	g.P()
+	g.P("var ", ccTypeName, "_value = map[string]int32{")
+	for _, e := range enum.Value {
+		g.P(strconv.Quote(*e.Name), ": ", e.Number, ",")
+	}
+	g.P("}")
+	g.P()
+
+	if !enum.proto3() {
+		g.P("func (x ", ccTypeName, ") Enum() *", ccTypeName, " {")
+		g.P("p := new(", ccTypeName, ")")
+		g.P("*p = x")
+		g.P("return p")
+		g.P("}")
+		g.P()
+	}
+
+	g.P("func (x ", ccTypeName, ") String() string {")
+	g.P("return ", g.Pkg["proto"], ".EnumName(", ccTypeName, "_name, int32(x))")
+	g.P("}")
+	g.P()
+
+	if !enum.proto3() {
+		g.P("func (x *", ccTypeName, ") UnmarshalJSON(data []byte) error {")
+		g.P("value, err := ", g.Pkg["proto"], ".UnmarshalJSONEnum(", ccTypeName, `_value, data, "`, ccTypeName, `")`)
+		g.P("if err != nil {")
+		g.P("return err")
+		g.P("}")
+		g.P("*x = ", ccTypeName, "(value)")
+		g.P("return nil")
+		g.P("}")
+		g.P()
+	}
+
+	var indexes []string
+	for m := enum.parent; m != nil; m = m.parent {
+		// XXX: skip groups?
+		indexes = append([]string{strconv.Itoa(m.index)}, indexes...)
+	}
+	indexes = append(indexes, strconv.Itoa(enum.index))
+	g.P("func (", ccTypeName, ") EnumDescriptor() ([]byte, []int) {")
+	g.P("return ", g.file.VarName(), ", []int{", strings.Join(indexes, ", "), "}")
+	g.P("}")
+	g.P()
+	if enum.file.GetPackage() == "google.protobuf" && enum.GetName() == "NullValue" {
+		g.P("func (", ccTypeName, `) XXX_WellKnownType() string { return "`, enum.GetName(), `" }`)
+		g.P()
+	}
+
+	g.generateEnumRegistration(enum)
+}
+
+// The tag is a string like "varint,2,opt,name=fieldname,def=7" that
+// identifies details of the field for the protocol buffer marshaling and unmarshaling
+// code.  The fields are:
+//	wire encoding
+//	protocol tag number
+//	opt,req,rep for optional, required, or repeated
+//	packed whether the encoding is "packed" (optional; repeated primitives only)
+//	name= the original declared name
+//	enum= the name of the enum type if it is an enum-typed field.
+//	proto3 if this field is in a proto3 message
+//	def= string representation of the default value, if any.
+// The default value must be in a representation that can be used at run-time
+// to generate the default value. Thus bools become 0 and 1, for instance.
+func (g *Generator) goTag(message *Descriptor, field *descriptor.FieldDescriptorProto, wiretype string) string {
+	optrepreq := ""
+	switch {
+	case isOptional(field):
+		optrepreq = "opt"
+	case isRequired(field):
+		optrepreq = "req"
+	case isRepeated(field):
+		optrepreq = "rep"
+	}
+	var defaultValue string
+	if dv := field.DefaultValue; dv != nil { // set means an explicit default
+		defaultValue = *dv
+		// Some types need tweaking.
+		switch *field.Type {
+		case descriptor.FieldDescriptorProto_TYPE_BOOL:
+			if defaultValue == "true" {
+				defaultValue = "1"
+			} else {
+				defaultValue = "0"
+			}
+		case descriptor.FieldDescriptorProto_TYPE_STRING,
+			descriptor.FieldDescriptorProto_TYPE_BYTES:
+			// Nothing to do. Quoting is done for the whole tag.
+		case descriptor.FieldDescriptorProto_TYPE_ENUM:
+			// For enums we need to provide the integer constant.
+			obj := g.ObjectNamed(field.GetTypeName())
+			if id, ok := obj.(*ImportedDescriptor); ok {
+				// It is an enum that was publicly imported.
+				// We need the underlying type.
+				obj = id.o
+			}
+			enum, ok := obj.(*EnumDescriptor)
+			if !ok {
+				log.Printf("obj is a %T", obj)
+				if id, ok := obj.(*ImportedDescriptor); ok {
+					log.Printf("id.o is a %T", id.o)
+				}
+				g.Fail("unknown enum type", CamelCaseSlice(obj.TypeName()))
+			}
+			defaultValue = enum.integerValueAsString(defaultValue)
+		case descriptor.FieldDescriptorProto_TYPE_FLOAT:
+			if def := defaultValue; def != "inf" && def != "-inf" && def != "nan" {
+				if f, err := strconv.ParseFloat(defaultValue, 32); err == nil {
+					defaultValue = fmt.Sprint(float32(f))
+				}
+			}
+		case descriptor.FieldDescriptorProto_TYPE_DOUBLE:
+			if def := defaultValue; def != "inf" && def != "-inf" && def != "nan" {
+				if f, err := strconv.ParseFloat(defaultValue, 64); err == nil {
+					defaultValue = fmt.Sprint(f)
+				}
+			}
+		}
+		defaultValue = ",def=" + defaultValue
+	}
+	enum := ""
+	if *field.Type == descriptor.FieldDescriptorProto_TYPE_ENUM {
+		// We avoid using obj.GoPackageName(), because we want to use the
+		// original (proto-world) package name.
+		obj := g.ObjectNamed(field.GetTypeName())
+		if id, ok := obj.(*ImportedDescriptor); ok {
+			obj = id.o
+		}
+		enum = ",enum="
+		if pkg := obj.File().GetPackage(); pkg != "" {
+			enum += pkg + "."
+		}
+		enum += CamelCaseSlice(obj.TypeName())
+	}
+	packed := ""
+	if (field.Options != nil && field.Options.GetPacked()) ||
+		// Per https://developers.google.com/protocol-buffers/docs/proto3#simple:
+		// "In proto3, repeated fields of scalar numeric types use packed encoding by default."
+		(message.proto3() && (field.Options == nil || field.Options.Packed == nil) &&
+			isRepeated(field) && isScalar(field)) {
+		packed = ",packed"
+	}
+	fieldName := field.GetName()
+	name := fieldName
+	if *field.Type == descriptor.FieldDescriptorProto_TYPE_GROUP {
+		// We must use the type name for groups instead of
+		// the field name to preserve capitalization.
+		// type_name in FieldDescriptorProto is fully-qualified,
+		// but we only want the local part.
+		name = *field.TypeName
+		if i := strings.LastIndex(name, "."); i >= 0 {
+			name = name[i+1:]
+		}
+	}
+	if json := field.GetJsonName(); field.Extendee == nil && json != "" && json != name {
+		// TODO: escaping might be needed, in which case
+		// perhaps this should be in its own "json" tag.
+		name += ",json=" + json
+	}
+	name = ",name=" + name
+	if message.proto3() {
+		name += ",proto3"
+	}
+	oneof := ""
+	if field.OneofIndex != nil {
+		oneof = ",oneof"
+	}
+	return strconv.Quote(fmt.Sprintf("%s,%d,%s%s%s%s%s%s",
+		wiretype,
+		field.GetNumber(),
+		optrepreq,
+		packed,
+		name,
+		enum,
+		oneof,
+		defaultValue))
+}
+
+func needsStar(typ descriptor.FieldDescriptorProto_Type) bool {
+	switch typ {
+	case descriptor.FieldDescriptorProto_TYPE_GROUP:
+		return false
+	case descriptor.FieldDescriptorProto_TYPE_MESSAGE:
+		return false
+	case descriptor.FieldDescriptorProto_TYPE_BYTES:
+		return false
+	}
+	return true
+}
+
+// TypeName is the printed name appropriate for an item. If the object is in the current file,
+// TypeName drops the package name and underscores the rest.
+// Otherwise the object is from another package; and the result is the underscored
+// package name followed by the item name.
+// The result always has an initial capital.
+func (g *Generator) TypeName(obj Object) string {
+	return g.DefaultPackageName(obj) + CamelCaseSlice(obj.TypeName())
+}
+
+// GoType returns a string representing the type name, and the wire type
+func (g *Generator) GoType(message *Descriptor, field *descriptor.FieldDescriptorProto) (typ string, wire string) {
+	// TODO: Options.
+	switch *field.Type {
+	case descriptor.FieldDescriptorProto_TYPE_DOUBLE:
+		typ, wire = "float64", "fixed64"
+	case descriptor.FieldDescriptorProto_TYPE_FLOAT:
+		typ, wire = "float32", "fixed32"
+	case descriptor.FieldDescriptorProto_TYPE_INT64:
+		typ, wire = "int64", "varint"
+	case descriptor.FieldDescriptorProto_TYPE_UINT64:
+		typ, wire = "uint64", "varint"
+	case descriptor.FieldDescriptorProto_TYPE_INT32:
+		typ, wire = "int32", "varint"
+	case descriptor.FieldDescriptorProto_TYPE_UINT32:
+		typ, wire = "uint32", "varint"
+	case descriptor.FieldDescriptorProto_TYPE_FIXED64:
+		typ, wire = "uint64", "fixed64"
+	case descriptor.FieldDescriptorProto_TYPE_FIXED32:
+		typ, wire = "uint32", "fixed32"
+	case descriptor.FieldDescriptorProto_TYPE_BOOL:
+		typ, wire = "bool", "varint"
+	case descriptor.FieldDescriptorProto_TYPE_STRING:
+		typ, wire = "string", "bytes"
+	case descriptor.FieldDescriptorProto_TYPE_GROUP:
+		desc := g.ObjectNamed(field.GetTypeName())
+		typ, wire = "*"+g.TypeName(desc), "group"
+	case descriptor.FieldDescriptorProto_TYPE_MESSAGE:
+		desc := g.ObjectNamed(field.GetTypeName())
+		typ, wire = "*"+g.TypeName(desc), "bytes"
+	case descriptor.FieldDescriptorProto_TYPE_BYTES:
+		typ, wire = "[]byte", "bytes"
+	case descriptor.FieldDescriptorProto_TYPE_ENUM:
+		desc := g.ObjectNamed(field.GetTypeName())
+		typ, wire = g.TypeName(desc), "varint"
+	case descriptor.FieldDescriptorProto_TYPE_SFIXED32:
+		typ, wire = "int32", "fixed32"
+	case descriptor.FieldDescriptorProto_TYPE_SFIXED64:
+		typ, wire = "int64", "fixed64"
+	case descriptor.FieldDescriptorProto_TYPE_SINT32:
+		typ, wire = "int32", "zigzag32"
+	case descriptor.FieldDescriptorProto_TYPE_SINT64:
+		typ, wire = "int64", "zigzag64"
+	default:
+		g.Fail("unknown type for", field.GetName())
+	}
+	if isRepeated(field) {
+		typ = "[]" + typ
+	} else if message != nil && message.proto3() {
+		return
+	} else if field.OneofIndex != nil && message != nil {
+		return
+	} else if needsStar(*field.Type) {
+		typ = "*" + typ
+	}
+	return
+}
+
+func (g *Generator) RecordTypeUse(t string) {
+	if _, ok := g.typeNameToObject[t]; !ok {
+		return
+	}
+	importPath := g.ObjectNamed(t).GoImportPath()
+	if importPath == g.outputImportPath {
+		// Don't record use of objects in our package.
+		return
+	}
+	g.AddImport(importPath)
+	g.usedPackages[importPath] = true
+}
+
+// Method names that may be generated.  Fields with these names get an
+// underscore appended. Any change to this set is a potential incompatible
+// API change because it changes generated field names.
+var methodNames = [...]string{
+	"Reset",
+	"String",
+	"ProtoMessage",
+	"Marshal",
+	"Unmarshal",
+	"ExtensionRangeArray",
+	"ExtensionMap",
+	"Descriptor",
+}
+
+// Names of messages in the `google.protobuf` package for which
+// we will generate XXX_WellKnownType methods.
+var wellKnownTypes = map[string]bool{
+	"Any":       true,
+	"Duration":  true,
+	"Empty":     true,
+	"Struct":    true,
+	"Timestamp": true,
+
+	"Value":       true,
+	"ListValue":   true,
+	"DoubleValue": true,
+	"FloatValue":  true,
+	"Int64Value":  true,
+	"UInt64Value": true,
+	"Int32Value":  true,
+	"UInt32Value": true,
+	"BoolValue":   true,
+	"StringValue": true,
+	"BytesValue":  true,
+}
+
+// getterDefault finds the default value for the field to return from a getter,
+// regardless of if it's a built in default or explicit from the source. Returns e.g. "nil", `""`, "Default_MessageType_FieldName"
+func (g *Generator) getterDefault(field *descriptor.FieldDescriptorProto, goMessageType string) string {
+	if isRepeated(field) {
+		return "nil"
+	}
+	if def := field.GetDefaultValue(); def != "" {
+		defaultConstant := g.defaultConstantName(goMessageType, field.GetName())
+		if *field.Type != descriptor.FieldDescriptorProto_TYPE_BYTES {
+			return defaultConstant
+		}
+		return "append([]byte(nil), " + defaultConstant + "...)"
+	}
+	switch *field.Type {
+	case descriptor.FieldDescriptorProto_TYPE_BOOL:
+		return "false"
+	case descriptor.FieldDescriptorProto_TYPE_STRING:
+		return `""`
+	case descriptor.FieldDescriptorProto_TYPE_GROUP, descriptor.FieldDescriptorProto_TYPE_MESSAGE, descriptor.FieldDescriptorProto_TYPE_BYTES:
+		return "nil"
+	case descriptor.FieldDescriptorProto_TYPE_ENUM:
+		obj := g.ObjectNamed(field.GetTypeName())
+		var enum *EnumDescriptor
+		if id, ok := obj.(*ImportedDescriptor); ok {
+			// The enum type has been publicly imported.
+			enum, _ = id.o.(*EnumDescriptor)
+		} else {
+			enum, _ = obj.(*EnumDescriptor)
+		}
+		if enum == nil {
+			log.Printf("don't know how to generate getter for %s", field.GetName())
+			return "nil"
+		}
+		if len(enum.Value) == 0 {
+			return "0 // empty enum"
+		}
+		first := enum.Value[0].GetName()
+		return g.DefaultPackageName(obj) + enum.prefix() + first
+	default:
+		return "0"
+	}
+}
+
+// defaultConstantName builds the name of the default constant from the message
+// type name and the untouched field name, e.g. "Default_MessageType_FieldName"
+func (g *Generator) defaultConstantName(goMessageType, protoFieldName string) string {
+	return "Default_" + goMessageType + "_" + CamelCase(protoFieldName)
+}
+
+// The different types of fields in a message and how to actually print them
+// Most of the logic for generateMessage is in the methods of these types.
+//
+// Note that the content of the field is irrelevant, a simpleField can contain
+// anything from a scalar to a group (which is just a message).
+//
+// Extension fields (and message sets) are however handled separately.
+//
+// simpleField - a field that is neiter weak nor oneof, possibly repeated
+// oneofField - field containing list of subfields:
+// - oneofSubField - a field within the oneof
+
+// msgCtx contains the context for the generator functions.
+type msgCtx struct {
+	goName  string      // Go struct name of the message, e.g. MessageName
+	message *Descriptor // The descriptor for the message
+}
+
+// fieldCommon contains data common to all types of fields.
+type fieldCommon struct {
+	goName     string // Go name of field, e.g. "FieldName" or "Descriptor_"
+	protoName  string // Name of field in proto language, e.g. "field_name" or "descriptor"
+	getterName string // Name of the getter, e.g. "GetFieldName" or "GetDescriptor_"
+	goType     string // The Go type as a string, e.g. "*int32" or "*OtherMessage"
+	tags       string // The tag string/annotation for the type, e.g. `protobuf:"varint,8,opt,name=region_id,json=regionId"`
+	fullPath   string // The full path of the field as used by Annotate etc, e.g. "4,0,2,0"
+}
+
+// getProtoName gets the proto name of a field, e.g. "field_name" or "descriptor".
+func (f *fieldCommon) getProtoName() string {
+	return f.protoName
+}
+
+// getGoType returns the go type of the field  as a string, e.g. "*int32".
+func (f *fieldCommon) getGoType() string {
+	return f.goType
+}
+
+// simpleField is not weak, not a oneof, not an extension. Can be required, optional or repeated.
+type simpleField struct {
+	fieldCommon
+	protoTypeName string                               // Proto type name, empty if primitive, e.g. ".google.protobuf.Duration"
+	protoType     descriptor.FieldDescriptorProto_Type // Actual type enum value, e.g. descriptor.FieldDescriptorProto_TYPE_FIXED64
+	deprecated    string                               // Deprecation comment, if any, e.g. "// Deprecated: Do not use."
+	getterDef     string                               // Default for getters, e.g. "nil", `""` or "Default_MessageType_FieldName"
+	protoDef      string                               // Default value as defined in the proto file, e.g "yoshi" or "5"
+	comment       string                               // The full comment for the field, e.g. "// Useful information"
+}
+
+// decl prints the declaration of the field in the struct (if any).
+func (f *simpleField) decl(g *Generator, mc *msgCtx) {
+	g.P(f.comment, Annotate(mc.message.file, f.fullPath, f.goName), "\t", f.goType, "\t`", f.tags, "`", f.deprecated)
+}
+
+// getter prints the getter for the field.
+func (f *simpleField) getter(g *Generator, mc *msgCtx) {
+	star := ""
+	tname := f.goType
+	if needsStar(f.protoType) && tname[0] == '*' {
+		tname = tname[1:]
+		star = "*"
+	}
+	if f.deprecated != "" {
+		g.P(f.deprecated)
+	}
+	g.P("func (m *", mc.goName, ") ", Annotate(mc.message.file, f.fullPath, f.getterName), "() "+tname+" {")
+	if f.getterDef == "nil" { // Simpler getter
+		g.P("if m != nil {")
+		g.P("return m." + f.goName)
+		g.P("}")
+		g.P("return nil")
+		g.P("}")
+		g.P()
+		return
+	}
+	if mc.message.proto3() {
+		g.P("if m != nil {")
+	} else {
+		g.P("if m != nil && m." + f.goName + " != nil {")
+	}
+	g.P("return " + star + "m." + f.goName)
+	g.P("}")
+	g.P("return ", f.getterDef)
+	g.P("}")
+	g.P()
+}
+
+// setter prints the setter method of the field.
+func (f *simpleField) setter(g *Generator, mc *msgCtx) {
+	// No setter for regular fields yet
+}
+
+// getProtoDef returns the default value explicitly stated in the proto file, e.g "yoshi" or "5".
+func (f *simpleField) getProtoDef() string {
+	return f.protoDef
+}
+
+// getProtoTypeName returns the protobuf type name for the field as returned by field.GetTypeName(), e.g. ".google.protobuf.Duration".
+func (f *simpleField) getProtoTypeName() string {
+	return f.protoTypeName
+}
+
+// getProtoType returns the *field.Type value, e.g. descriptor.FieldDescriptorProto_TYPE_FIXED64.
+func (f *simpleField) getProtoType() descriptor.FieldDescriptorProto_Type {
+	return f.protoType
+}
+
+// oneofSubFields are kept slize held by each oneofField. They do not appear in the top level slize of fields for the message.
+type oneofSubField struct {
+	fieldCommon
+	protoTypeName string                               // Proto type name, empty if primitive, e.g. ".google.protobuf.Duration"
+	protoType     descriptor.FieldDescriptorProto_Type // Actual type enum value, e.g. descriptor.FieldDescriptorProto_TYPE_FIXED64
+	oneofTypeName string                               // Type name of the enclosing struct, e.g. "MessageName_FieldName"
+	fieldNumber   int                                  // Actual field number, as defined in proto, e.g. 12
+	getterDef     string                               // Default for getters, e.g. "nil", `""` or "Default_MessageType_FieldName"
+	protoDef      string                               // Default value as defined in the proto file, e.g "yoshi" or "5"
+	deprecated    string                               // Deprecation comment, if any.
+}
+
+// typedNil prints a nil casted to the pointer to this field.
+// - for XXX_OneofWrappers
+func (f *oneofSubField) typedNil(g *Generator) {
+	g.P("(*", f.oneofTypeName, ")(nil),")
+}
+
+// getProtoDef returns the default value explicitly stated in the proto file, e.g "yoshi" or "5".
+func (f *oneofSubField) getProtoDef() string {
+	return f.protoDef
+}
+
+// getProtoTypeName returns the protobuf type name for the field as returned by field.GetTypeName(), e.g. ".google.protobuf.Duration".
+func (f *oneofSubField) getProtoTypeName() string {
+	return f.protoTypeName
+}
+
+// getProtoType returns the *field.Type value, e.g. descriptor.FieldDescriptorProto_TYPE_FIXED64.
+func (f *oneofSubField) getProtoType() descriptor.FieldDescriptorProto_Type {
+	return f.protoType
+}
+
+// oneofField represents the oneof on top level.
+// The alternative fields within the oneof are represented by oneofSubField.
+type oneofField struct {
+	fieldCommon
+	subFields []*oneofSubField // All the possible oneof fields
+	comment   string           // The full comment for the field, e.g. "// Types that are valid to be assigned to MyOneof:\n\\"
+}
+
+// decl prints the declaration of the field in the struct (if any).
+func (f *oneofField) decl(g *Generator, mc *msgCtx) {
+	comment := f.comment
+	for _, sf := range f.subFields {
+		comment += "//\t*" + sf.oneofTypeName + "\n"
+	}
+	g.P(comment, Annotate(mc.message.file, f.fullPath, f.goName), " ", f.goType, " `", f.tags, "`")
+}
+
+// getter for a oneof field will print additional discriminators and interfaces for the oneof,
+// also it prints all the getters for the sub fields.
+func (f *oneofField) getter(g *Generator, mc *msgCtx) {
+	// The discriminator type
+	g.P("type ", f.goType, " interface {")
+	g.P(f.goType, "()")
+	g.P("}")
+	g.P()
+	// The subField types, fulfilling the discriminator type contract
+	for _, sf := range f.subFields {
+		g.P("type ", Annotate(mc.message.file, sf.fullPath, sf.oneofTypeName), " struct {")
+		g.P(Annotate(mc.message.file, sf.fullPath, sf.goName), " ", sf.goType, " `", sf.tags, "`")
+		g.P("}")
+		g.P()
+	}
+	for _, sf := range f.subFields {
+		g.P("func (*", sf.oneofTypeName, ") ", f.goType, "() {}")
+		g.P()
+	}
+	// Getter for the oneof field
+	g.P("func (m *", mc.goName, ") ", Annotate(mc.message.file, f.fullPath, f.getterName), "() ", f.goType, " {")
+	g.P("if m != nil { return m.", f.goName, " }")
+	g.P("return nil")
+	g.P("}")
+	g.P()
+	// Getters for each oneof
+	for _, sf := range f.subFields {
+		if sf.deprecated != "" {
+			g.P(sf.deprecated)
+		}
+		g.P("func (m *", mc.goName, ") ", Annotate(mc.message.file, sf.fullPath, sf.getterName), "() "+sf.goType+" {")
+		g.P("if x, ok := m.", f.getterName, "().(*", sf.oneofTypeName, "); ok {")
+		g.P("return x.", sf.goName)
+		g.P("}")
+		g.P("return ", sf.getterDef)
+		g.P("}")
+		g.P()
+	}
+}
+
+// setter prints the setter method of the field.
+func (f *oneofField) setter(g *Generator, mc *msgCtx) {
+	// No setters for oneof yet
+}
+
+// topLevelField interface implemented by all types of fields on the top level (not oneofSubField).
+type topLevelField interface {
+	decl(g *Generator, mc *msgCtx)   // print declaration within the struct
+	getter(g *Generator, mc *msgCtx) // print getter
+	setter(g *Generator, mc *msgCtx) // print setter if applicable
+}
+
+// defField interface implemented by all types of fields that can have defaults (not oneofField, but instead oneofSubField).
+type defField interface {
+	getProtoDef() string                                // default value explicitly stated in the proto file, e.g "yoshi" or "5"
+	getProtoName() string                               // proto name of a field, e.g. "field_name" or "descriptor"
+	getGoType() string                                  // go type of the field  as a string, e.g. "*int32"
+	getProtoTypeName() string                           // protobuf type name for the field, e.g. ".google.protobuf.Duration"
+	getProtoType() descriptor.FieldDescriptorProto_Type // *field.Type value, e.g. descriptor.FieldDescriptorProto_TYPE_FIXED64
+}
+
+// generateDefaultConstants adds constants for default values if needed, which is only if the default value is.
+// explicit in the proto.
+func (g *Generator) generateDefaultConstants(mc *msgCtx, topLevelFields []topLevelField) {
+	// Collect fields that can have defaults
+	dFields := []defField{}
+	for _, pf := range topLevelFields {
+		if f, ok := pf.(*oneofField); ok {
+			for _, osf := range f.subFields {
+				dFields = append(dFields, osf)
+			}
+			continue
+		}
+		dFields = append(dFields, pf.(defField))
+	}
+	for _, df := range dFields {
+		def := df.getProtoDef()
+		if def == "" {
+			continue
+		}
+		fieldname := g.defaultConstantName(mc.goName, df.getProtoName())
+		typename := df.getGoType()
+		if typename[0] == '*' {
+			typename = typename[1:]
+		}
+		kind := "const "
+		switch {
+		case typename == "bool":
+		case typename == "string":
+			def = strconv.Quote(def)
+		case typename == "[]byte":
+			def = "[]byte(" + strconv.Quote(unescape(def)) + ")"
+			kind = "var "
+		case def == "inf", def == "-inf", def == "nan":
+			// These names are known to, and defined by, the protocol language.
+			switch def {
+			case "inf":
+				def = "math.Inf(1)"
+			case "-inf":
+				def = "math.Inf(-1)"
+			case "nan":
+				def = "math.NaN()"
+			}
+			if df.getProtoType() == descriptor.FieldDescriptorProto_TYPE_FLOAT {
+				def = "float32(" + def + ")"
+			}
+			kind = "var "
+		case df.getProtoType() == descriptor.FieldDescriptorProto_TYPE_FLOAT:
+			if f, err := strconv.ParseFloat(def, 32); err == nil {
+				def = fmt.Sprint(float32(f))
+			}
+		case df.getProtoType() == descriptor.FieldDescriptorProto_TYPE_DOUBLE:
+			if f, err := strconv.ParseFloat(def, 64); err == nil {
+				def = fmt.Sprint(f)
+			}
+		case df.getProtoType() == descriptor.FieldDescriptorProto_TYPE_ENUM:
+			// Must be an enum.  Need to construct the prefixed name.
+			obj := g.ObjectNamed(df.getProtoTypeName())
+			var enum *EnumDescriptor
+			if id, ok := obj.(*ImportedDescriptor); ok {
+				// The enum type has been publicly imported.
+				enum, _ = id.o.(*EnumDescriptor)
+			} else {
+				enum, _ = obj.(*EnumDescriptor)
+			}
+			if enum == nil {
+				log.Printf("don't know how to generate constant for %s", fieldname)
+				continue
+			}
+			def = g.DefaultPackageName(obj) + enum.prefix() + def
+		}
+		g.P(kind, fieldname, " ", typename, " = ", def)
+		g.file.addExport(mc.message, constOrVarSymbol{fieldname, kind, ""})
+	}
+	g.P()
+}
+
+// generateInternalStructFields just adds the XXX_<something> fields to the message struct.
+func (g *Generator) generateInternalStructFields(mc *msgCtx, topLevelFields []topLevelField) {
+	g.P("XXX_NoUnkeyedLiteral\tstruct{} `json:\"-\"`") // prevent unkeyed struct literals
+	if len(mc.message.ExtensionRange) > 0 {
+		messageset := ""
+		if opts := mc.message.Options; opts != nil && opts.GetMessageSetWireFormat() {
+			messageset = "protobuf_messageset:\"1\" "
+		}
+		g.P(g.Pkg["proto"], ".XXX_InternalExtensions `", messageset, "json:\"-\"`")
+	}
+	g.P("XXX_unrecognized\t[]byte `json:\"-\"`")
+	g.P("XXX_sizecache\tint32 `json:\"-\"`")
+
+}
+
+// generateOneofFuncs adds all the utility functions for oneof, including marshalling, unmarshalling and sizer.
+func (g *Generator) generateOneofFuncs(mc *msgCtx, topLevelFields []topLevelField) {
+	ofields := []*oneofField{}
+	for _, f := range topLevelFields {
+		if o, ok := f.(*oneofField); ok {
+			ofields = append(ofields, o)
+		}
+	}
+	if len(ofields) == 0 {
+		return
+	}
+
+	// OneofFuncs
+	g.P("// XXX_OneofWrappers is for the internal use of the proto package.")
+	g.P("func (*", mc.goName, ") XXX_OneofWrappers() []interface{} {")
+	g.P("return []interface{}{")
+	for _, of := range ofields {
+		for _, sf := range of.subFields {
+			sf.typedNil(g)
+		}
+	}
+	g.P("}")
+	g.P("}")
+	g.P()
+}
+
+// generateMessageStruct adds the actual struct with it's members (but not methods) to the output.
+func (g *Generator) generateMessageStruct(mc *msgCtx, topLevelFields []topLevelField) {
+	comments := g.PrintComments(mc.message.path)
+
+	// Guarantee deprecation comments appear after user-provided comments.
+	if mc.message.GetOptions().GetDeprecated() {
+		if comments {
+			// Convention: Separate deprecation comments from original
+			// comments with an empty line.
+			g.P("//")
+		}
+		g.P(deprecationComment)
+	}
+
+	g.P("type ", Annotate(mc.message.file, mc.message.path, mc.goName), " struct {")
+	for _, pf := range topLevelFields {
+		pf.decl(g, mc)
+	}
+	g.generateInternalStructFields(mc, topLevelFields)
+	g.P("}")
+}
+
+// generateGetters adds getters for all fields, including oneofs and weak fields when applicable.
+func (g *Generator) generateGetters(mc *msgCtx, topLevelFields []topLevelField) {
+	for _, pf := range topLevelFields {
+		pf.getter(g, mc)
+	}
+}
+
+// generateSetters add setters for all fields, including oneofs and weak fields when applicable.
+func (g *Generator) generateSetters(mc *msgCtx, topLevelFields []topLevelField) {
+	for _, pf := range topLevelFields {
+		pf.setter(g, mc)
+	}
+}
+
+// generateCommonMethods adds methods to the message that are not on a per field basis.
+func (g *Generator) generateCommonMethods(mc *msgCtx) {
+	// Reset, String and ProtoMessage methods.
+	g.P("func (m *", mc.goName, ") Reset() { *m = ", mc.goName, "{} }")
+	g.P("func (m *", mc.goName, ") String() string { return ", g.Pkg["proto"], ".CompactTextString(m) }")
+	g.P("func (*", mc.goName, ") ProtoMessage() {}")
+	var indexes []string
+	for m := mc.message; m != nil; m = m.parent {
+		indexes = append([]string{strconv.Itoa(m.index)}, indexes...)
+	}
+	g.P("func (*", mc.goName, ") Descriptor() ([]byte, []int) {")
+	g.P("return ", g.file.VarName(), ", []int{", strings.Join(indexes, ", "), "}")
+	g.P("}")
+	g.P()
+	// TODO: Revisit the decision to use a XXX_WellKnownType method
+	// if we change proto.MessageName to work with multiple equivalents.
+	if mc.message.file.GetPackage() == "google.protobuf" && wellKnownTypes[mc.message.GetName()] {
+		g.P("func (*", mc.goName, `) XXX_WellKnownType() string { return "`, mc.message.GetName(), `" }`)
+		g.P()
+	}
+
+	// Extension support methods
+	if len(mc.message.ExtensionRange) > 0 {
+		g.P()
+		g.P("var extRange_", mc.goName, " = []", g.Pkg["proto"], ".ExtensionRange{")
+		for _, r := range mc.message.ExtensionRange {
+			end := fmt.Sprint(*r.End - 1) // make range inclusive on both ends
+			g.P("{Start: ", r.Start, ", End: ", end, "},")
+		}
+		g.P("}")
+		g.P("func (*", mc.goName, ") ExtensionRangeArray() []", g.Pkg["proto"], ".ExtensionRange {")
+		g.P("return extRange_", mc.goName)
+		g.P("}")
+		g.P()
+	}
+
+	// TODO: It does not scale to keep adding another method for every
+	// operation on protos that we want to switch over to using the
+	// table-driven approach. Instead, we should only add a single method
+	// that allows getting access to the *InternalMessageInfo struct and then
+	// calling Unmarshal, Marshal, Merge, Size, and Discard directly on that.
+
+	// Wrapper for table-driven marshaling and unmarshaling.
+	g.P("func (m *", mc.goName, ") XXX_Unmarshal(b []byte) error {")
+	g.P("return xxx_messageInfo_", mc.goName, ".Unmarshal(m, b)")
+	g.P("}")
+
+	g.P("func (m *", mc.goName, ") XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {")
+	g.P("return xxx_messageInfo_", mc.goName, ".Marshal(b, m, deterministic)")
+	g.P("}")
+
+	g.P("func (m *", mc.goName, ") XXX_Merge(src ", g.Pkg["proto"], ".Message) {")
+	g.P("xxx_messageInfo_", mc.goName, ".Merge(m, src)")
+	g.P("}")
+
+	g.P("func (m *", mc.goName, ") XXX_Size() int {") // avoid name clash with "Size" field in some message
+	g.P("return xxx_messageInfo_", mc.goName, ".Size(m)")
+	g.P("}")
+
+	g.P("func (m *", mc.goName, ") XXX_DiscardUnknown() {")
+	g.P("xxx_messageInfo_", mc.goName, ".DiscardUnknown(m)")
+	g.P("}")
+
+	g.P("var xxx_messageInfo_", mc.goName, " ", g.Pkg["proto"], ".InternalMessageInfo")
+	g.P()
+}
+
+// Generate the type, methods and default constant definitions for this Descriptor.
+func (g *Generator) generateMessage(message *Descriptor) {
+	topLevelFields := []topLevelField{}
+	oFields := make(map[int32]*oneofField)
+	// The full type name
+	typeName := message.TypeName()
+	// The full type name, CamelCased.
+	goTypeName := CamelCaseSlice(typeName)
+
+	usedNames := make(map[string]bool)
+	for _, n := range methodNames {
+		usedNames[n] = true
+	}
+
+	// allocNames finds a conflict-free variation of the given strings,
+	// consistently mutating their suffixes.
+	// It returns the same number of strings.
+	allocNames := func(ns ...string) []string {
+	Loop:
+		for {
+			for _, n := range ns {
+				if usedNames[n] {
+					for i := range ns {
+						ns[i] += "_"
+					}
+					continue Loop
+				}
+			}
+			for _, n := range ns {
+				usedNames[n] = true
+			}
+			return ns
+		}
+	}
+
+	mapFieldTypes := make(map[*descriptor.FieldDescriptorProto]string) // keep track of the map fields to be added later
+
+	// Build a structure more suitable for generating the text in one pass
+	for i, field := range message.Field {
+		// Allocate the getter and the field at the same time so name
+		// collisions create field/method consistent names.
+		// TODO: This allocation occurs based on the order of the fields
+		// in the proto file, meaning that a change in the field
+		// ordering can change generated Method/Field names.
+		base := CamelCase(*field.Name)
+		ns := allocNames(base, "Get"+base)
+		fieldName, fieldGetterName := ns[0], ns[1]
+		typename, wiretype := g.GoType(message, field)
+		jsonName := *field.Name
+		tag := fmt.Sprintf("protobuf:%s json:%q", g.goTag(message, field, wiretype), jsonName+",omitempty")
+
+		oneof := field.OneofIndex != nil
+		if oneof && oFields[*field.OneofIndex] == nil {
+			odp := message.OneofDecl[int(*field.OneofIndex)]
+			base := CamelCase(odp.GetName())
+			fname := allocNames(base)[0]
+
+			// This is the first field of a oneof we haven't seen before.
+			// Generate the union field.
+			oneofFullPath := fmt.Sprintf("%s,%d,%d", message.path, messageOneofPath, *field.OneofIndex)
+			c, ok := g.makeComments(oneofFullPath)
+			if ok {
+				c += "\n//\n"
+			}
+			c += "// Types that are valid to be assigned to " + fname + ":\n"
+			// Generate the rest of this comment later,
+			// when we've computed any disambiguation.
+
+			dname := "is" + goTypeName + "_" + fname
+			tag := `protobuf_oneof:"` + odp.GetName() + `"`
+			of := oneofField{
+				fieldCommon: fieldCommon{
+					goName:     fname,
+					getterName: "Get"+fname,
+					goType:     dname,
+					tags:       tag,
+					protoName:  odp.GetName(),
+					fullPath:   oneofFullPath,
+				},
+				comment: c,
+			}
+			topLevelFields = append(topLevelFields, &of)
+			oFields[*field.OneofIndex] = &of
+		}
+
+		if *field.Type == descriptor.FieldDescriptorProto_TYPE_MESSAGE {
+			desc := g.ObjectNamed(field.GetTypeName())
+			if d, ok := desc.(*Descriptor); ok && d.GetOptions().GetMapEntry() {
+				// Figure out the Go types and tags for the key and value types.
+				keyField, valField := d.Field[0], d.Field[1]
+				keyType, keyWire := g.GoType(d, keyField)
+				valType, valWire := g.GoType(d, valField)
+				keyTag, valTag := g.goTag(d, keyField, keyWire), g.goTag(d, valField, valWire)
+
+				// We don't use stars, except for message-typed values.
+				// Message and enum types are the only two possibly foreign types used in maps,
+				// so record their use. They are not permitted as map keys.
+				keyType = strings.TrimPrefix(keyType, "*")
+				switch *valField.Type {
+				case descriptor.FieldDescriptorProto_TYPE_ENUM:
+					valType = strings.TrimPrefix(valType, "*")
+					g.RecordTypeUse(valField.GetTypeName())
+				case descriptor.FieldDescriptorProto_TYPE_MESSAGE:
+					g.RecordTypeUse(valField.GetTypeName())
+				default:
+					valType = strings.TrimPrefix(valType, "*")
+				}
+
+				typename = fmt.Sprintf("map[%s]%s", keyType, valType)
+				mapFieldTypes[field] = typename // record for the getter generation
+
+				tag += fmt.Sprintf(" protobuf_key:%s protobuf_val:%s", keyTag, valTag)
+			}
+		}
+
+		fieldDeprecated := ""
+		if field.GetOptions().GetDeprecated() {
+			fieldDeprecated = deprecationComment
+		}
+
+		dvalue := g.getterDefault(field, goTypeName)
+		if oneof {
+			tname := goTypeName + "_" + fieldName
+			// It is possible for this to collide with a message or enum
+			// nested in this message. Check for collisions.
+			for {
+				ok := true
+				for _, desc := range message.nested {
+					if CamelCaseSlice(desc.TypeName()) == tname {
+						ok = false
+						break
+					}
+				}
+				for _, enum := range message.enums {
+					if CamelCaseSlice(enum.TypeName()) == tname {
+						ok = false
+						break
+					}
+				}
+				if !ok {
+					tname += "_"
+					continue
+				}
+				break
+			}
+
+			oneofField := oFields[*field.OneofIndex]
+			tag := "protobuf:" + g.goTag(message, field, wiretype)
+			sf := oneofSubField{
+				fieldCommon: fieldCommon{
+					goName:     fieldName,
+					getterName: fieldGetterName,
+					goType:     typename,
+					tags:       tag,
+					protoName:  field.GetName(),
+					fullPath:   fmt.Sprintf("%s,%d,%d", message.path, messageFieldPath, i),
+				},
+				protoTypeName: field.GetTypeName(),
+				fieldNumber:   int(*field.Number),
+				protoType:     *field.Type,
+				getterDef:     dvalue,
+				protoDef:      field.GetDefaultValue(),
+				oneofTypeName: tname,
+				deprecated:    fieldDeprecated,
+			}
+			oneofField.subFields = append(oneofField.subFields, &sf)
+			g.RecordTypeUse(field.GetTypeName())
+			continue
+		}
+
+		fieldFullPath := fmt.Sprintf("%s,%d,%d", message.path, messageFieldPath, i)
+		c, ok := g.makeComments(fieldFullPath)
+		if ok {
+			c += "\n"
+		}
+		rf := simpleField{
+			fieldCommon: fieldCommon{
+				goName:     fieldName,
+				getterName: fieldGetterName,
+				goType:     typename,
+				tags:       tag,
+				protoName:  field.GetName(),
+				fullPath:   fieldFullPath,
+			},
+			protoTypeName: field.GetTypeName(),
+			protoType:     *field.Type,
+			deprecated:    fieldDeprecated,
+			getterDef:     dvalue,
+			protoDef:      field.GetDefaultValue(),
+			comment:       c,
+		}
+		var pf topLevelField = &rf
+
+		topLevelFields = append(topLevelFields, pf)
+		g.RecordTypeUse(field.GetTypeName())
+	}
+
+	mc := &msgCtx{
+		goName:  goTypeName,
+		message: message,
+	}
+
+	g.generateMessageStruct(mc, topLevelFields)
+	g.P()
+	g.generateCommonMethods(mc)
+	g.P()
+	g.generateDefaultConstants(mc, topLevelFields)
+	g.P()
+	g.generateGetters(mc, topLevelFields)
+	g.P()
+	g.generateSetters(mc, topLevelFields)
+	g.P()
+	g.generateOneofFuncs(mc, topLevelFields)
+	g.P()
+
+	var oneofTypes []string
+	for _, f := range topLevelFields {
+		if of, ok := f.(*oneofField); ok {
+			for _, osf := range of.subFields {
+				oneofTypes = append(oneofTypes, osf.oneofTypeName)
+			}
+		}
+	}
+
+	opts := message.Options
+	ms := &messageSymbol{
+		sym:           goTypeName,
+		hasExtensions: len(message.ExtensionRange) > 0,
+		isMessageSet:  opts != nil && opts.GetMessageSetWireFormat(),
+		oneofTypes:    oneofTypes,
+	}
+	g.file.addExport(message, ms)
+
+	for _, ext := range message.ext {
+		g.generateExtension(ext)
+	}
+
+	fullName := strings.Join(message.TypeName(), ".")
+	if g.file.Package != nil {
+		fullName = *g.file.Package + "." + fullName
+	}
+
+	g.addInitf("%s.RegisterType((*%s)(nil), %q)", g.Pkg["proto"], goTypeName, fullName)
+	// Register types for native map types.
+	for _, k := range mapFieldKeys(mapFieldTypes) {
+		fullName := strings.TrimPrefix(*k.TypeName, ".")
+		g.addInitf("%s.RegisterMapType((%s)(nil), %q)", g.Pkg["proto"], mapFieldTypes[k], fullName)
+	}
+
+}
+
+type byTypeName []*descriptor.FieldDescriptorProto
+
+func (a byTypeName) Len() int           { return len(a) }
+func (a byTypeName) Swap(i, j int)      { a[i], a[j] = a[j], a[i] }
+func (a byTypeName) Less(i, j int) bool { return *a[i].TypeName < *a[j].TypeName }
+
+// mapFieldKeys returns the keys of m in a consistent order.
+func mapFieldKeys(m map[*descriptor.FieldDescriptorProto]string) []*descriptor.FieldDescriptorProto {
+	keys := make([]*descriptor.FieldDescriptorProto, 0, len(m))
+	for k := range m {
+		keys = append(keys, k)
+	}
+	sort.Sort(byTypeName(keys))
+	return keys
+}
+
+var escapeChars = [256]byte{
+	'a': '\a', 'b': '\b', 'f': '\f', 'n': '\n', 'r': '\r', 't': '\t', 'v': '\v', '\\': '\\', '"': '"', '\'': '\'', '?': '?',
+}
+
+// unescape reverses the "C" escaping that protoc does for default values of bytes fields.
+// It is best effort in that it effectively ignores malformed input. Seemingly invalid escape
+// sequences are conveyed, unmodified, into the decoded result.
+func unescape(s string) string {
+	// NB: Sadly, we can't use strconv.Unquote because protoc will escape both
+	// single and double quotes, but strconv.Unquote only allows one or the
+	// other (based on actual surrounding quotes of its input argument).
+
+	var out []byte
+	for len(s) > 0 {
+		// regular character, or too short to be valid escape
+		if s[0] != '\\' || len(s) < 2 {
+			out = append(out, s[0])
+			s = s[1:]
+		} else if c := escapeChars[s[1]]; c != 0 {
+			// escape sequence
+			out = append(out, c)
+			s = s[2:]
+		} else if s[1] == 'x' || s[1] == 'X' {
+			// hex escape, e.g. "\x80
+			if len(s) < 4 {
+				// too short to be valid
+				out = append(out, s[:2]...)
+				s = s[2:]
+				continue
+			}
+			v, err := strconv.ParseUint(s[2:4], 16, 8)
+			if err != nil {
+				out = append(out, s[:4]...)
+			} else {
+				out = append(out, byte(v))
+			}
+			s = s[4:]
+		} else if '0' <= s[1] && s[1] <= '7' {
+			// octal escape, can vary from 1 to 3 octal digits; e.g., "\0" "\40" or "\164"
+			// so consume up to 2 more bytes or up to end-of-string
+			n := len(s[1:]) - len(strings.TrimLeft(s[1:], "01234567"))
+			if n > 3 {
+				n = 3
+			}
+			v, err := strconv.ParseUint(s[1:1+n], 8, 8)
+			if err != nil {
+				out = append(out, s[:1+n]...)
+			} else {
+				out = append(out, byte(v))
+			}
+			s = s[1+n:]
+		} else {
+			// bad escape, just propagate the slash as-is
+			out = append(out, s[0])
+			s = s[1:]
+		}
+	}
+
+	return string(out)
+}
+
+func (g *Generator) generateExtension(ext *ExtensionDescriptor) {
+	ccTypeName := ext.DescName()
+
+	extObj := g.ObjectNamed(*ext.Extendee)
+	var extDesc *Descriptor
+	if id, ok := extObj.(*ImportedDescriptor); ok {
+		// This is extending a publicly imported message.
+		// We need the underlying type for goTag.
+		extDesc = id.o.(*Descriptor)
+	} else {
+		extDesc = extObj.(*Descriptor)
+	}
+	extendedType := "*" + g.TypeName(extObj) // always use the original
+	field := ext.FieldDescriptorProto
+	fieldType, wireType := g.GoType(ext.parent, field)
+	tag := g.goTag(extDesc, field, wireType)
+	g.RecordTypeUse(*ext.Extendee)
+	if n := ext.FieldDescriptorProto.TypeName; n != nil {
+		// foreign extension type
+		g.RecordTypeUse(*n)
+	}
+
+	typeName := ext.TypeName()
+
+	// Special case for proto2 message sets: If this extension is extending
+	// proto2.bridge.MessageSet, and its final name component is "message_set_extension",
+	// then drop that last component.
+	//
+	// TODO: This should be implemented in the text formatter rather than the generator.
+	// In addition, the situation for when to apply this special case is implemented
+	// differently in other languages:
+	// https://github.com/google/protobuf/blob/aff10976/src/google/protobuf/text_format.cc#L1560
+	if extDesc.GetOptions().GetMessageSetWireFormat() && typeName[len(typeName)-1] == "message_set_extension" {
+		typeName = typeName[:len(typeName)-1]
+	}
+
+	// For text formatting, the package must be exactly what the .proto file declares,
+	// ignoring overrides such as the go_package option, and with no dot/underscore mapping.
+	extName := strings.Join(typeName, ".")
+	if g.file.Package != nil {
+		extName = *g.file.Package + "." + extName
+	}
+
+	g.P("var ", ccTypeName, " = &", g.Pkg["proto"], ".ExtensionDesc{")
+	g.P("ExtendedType: (", extendedType, ")(nil),")
+	g.P("ExtensionType: (", fieldType, ")(nil),")
+	g.P("Field: ", field.Number, ",")
+	g.P(`Name: "`, extName, `",`)
+	g.P("Tag: ", tag, ",")
+	g.P(`Filename: "`, g.file.GetName(), `",`)
+
+	g.P("}")
+	g.P()
+
+	g.addInitf("%s.RegisterExtension(%s)", g.Pkg["proto"], ext.DescName())
+
+	g.file.addExport(ext, constOrVarSymbol{ccTypeName, "var", ""})
+}
+
+func (g *Generator) generateInitFunction() {
+	if len(g.init) == 0 {
+		return
+	}
+	g.P("func init() {")
+	for _, l := range g.init {
+		g.P(l)
+	}
+	g.P("}")
+	g.init = nil
+}
+
+func (g *Generator) generateFileDescriptor(file *FileDescriptor) {
+	// Make a copy and trim source_code_info data.
+	// TODO: Trim this more when we know exactly what we need.
+	pb := proto.Clone(file.FileDescriptorProto).(*descriptor.FileDescriptorProto)
+	pb.SourceCodeInfo = nil
+
+	b, err := proto.Marshal(pb)
+	if err != nil {
+		g.Fail(err.Error())
+	}
+
+	var buf bytes.Buffer
+	w, _ := gzip.NewWriterLevel(&buf, gzip.BestCompression)
+	w.Write(b)
+	w.Close()
+	b = buf.Bytes()
+
+	v := file.VarName()
+	g.P()
+	g.P("func init() { ", g.Pkg["proto"], ".RegisterFile(", strconv.Quote(*file.Name), ", ", v, ") }")
+	g.P("var ", v, " = []byte{")
+	g.P("// ", len(b), " bytes of a gzipped FileDescriptorProto")
+	for len(b) > 0 {
+		n := 16
+		if n > len(b) {
+			n = len(b)
+		}
+
+		s := ""
+		for _, c := range b[:n] {
+			s += fmt.Sprintf("0x%02x,", c)
+		}
+		g.P(s)
+
+		b = b[n:]
+	}
+	g.P("}")
+}
+
+func (g *Generator) generateEnumRegistration(enum *EnumDescriptor) {
+	// // We always print the full (proto-world) package name here.
+	pkg := enum.File().GetPackage()
+	if pkg != "" {
+		pkg += "."
+	}
+	// The full type name
+	typeName := enum.TypeName()
+	// The full type name, CamelCased.
+	ccTypeName := CamelCaseSlice(typeName)
+	g.addInitf("%s.RegisterEnum(%q, %[3]s_name, %[3]s_value)", g.Pkg["proto"], pkg+ccTypeName, ccTypeName)
+}
+
+// And now lots of helper functions.
+
+// Is c an ASCII lower-case letter?
+func isASCIILower(c byte) bool {
+	return 'a' <= c && c <= 'z'
+}
+
+// Is c an ASCII digit?
+func isASCIIDigit(c byte) bool {
+	return '0' <= c && c <= '9'
+}
+
+// CamelCase returns the CamelCased name.
+// If there is an interior underscore followed by a lower case letter,
+// drop the underscore and convert the letter to upper case.
+// There is a remote possibility of this rewrite causing a name collision,
+// but it's so remote we're prepared to pretend it's nonexistent - since the
+// C++ generator lowercases names, it's extremely unlikely to have two fields
+// with different capitalizations.
+// In short, _my_field_name_2 becomes XMyFieldName_2.
+func CamelCase(s string) string {
+	if s == "" {
+		return ""
+	}
+	t := make([]byte, 0, 32)
+	i := 0
+	if s[0] == '_' {
+		// Need a capital letter; drop the '_'.
+		t = append(t, 'X')
+		i++
+	}
+	// Invariant: if the next letter is lower case, it must be converted
+	// to upper case.
+	// That is, we process a word at a time, where words are marked by _ or
+	// upper case letter. Digits are treated as words.
+	for ; i < len(s); i++ {
+		c := s[i]
+		if c == '_' && i+1 < len(s) && isASCIILower(s[i+1]) {
+			continue // Skip the underscore in s.
+		}
+		if isASCIIDigit(c) {
+			t = append(t, c)
+			continue
+		}
+		// Assume we have a letter now - if not, it's a bogus identifier.
+		// The next word is a sequence of characters that must start upper case.
+		if isASCIILower(c) {
+			c ^= ' ' // Make it a capital letter.
+		}
+		t = append(t, c) // Guaranteed not lower case.
+		// Accept lower case sequence that follows.
+		for i+1 < len(s) && isASCIILower(s[i+1]) {
+			i++
+			t = append(t, s[i])
+		}
+	}
+	return string(t)
+}
+
+// CamelCaseSlice is like CamelCase, but the argument is a slice of strings to
+// be joined with "_".
+func CamelCaseSlice(elem []string) string { return CamelCase(strings.Join(elem, "_")) }
+
+// dottedSlice turns a sliced name into a dotted name.
+func dottedSlice(elem []string) string { return strings.Join(elem, ".") }
+
+// Is this field optional?
+func isOptional(field *descriptor.FieldDescriptorProto) bool {
+	return field.Label != nil && *field.Label == descriptor.FieldDescriptorProto_LABEL_OPTIONAL
+}
+
+// Is this field required?
+func isRequired(field *descriptor.FieldDescriptorProto) bool {
+	return field.Label != nil && *field.Label == descriptor.FieldDescriptorProto_LABEL_REQUIRED
+}
+
+// Is this field repeated?
+func isRepeated(field *descriptor.FieldDescriptorProto) bool {
+	return field.Label != nil && *field.Label == descriptor.FieldDescriptorProto_LABEL_REPEATED
+}
+
+// Is this field a scalar numeric type?
+func isScalar(field *descriptor.FieldDescriptorProto) bool {
+	if field.Type == nil {
+		return false
+	}
+	switch *field.Type {
+	case descriptor.FieldDescriptorProto_TYPE_DOUBLE,
+		descriptor.FieldDescriptorProto_TYPE_FLOAT,
+		descriptor.FieldDescriptorProto_TYPE_INT64,
+		descriptor.FieldDescriptorProto_TYPE_UINT64,
+		descriptor.FieldDescriptorProto_TYPE_INT32,
+		descriptor.FieldDescriptorProto_TYPE_FIXED64,
+		descriptor.FieldDescriptorProto_TYPE_FIXED32,
+		descriptor.FieldDescriptorProto_TYPE_BOOL,
+		descriptor.FieldDescriptorProto_TYPE_UINT32,
+		descriptor.FieldDescriptorProto_TYPE_ENUM,
+		descriptor.FieldDescriptorProto_TYPE_SFIXED32,
+		descriptor.FieldDescriptorProto_TYPE_SFIXED64,
+		descriptor.FieldDescriptorProto_TYPE_SINT32,
+		descriptor.FieldDescriptorProto_TYPE_SINT64:
+		return true
+	default:
+		return false
+	}
+}
+
+// badToUnderscore is the mapping function used to generate Go names from package names,
+// which can be dotted in the input .proto file.  It replaces non-identifier characters such as
+// dot or dash with underscore.
+func badToUnderscore(r rune) rune {
+	if unicode.IsLetter(r) || unicode.IsDigit(r) || r == '_' {
+		return r
+	}
+	return '_'
+}
+
+// baseName returns the last path element of the name, with the last dotted suffix removed.
+func baseName(name string) string {
+	// First, find the last element
+	if i := strings.LastIndex(name, "/"); i >= 0 {
+		name = name[i+1:]
+	}
+	// Now drop the suffix
+	if i := strings.LastIndex(name, "."); i >= 0 {
+		name = name[0:i]
+	}
+	return name
+}
+
+// The SourceCodeInfo message describes the location of elements of a parsed
+// .proto file by way of a "path", which is a sequence of integers that
+// describe the route from a FileDescriptorProto to the relevant submessage.
+// The path alternates between a field number of a repeated field, and an index
+// into that repeated field. The constants below define the field numbers that
+// are used.
+//
+// See descriptor.proto for more information about this.
+const (
+	// tag numbers in FileDescriptorProto
+	packagePath = 2 // package
+	messagePath = 4 // message_type
+	enumPath    = 5 // enum_type
+	// tag numbers in DescriptorProto
+	messageFieldPath   = 2 // field
+	messageMessagePath = 3 // nested_type
+	messageEnumPath    = 4 // enum_type
+	messageOneofPath   = 8 // oneof_decl
+	// tag numbers in EnumDescriptorProto
+	enumValuePath = 2 // value
+)
+
+var supportTypeAliases bool
+
+func init() {
+	for _, tag := range build.Default.ReleaseTags {
+		if tag == "go1.9" {
+			supportTypeAliases = true
+			return
+		}
+	}
+}
diff --git a/vendor/github.com/golang/protobuf/protoc-gen-go/generator/internal/remap/remap.go b/vendor/github.com/golang/protobuf/protoc-gen-go/generator/internal/remap/remap.go
new file mode 100644
index 0000000..a9b6103
--- /dev/null
+++ b/vendor/github.com/golang/protobuf/protoc-gen-go/generator/internal/remap/remap.go
@@ -0,0 +1,117 @@
+// Go support for Protocol Buffers - Google's data interchange format
+//
+// Copyright 2017 The Go Authors.  All rights reserved.
+// https://github.com/golang/protobuf
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+/*
+Package remap handles tracking the locations of Go tokens in a source text
+across a rewrite by the Go formatter.
+*/
+package remap
+
+import (
+	"fmt"
+	"go/scanner"
+	"go/token"
+)
+
+// A Location represents a span of byte offsets in the source text.
+type Location struct {
+	Pos, End int // End is exclusive
+}
+
+// A Map represents a mapping between token locations in an input source text
+// and locations in the correspnding output text.
+type Map map[Location]Location
+
+// Find reports whether the specified span is recorded by m, and if so returns
+// the new location it was mapped to. If the input span was not found, the
+// returned location is the same as the input.
+func (m Map) Find(pos, end int) (Location, bool) {
+	key := Location{
+		Pos: pos,
+		End: end,
+	}
+	if loc, ok := m[key]; ok {
+		return loc, true
+	}
+	return key, false
+}
+
+func (m Map) add(opos, oend, npos, nend int) {
+	m[Location{Pos: opos, End: oend}] = Location{Pos: npos, End: nend}
+}
+
+// Compute constructs a location mapping from input to output.  An error is
+// reported if any of the tokens of output cannot be mapped.
+func Compute(input, output []byte) (Map, error) {
+	itok := tokenize(input)
+	otok := tokenize(output)
+	if len(itok) != len(otok) {
+		return nil, fmt.Errorf("wrong number of tokens, %d ≠ %d", len(itok), len(otok))
+	}
+	m := make(Map)
+	for i, ti := range itok {
+		to := otok[i]
+		if ti.Token != to.Token {
+			return nil, fmt.Errorf("token %d type mismatch: %s ≠ %s", i+1, ti, to)
+		}
+		m.add(ti.pos, ti.end, to.pos, to.end)
+	}
+	return m, nil
+}
+
+// tokinfo records the span and type of a source token.
+type tokinfo struct {
+	pos, end int
+	token.Token
+}
+
+func tokenize(src []byte) []tokinfo {
+	fs := token.NewFileSet()
+	var s scanner.Scanner
+	s.Init(fs.AddFile("src", fs.Base(), len(src)), src, nil, scanner.ScanComments)
+	var info []tokinfo
+	for {
+		pos, next, lit := s.Scan()
+		switch next {
+		case token.SEMICOLON:
+			continue
+		}
+		info = append(info, tokinfo{
+			pos:   int(pos - 1),
+			end:   int(pos + token.Pos(len(lit)) - 1),
+			Token: next,
+		})
+		if next == token.EOF {
+			break
+		}
+	}
+	return info
+}
diff --git a/vendor/github.com/golang/protobuf/protoc-gen-go/plugin/plugin.pb.go b/vendor/github.com/golang/protobuf/protoc-gen-go/plugin/plugin.pb.go
new file mode 100644
index 0000000..61bfc10
--- /dev/null
+++ b/vendor/github.com/golang/protobuf/protoc-gen-go/plugin/plugin.pb.go
@@ -0,0 +1,369 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: google/protobuf/compiler/plugin.proto
+
+/*
+Package plugin_go is a generated protocol buffer package.
+
+It is generated from these files:
+	google/protobuf/compiler/plugin.proto
+
+It has these top-level messages:
+	Version
+	CodeGeneratorRequest
+	CodeGeneratorResponse
+*/
+package plugin_go
+
+import proto "github.com/golang/protobuf/proto"
+import fmt "fmt"
+import math "math"
+import google_protobuf "github.com/golang/protobuf/protoc-gen-go/descriptor"
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
+
+// The version number of protocol compiler.
+type Version struct {
+	Major *int32 `protobuf:"varint,1,opt,name=major" json:"major,omitempty"`
+	Minor *int32 `protobuf:"varint,2,opt,name=minor" json:"minor,omitempty"`
+	Patch *int32 `protobuf:"varint,3,opt,name=patch" json:"patch,omitempty"`
+	// A suffix for alpha, beta or rc release, e.g., "alpha-1", "rc2". It should
+	// be empty for mainline stable releases.
+	Suffix               *string  `protobuf:"bytes,4,opt,name=suffix" json:"suffix,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Version) Reset()                    { *m = Version{} }
+func (m *Version) String() string            { return proto.CompactTextString(m) }
+func (*Version) ProtoMessage()               {}
+func (*Version) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} }
+func (m *Version) Unmarshal(b []byte) error {
+	return xxx_messageInfo_Version.Unmarshal(m, b)
+}
+func (m *Version) Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Version.Marshal(b, m, deterministic)
+}
+func (dst *Version) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Version.Merge(dst, src)
+}
+func (m *Version) XXX_Size() int {
+	return xxx_messageInfo_Version.Size(m)
+}
+func (m *Version) XXX_DiscardUnknown() {
+	xxx_messageInfo_Version.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Version proto.InternalMessageInfo
+
+func (m *Version) GetMajor() int32 {
+	if m != nil && m.Major != nil {
+		return *m.Major
+	}
+	return 0
+}
+
+func (m *Version) GetMinor() int32 {
+	if m != nil && m.Minor != nil {
+		return *m.Minor
+	}
+	return 0
+}
+
+func (m *Version) GetPatch() int32 {
+	if m != nil && m.Patch != nil {
+		return *m.Patch
+	}
+	return 0
+}
+
+func (m *Version) GetSuffix() string {
+	if m != nil && m.Suffix != nil {
+		return *m.Suffix
+	}
+	return ""
+}
+
+// An encoded CodeGeneratorRequest is written to the plugin's stdin.
+type CodeGeneratorRequest struct {
+	// The .proto files that were explicitly listed on the command-line.  The
+	// code generator should generate code only for these files.  Each file's
+	// descriptor will be included in proto_file, below.
+	FileToGenerate []string `protobuf:"bytes,1,rep,name=file_to_generate,json=fileToGenerate" json:"file_to_generate,omitempty"`
+	// The generator parameter passed on the command-line.
+	Parameter *string `protobuf:"bytes,2,opt,name=parameter" json:"parameter,omitempty"`
+	// FileDescriptorProtos for all files in files_to_generate and everything
+	// they import.  The files will appear in topological order, so each file
+	// appears before any file that imports it.
+	//
+	// protoc guarantees that all proto_files will be written after
+	// the fields above, even though this is not technically guaranteed by the
+	// protobuf wire format.  This theoretically could allow a plugin to stream
+	// in the FileDescriptorProtos and handle them one by one rather than read
+	// the entire set into memory at once.  However, as of this writing, this
+	// is not similarly optimized on protoc's end -- it will store all fields in
+	// memory at once before sending them to the plugin.
+	//
+	// Type names of fields and extensions in the FileDescriptorProto are always
+	// fully qualified.
+	ProtoFile []*google_protobuf.FileDescriptorProto `protobuf:"bytes,15,rep,name=proto_file,json=protoFile" json:"proto_file,omitempty"`
+	// The version number of protocol compiler.
+	CompilerVersion      *Version `protobuf:"bytes,3,opt,name=compiler_version,json=compilerVersion" json:"compiler_version,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *CodeGeneratorRequest) Reset()                    { *m = CodeGeneratorRequest{} }
+func (m *CodeGeneratorRequest) String() string            { return proto.CompactTextString(m) }
+func (*CodeGeneratorRequest) ProtoMessage()               {}
+func (*CodeGeneratorRequest) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} }
+func (m *CodeGeneratorRequest) Unmarshal(b []byte) error {
+	return xxx_messageInfo_CodeGeneratorRequest.Unmarshal(m, b)
+}
+func (m *CodeGeneratorRequest) Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_CodeGeneratorRequest.Marshal(b, m, deterministic)
+}
+func (dst *CodeGeneratorRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_CodeGeneratorRequest.Merge(dst, src)
+}
+func (m *CodeGeneratorRequest) XXX_Size() int {
+	return xxx_messageInfo_CodeGeneratorRequest.Size(m)
+}
+func (m *CodeGeneratorRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_CodeGeneratorRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_CodeGeneratorRequest proto.InternalMessageInfo
+
+func (m *CodeGeneratorRequest) GetFileToGenerate() []string {
+	if m != nil {
+		return m.FileToGenerate
+	}
+	return nil
+}
+
+func (m *CodeGeneratorRequest) GetParameter() string {
+	if m != nil && m.Parameter != nil {
+		return *m.Parameter
+	}
+	return ""
+}
+
+func (m *CodeGeneratorRequest) GetProtoFile() []*google_protobuf.FileDescriptorProto {
+	if m != nil {
+		return m.ProtoFile
+	}
+	return nil
+}
+
+func (m *CodeGeneratorRequest) GetCompilerVersion() *Version {
+	if m != nil {
+		return m.CompilerVersion
+	}
+	return nil
+}
+
+// The plugin writes an encoded CodeGeneratorResponse to stdout.
+type CodeGeneratorResponse struct {
+	// Error message.  If non-empty, code generation failed.  The plugin process
+	// should exit with status code zero even if it reports an error in this way.
+	//
+	// This should be used to indicate errors in .proto files which prevent the
+	// code generator from generating correct code.  Errors which indicate a
+	// problem in protoc itself -- such as the input CodeGeneratorRequest being
+	// unparseable -- should be reported by writing a message to stderr and
+	// exiting with a non-zero status code.
+	Error                *string                       `protobuf:"bytes,1,opt,name=error" json:"error,omitempty"`
+	File                 []*CodeGeneratorResponse_File `protobuf:"bytes,15,rep,name=file" json:"file,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                      `json:"-"`
+	XXX_unrecognized     []byte                        `json:"-"`
+	XXX_sizecache        int32                         `json:"-"`
+}
+
+func (m *CodeGeneratorResponse) Reset()                    { *m = CodeGeneratorResponse{} }
+func (m *CodeGeneratorResponse) String() string            { return proto.CompactTextString(m) }
+func (*CodeGeneratorResponse) ProtoMessage()               {}
+func (*CodeGeneratorResponse) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} }
+func (m *CodeGeneratorResponse) Unmarshal(b []byte) error {
+	return xxx_messageInfo_CodeGeneratorResponse.Unmarshal(m, b)
+}
+func (m *CodeGeneratorResponse) Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_CodeGeneratorResponse.Marshal(b, m, deterministic)
+}
+func (dst *CodeGeneratorResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_CodeGeneratorResponse.Merge(dst, src)
+}
+func (m *CodeGeneratorResponse) XXX_Size() int {
+	return xxx_messageInfo_CodeGeneratorResponse.Size(m)
+}
+func (m *CodeGeneratorResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_CodeGeneratorResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_CodeGeneratorResponse proto.InternalMessageInfo
+
+func (m *CodeGeneratorResponse) GetError() string {
+	if m != nil && m.Error != nil {
+		return *m.Error
+	}
+	return ""
+}
+
+func (m *CodeGeneratorResponse) GetFile() []*CodeGeneratorResponse_File {
+	if m != nil {
+		return m.File
+	}
+	return nil
+}
+
+// Represents a single generated file.
+type CodeGeneratorResponse_File struct {
+	// The file name, relative to the output directory.  The name must not
+	// contain "." or ".." components and must be relative, not be absolute (so,
+	// the file cannot lie outside the output directory).  "/" must be used as
+	// the path separator, not "\".
+	//
+	// If the name is omitted, the content will be appended to the previous
+	// file.  This allows the generator to break large files into small chunks,
+	// and allows the generated text to be streamed back to protoc so that large
+	// files need not reside completely in memory at one time.  Note that as of
+	// this writing protoc does not optimize for this -- it will read the entire
+	// CodeGeneratorResponse before writing files to disk.
+	Name *string `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
+	// If non-empty, indicates that the named file should already exist, and the
+	// content here is to be inserted into that file at a defined insertion
+	// point.  This feature allows a code generator to extend the output
+	// produced by another code generator.  The original generator may provide
+	// insertion points by placing special annotations in the file that look
+	// like:
+	//   @@protoc_insertion_point(NAME)
+	// The annotation can have arbitrary text before and after it on the line,
+	// which allows it to be placed in a comment.  NAME should be replaced with
+	// an identifier naming the point -- this is what other generators will use
+	// as the insertion_point.  Code inserted at this point will be placed
+	// immediately above the line containing the insertion point (thus multiple
+	// insertions to the same point will come out in the order they were added).
+	// The double-@ is intended to make it unlikely that the generated code
+	// could contain things that look like insertion points by accident.
+	//
+	// For example, the C++ code generator places the following line in the
+	// .pb.h files that it generates:
+	//   // @@protoc_insertion_point(namespace_scope)
+	// This line appears within the scope of the file's package namespace, but
+	// outside of any particular class.  Another plugin can then specify the
+	// insertion_point "namespace_scope" to generate additional classes or
+	// other declarations that should be placed in this scope.
+	//
+	// Note that if the line containing the insertion point begins with
+	// whitespace, the same whitespace will be added to every line of the
+	// inserted text.  This is useful for languages like Python, where
+	// indentation matters.  In these languages, the insertion point comment
+	// should be indented the same amount as any inserted code will need to be
+	// in order to work correctly in that context.
+	//
+	// The code generator that generates the initial file and the one which
+	// inserts into it must both run as part of a single invocation of protoc.
+	// Code generators are executed in the order in which they appear on the
+	// command line.
+	//
+	// If |insertion_point| is present, |name| must also be present.
+	InsertionPoint *string `protobuf:"bytes,2,opt,name=insertion_point,json=insertionPoint" json:"insertion_point,omitempty"`
+	// The file contents.
+	Content              *string  `protobuf:"bytes,15,opt,name=content" json:"content,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *CodeGeneratorResponse_File) Reset()                    { *m = CodeGeneratorResponse_File{} }
+func (m *CodeGeneratorResponse_File) String() string            { return proto.CompactTextString(m) }
+func (*CodeGeneratorResponse_File) ProtoMessage()               {}
+func (*CodeGeneratorResponse_File) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2, 0} }
+func (m *CodeGeneratorResponse_File) Unmarshal(b []byte) error {
+	return xxx_messageInfo_CodeGeneratorResponse_File.Unmarshal(m, b)
+}
+func (m *CodeGeneratorResponse_File) Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_CodeGeneratorResponse_File.Marshal(b, m, deterministic)
+}
+func (dst *CodeGeneratorResponse_File) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_CodeGeneratorResponse_File.Merge(dst, src)
+}
+func (m *CodeGeneratorResponse_File) XXX_Size() int {
+	return xxx_messageInfo_CodeGeneratorResponse_File.Size(m)
+}
+func (m *CodeGeneratorResponse_File) XXX_DiscardUnknown() {
+	xxx_messageInfo_CodeGeneratorResponse_File.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_CodeGeneratorResponse_File proto.InternalMessageInfo
+
+func (m *CodeGeneratorResponse_File) GetName() string {
+	if m != nil && m.Name != nil {
+		return *m.Name
+	}
+	return ""
+}
+
+func (m *CodeGeneratorResponse_File) GetInsertionPoint() string {
+	if m != nil && m.InsertionPoint != nil {
+		return *m.InsertionPoint
+	}
+	return ""
+}
+
+func (m *CodeGeneratorResponse_File) GetContent() string {
+	if m != nil && m.Content != nil {
+		return *m.Content
+	}
+	return ""
+}
+
+func init() {
+	proto.RegisterType((*Version)(nil), "google.protobuf.compiler.Version")
+	proto.RegisterType((*CodeGeneratorRequest)(nil), "google.protobuf.compiler.CodeGeneratorRequest")
+	proto.RegisterType((*CodeGeneratorResponse)(nil), "google.protobuf.compiler.CodeGeneratorResponse")
+	proto.RegisterType((*CodeGeneratorResponse_File)(nil), "google.protobuf.compiler.CodeGeneratorResponse.File")
+}
+
+func init() { proto.RegisterFile("google/protobuf/compiler/plugin.proto", fileDescriptor0) }
+
+var fileDescriptor0 = []byte{
+	// 417 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x92, 0xcf, 0x6a, 0x14, 0x41,
+	0x10, 0xc6, 0x19, 0x77, 0x63, 0x98, 0x8a, 0x64, 0x43, 0x13, 0xa5, 0x09, 0x39, 0x8c, 0x8b, 0xe2,
+	0x5c, 0x32, 0x0b, 0xc1, 0x8b, 0x78, 0x4b, 0x44, 0x3d, 0x78, 0x58, 0x1a, 0xf1, 0x20, 0xc8, 0x30,
+	0x99, 0xd4, 0x74, 0x5a, 0x66, 0xba, 0xc6, 0xee, 0x1e, 0xf1, 0x49, 0x7d, 0x0f, 0xdf, 0x40, 0xfa,
+	0xcf, 0x24, 0xb2, 0xb8, 0xa7, 0xee, 0xef, 0x57, 0xd5, 0xd5, 0x55, 0x1f, 0x05, 0x2f, 0x25, 0x91,
+	0xec, 0x71, 0x33, 0x1a, 0x72, 0x74, 0x33, 0x75, 0x9b, 0x96, 0x86, 0x51, 0xf5, 0x68, 0x36, 0x63,
+	0x3f, 0x49, 0xa5, 0xab, 0x10, 0x60, 0x3c, 0xa6, 0x55, 0x73, 0x5a, 0x35, 0xa7, 0x9d, 0x15, 0xbb,
+	0x05, 0x6e, 0xd1, 0xb6, 0x46, 0x8d, 0x8e, 0x4c, 0xcc, 0x5e, 0xb7, 0x70, 0xf8, 0x05, 0x8d, 0x55,
+	0xa4, 0xd9, 0x29, 0x1c, 0x0c, 0xcd, 0x77, 0x32, 0x3c, 0x2b, 0xb2, 0xf2, 0x40, 0x44, 0x11, 0xa8,
+	0xd2, 0x64, 0xf8, 0xa3, 0x44, 0xbd, 0xf0, 0x74, 0x6c, 0x5c, 0x7b, 0xc7, 0x17, 0x91, 0x06, 0xc1,
+	0x9e, 0xc1, 0x63, 0x3b, 0x75, 0x9d, 0xfa, 0xc5, 0x97, 0x45, 0x56, 0xe6, 0x22, 0xa9, 0xf5, 0x9f,
+	0x0c, 0x4e, 0xaf, 0xe9, 0x16, 0x3f, 0xa0, 0x46, 0xd3, 0x38, 0x32, 0x02, 0x7f, 0x4c, 0x68, 0x1d,
+	0x2b, 0xe1, 0xa4, 0x53, 0x3d, 0xd6, 0x8e, 0x6a, 0x19, 0x63, 0xc8, 0xb3, 0x62, 0x51, 0xe6, 0xe2,
+	0xd8, 0xf3, 0xcf, 0x94, 0x5e, 0x20, 0x3b, 0x87, 0x7c, 0x6c, 0x4c, 0x33, 0xa0, 0xc3, 0xd8, 0x4a,
+	0x2e, 0x1e, 0x00, 0xbb, 0x06, 0x08, 0xe3, 0xd4, 0xfe, 0x15, 0x5f, 0x15, 0x8b, 0xf2, 0xe8, 0xf2,
+	0x45, 0xb5, 0x6b, 0xcb, 0x7b, 0xd5, 0xe3, 0xbb, 0x7b, 0x03, 0xb6, 0x1e, 0x8b, 0x3c, 0x44, 0x7d,
+	0x84, 0x7d, 0x82, 0x93, 0xd9, 0xb8, 0xfa, 0x67, 0xf4, 0x24, 0x8c, 0x77, 0x74, 0xf9, 0xbc, 0xda,
+	0xe7, 0x70, 0x95, 0xcc, 0x13, 0xab, 0x99, 0x24, 0xb0, 0xfe, 0x9d, 0xc1, 0xd3, 0x9d, 0x99, 0xed,
+	0x48, 0xda, 0xa2, 0xf7, 0x0e, 0x8d, 0x49, 0x3e, 0xe7, 0x22, 0x0a, 0xf6, 0x11, 0x96, 0xff, 0x34,
+	0xff, 0x7a, 0xff, 0x8f, 0xff, 0x2d, 0x1a, 0x66, 0x13, 0xa1, 0xc2, 0xd9, 0x37, 0x58, 0x86, 0x79,
+	0x18, 0x2c, 0x75, 0x33, 0x60, 0xfa, 0x26, 0xdc, 0xd9, 0x2b, 0x58, 0x29, 0x6d, 0xd1, 0x38, 0x45,
+	0xba, 0x1e, 0x49, 0x69, 0x97, 0xcc, 0x3c, 0xbe, 0xc7, 0x5b, 0x4f, 0x19, 0x87, 0xc3, 0x96, 0xb4,
+	0x43, 0xed, 0xf8, 0x2a, 0x24, 0xcc, 0xf2, 0x4a, 0xc2, 0x79, 0x4b, 0xc3, 0xde, 0xfe, 0xae, 0x9e,
+	0x6c, 0xc3, 0x6e, 0x06, 0x7b, 0xed, 0xd7, 0x37, 0x52, 0xb9, 0xbb, 0xe9, 0xc6, 0x87, 0x37, 0x92,
+	0xfa, 0x46, 0xcb, 0x87, 0x65, 0x0c, 0x97, 0xf6, 0x42, 0xa2, 0xbe, 0x90, 0x94, 0x56, 0xfa, 0x6d,
+	0x3c, 0x6a, 0x49, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0xf7, 0x15, 0x40, 0xc5, 0xfe, 0x02, 0x00,
+	0x00,
+}
diff --git a/vendor/github.com/golang/protobuf/protoc-gen-go/plugin/plugin.pb.golden b/vendor/github.com/golang/protobuf/protoc-gen-go/plugin/plugin.pb.golden
new file mode 100644
index 0000000..8953d0f
--- /dev/null
+++ b/vendor/github.com/golang/protobuf/protoc-gen-go/plugin/plugin.pb.golden
@@ -0,0 +1,83 @@
+// Code generated by protoc-gen-go.
+// source: google/protobuf/compiler/plugin.proto
+// DO NOT EDIT!
+
+package google_protobuf_compiler
+
+import proto "github.com/golang/protobuf/proto"
+import "math"
+import google_protobuf "github.com/golang/protobuf/protoc-gen-go/descriptor"
+
+// Reference proto and math imports to suppress error if they are not otherwise used.
+var _ = proto.GetString
+var _ = math.Inf
+
+type CodeGeneratorRequest struct {
+	FileToGenerate   []string                               `protobuf:"bytes,1,rep,name=file_to_generate" json:"file_to_generate,omitempty"`
+	Parameter        *string                                `protobuf:"bytes,2,opt,name=parameter" json:"parameter,omitempty"`
+	ProtoFile        []*google_protobuf.FileDescriptorProto `protobuf:"bytes,15,rep,name=proto_file" json:"proto_file,omitempty"`
+	XXX_unrecognized []byte                                 `json:"-"`
+}
+
+func (this *CodeGeneratorRequest) Reset()         { *this = CodeGeneratorRequest{} }
+func (this *CodeGeneratorRequest) String() string { return proto.CompactTextString(this) }
+func (*CodeGeneratorRequest) ProtoMessage()       {}
+
+func (this *CodeGeneratorRequest) GetParameter() string {
+	if this != nil && this.Parameter != nil {
+		return *this.Parameter
+	}
+	return ""
+}
+
+type CodeGeneratorResponse struct {
+	Error            *string                       `protobuf:"bytes,1,opt,name=error" json:"error,omitempty"`
+	File             []*CodeGeneratorResponse_File `protobuf:"bytes,15,rep,name=file" json:"file,omitempty"`
+	XXX_unrecognized []byte                        `json:"-"`
+}
+
+func (this *CodeGeneratorResponse) Reset()         { *this = CodeGeneratorResponse{} }
+func (this *CodeGeneratorResponse) String() string { return proto.CompactTextString(this) }
+func (*CodeGeneratorResponse) ProtoMessage()       {}
+
+func (this *CodeGeneratorResponse) GetError() string {
+	if this != nil && this.Error != nil {
+		return *this.Error
+	}
+	return ""
+}
+
+type CodeGeneratorResponse_File struct {
+	Name             *string `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
+	InsertionPoint   *string `protobuf:"bytes,2,opt,name=insertion_point" json:"insertion_point,omitempty"`
+	Content          *string `protobuf:"bytes,15,opt,name=content" json:"content,omitempty"`
+	XXX_unrecognized []byte  `json:"-"`
+}
+
+func (this *CodeGeneratorResponse_File) Reset()         { *this = CodeGeneratorResponse_File{} }
+func (this *CodeGeneratorResponse_File) String() string { return proto.CompactTextString(this) }
+func (*CodeGeneratorResponse_File) ProtoMessage()       {}
+
+func (this *CodeGeneratorResponse_File) GetName() string {
+	if this != nil && this.Name != nil {
+		return *this.Name
+	}
+	return ""
+}
+
+func (this *CodeGeneratorResponse_File) GetInsertionPoint() string {
+	if this != nil && this.InsertionPoint != nil {
+		return *this.InsertionPoint
+	}
+	return ""
+}
+
+func (this *CodeGeneratorResponse_File) GetContent() string {
+	if this != nil && this.Content != nil {
+		return *this.Content
+	}
+	return ""
+}
+
+func init() {
+}
diff --git a/vendor/github.com/golang/protobuf/protoc-gen-go/plugin/plugin.proto b/vendor/github.com/golang/protobuf/protoc-gen-go/plugin/plugin.proto
new file mode 100644
index 0000000..5b55745
--- /dev/null
+++ b/vendor/github.com/golang/protobuf/protoc-gen-go/plugin/plugin.proto
@@ -0,0 +1,167 @@
+// Protocol Buffers - Google's data interchange format
+// Copyright 2008 Google Inc.  All rights reserved.
+// https://developers.google.com/protocol-buffers/
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+// Author: kenton@google.com (Kenton Varda)
+//
+// WARNING:  The plugin interface is currently EXPERIMENTAL and is subject to
+//   change.
+//
+// protoc (aka the Protocol Compiler) can be extended via plugins.  A plugin is
+// just a program that reads a CodeGeneratorRequest from stdin and writes a
+// CodeGeneratorResponse to stdout.
+//
+// Plugins written using C++ can use google/protobuf/compiler/plugin.h instead
+// of dealing with the raw protocol defined here.
+//
+// A plugin executable needs only to be placed somewhere in the path.  The
+// plugin should be named "protoc-gen-$NAME", and will then be used when the
+// flag "--${NAME}_out" is passed to protoc.
+
+syntax = "proto2";
+package google.protobuf.compiler;
+option java_package = "com.google.protobuf.compiler";
+option java_outer_classname = "PluginProtos";
+
+option go_package = "github.com/golang/protobuf/protoc-gen-go/plugin;plugin_go";
+
+import "google/protobuf/descriptor.proto";
+
+// The version number of protocol compiler.
+message Version {
+  optional int32 major = 1;
+  optional int32 minor = 2;
+  optional int32 patch = 3;
+  // A suffix for alpha, beta or rc release, e.g., "alpha-1", "rc2". It should
+  // be empty for mainline stable releases.
+  optional string suffix = 4;
+}
+
+// An encoded CodeGeneratorRequest is written to the plugin's stdin.
+message CodeGeneratorRequest {
+  // The .proto files that were explicitly listed on the command-line.  The
+  // code generator should generate code only for these files.  Each file's
+  // descriptor will be included in proto_file, below.
+  repeated string file_to_generate = 1;
+
+  // The generator parameter passed on the command-line.
+  optional string parameter = 2;
+
+  // FileDescriptorProtos for all files in files_to_generate and everything
+  // they import.  The files will appear in topological order, so each file
+  // appears before any file that imports it.
+  //
+  // protoc guarantees that all proto_files will be written after
+  // the fields above, even though this is not technically guaranteed by the
+  // protobuf wire format.  This theoretically could allow a plugin to stream
+  // in the FileDescriptorProtos and handle them one by one rather than read
+  // the entire set into memory at once.  However, as of this writing, this
+  // is not similarly optimized on protoc's end -- it will store all fields in
+  // memory at once before sending them to the plugin.
+  //
+  // Type names of fields and extensions in the FileDescriptorProto are always
+  // fully qualified.
+  repeated FileDescriptorProto proto_file = 15;
+
+  // The version number of protocol compiler.
+  optional Version compiler_version = 3;
+
+}
+
+// The plugin writes an encoded CodeGeneratorResponse to stdout.
+message CodeGeneratorResponse {
+  // Error message.  If non-empty, code generation failed.  The plugin process
+  // should exit with status code zero even if it reports an error in this way.
+  //
+  // This should be used to indicate errors in .proto files which prevent the
+  // code generator from generating correct code.  Errors which indicate a
+  // problem in protoc itself -- such as the input CodeGeneratorRequest being
+  // unparseable -- should be reported by writing a message to stderr and
+  // exiting with a non-zero status code.
+  optional string error = 1;
+
+  // Represents a single generated file.
+  message File {
+    // The file name, relative to the output directory.  The name must not
+    // contain "." or ".." components and must be relative, not be absolute (so,
+    // the file cannot lie outside the output directory).  "/" must be used as
+    // the path separator, not "\".
+    //
+    // If the name is omitted, the content will be appended to the previous
+    // file.  This allows the generator to break large files into small chunks,
+    // and allows the generated text to be streamed back to protoc so that large
+    // files need not reside completely in memory at one time.  Note that as of
+    // this writing protoc does not optimize for this -- it will read the entire
+    // CodeGeneratorResponse before writing files to disk.
+    optional string name = 1;
+
+    // If non-empty, indicates that the named file should already exist, and the
+    // content here is to be inserted into that file at a defined insertion
+    // point.  This feature allows a code generator to extend the output
+    // produced by another code generator.  The original generator may provide
+    // insertion points by placing special annotations in the file that look
+    // like:
+    //   @@protoc_insertion_point(NAME)
+    // The annotation can have arbitrary text before and after it on the line,
+    // which allows it to be placed in a comment.  NAME should be replaced with
+    // an identifier naming the point -- this is what other generators will use
+    // as the insertion_point.  Code inserted at this point will be placed
+    // immediately above the line containing the insertion point (thus multiple
+    // insertions to the same point will come out in the order they were added).
+    // The double-@ is intended to make it unlikely that the generated code
+    // could contain things that look like insertion points by accident.
+    //
+    // For example, the C++ code generator places the following line in the
+    // .pb.h files that it generates:
+    //   // @@protoc_insertion_point(namespace_scope)
+    // This line appears within the scope of the file's package namespace, but
+    // outside of any particular class.  Another plugin can then specify the
+    // insertion_point "namespace_scope" to generate additional classes or
+    // other declarations that should be placed in this scope.
+    //
+    // Note that if the line containing the insertion point begins with
+    // whitespace, the same whitespace will be added to every line of the
+    // inserted text.  This is useful for languages like Python, where
+    // indentation matters.  In these languages, the insertion point comment
+    // should be indented the same amount as any inserted code will need to be
+    // in order to work correctly in that context.
+    //
+    // The code generator that generates the initial file and the one which
+    // inserts into it must both run as part of a single invocation of protoc.
+    // Code generators are executed in the order in which they appear on the
+    // command line.
+    //
+    // If |insertion_point| is present, |name| must also be present.
+    optional string insertion_point = 2;
+
+    // The file contents.
+    optional string content = 15;
+  }
+  repeated File file = 15;
+}
diff --git a/vendor/github.com/golang/protobuf/ptypes/struct/struct.pb.go b/vendor/github.com/golang/protobuf/ptypes/struct/struct.pb.go
new file mode 100644
index 0000000..33daa73
--- /dev/null
+++ b/vendor/github.com/golang/protobuf/ptypes/struct/struct.pb.go
@@ -0,0 +1,336 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: google/protobuf/struct.proto
+
+package structpb
+
+import (
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	math "math"
+)
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
+
+// `NullValue` is a singleton enumeration to represent the null value for the
+// `Value` type union.
+//
+//  The JSON representation for `NullValue` is JSON `null`.
+type NullValue int32
+
+const (
+	// Null value.
+	NullValue_NULL_VALUE NullValue = 0
+)
+
+var NullValue_name = map[int32]string{
+	0: "NULL_VALUE",
+}
+
+var NullValue_value = map[string]int32{
+	"NULL_VALUE": 0,
+}
+
+func (x NullValue) String() string {
+	return proto.EnumName(NullValue_name, int32(x))
+}
+
+func (NullValue) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_df322afd6c9fb402, []int{0}
+}
+
+func (NullValue) XXX_WellKnownType() string { return "NullValue" }
+
+// `Struct` represents a structured data value, consisting of fields
+// which map to dynamically typed values. In some languages, `Struct`
+// might be supported by a native representation. For example, in
+// scripting languages like JS a struct is represented as an
+// object. The details of that representation are described together
+// with the proto support for the language.
+//
+// The JSON representation for `Struct` is JSON object.
+type Struct struct {
+	// Unordered map of dynamically typed values.
+	Fields               map[string]*Value `protobuf:"bytes,1,rep,name=fields,proto3" json:"fields,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
+	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
+	XXX_unrecognized     []byte            `json:"-"`
+	XXX_sizecache        int32             `json:"-"`
+}
+
+func (m *Struct) Reset()         { *m = Struct{} }
+func (m *Struct) String() string { return proto.CompactTextString(m) }
+func (*Struct) ProtoMessage()    {}
+func (*Struct) Descriptor() ([]byte, []int) {
+	return fileDescriptor_df322afd6c9fb402, []int{0}
+}
+
+func (*Struct) XXX_WellKnownType() string { return "Struct" }
+
+func (m *Struct) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Struct.Unmarshal(m, b)
+}
+func (m *Struct) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Struct.Marshal(b, m, deterministic)
+}
+func (m *Struct) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Struct.Merge(m, src)
+}
+func (m *Struct) XXX_Size() int {
+	return xxx_messageInfo_Struct.Size(m)
+}
+func (m *Struct) XXX_DiscardUnknown() {
+	xxx_messageInfo_Struct.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Struct proto.InternalMessageInfo
+
+func (m *Struct) GetFields() map[string]*Value {
+	if m != nil {
+		return m.Fields
+	}
+	return nil
+}
+
+// `Value` represents a dynamically typed value which can be either
+// null, a number, a string, a boolean, a recursive struct value, or a
+// list of values. A producer of value is expected to set one of that
+// variants, absence of any variant indicates an error.
+//
+// The JSON representation for `Value` is JSON value.
+type Value struct {
+	// The kind of value.
+	//
+	// Types that are valid to be assigned to Kind:
+	//	*Value_NullValue
+	//	*Value_NumberValue
+	//	*Value_StringValue
+	//	*Value_BoolValue
+	//	*Value_StructValue
+	//	*Value_ListValue
+	Kind                 isValue_Kind `protobuf_oneof:"kind"`
+	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
+	XXX_unrecognized     []byte       `json:"-"`
+	XXX_sizecache        int32        `json:"-"`
+}
+
+func (m *Value) Reset()         { *m = Value{} }
+func (m *Value) String() string { return proto.CompactTextString(m) }
+func (*Value) ProtoMessage()    {}
+func (*Value) Descriptor() ([]byte, []int) {
+	return fileDescriptor_df322afd6c9fb402, []int{1}
+}
+
+func (*Value) XXX_WellKnownType() string { return "Value" }
+
+func (m *Value) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Value.Unmarshal(m, b)
+}
+func (m *Value) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Value.Marshal(b, m, deterministic)
+}
+func (m *Value) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Value.Merge(m, src)
+}
+func (m *Value) XXX_Size() int {
+	return xxx_messageInfo_Value.Size(m)
+}
+func (m *Value) XXX_DiscardUnknown() {
+	xxx_messageInfo_Value.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Value proto.InternalMessageInfo
+
+type isValue_Kind interface {
+	isValue_Kind()
+}
+
+type Value_NullValue struct {
+	NullValue NullValue `protobuf:"varint,1,opt,name=null_value,json=nullValue,proto3,enum=google.protobuf.NullValue,oneof"`
+}
+
+type Value_NumberValue struct {
+	NumberValue float64 `protobuf:"fixed64,2,opt,name=number_value,json=numberValue,proto3,oneof"`
+}
+
+type Value_StringValue struct {
+	StringValue string `protobuf:"bytes,3,opt,name=string_value,json=stringValue,proto3,oneof"`
+}
+
+type Value_BoolValue struct {
+	BoolValue bool `protobuf:"varint,4,opt,name=bool_value,json=boolValue,proto3,oneof"`
+}
+
+type Value_StructValue struct {
+	StructValue *Struct `protobuf:"bytes,5,opt,name=struct_value,json=structValue,proto3,oneof"`
+}
+
+type Value_ListValue struct {
+	ListValue *ListValue `protobuf:"bytes,6,opt,name=list_value,json=listValue,proto3,oneof"`
+}
+
+func (*Value_NullValue) isValue_Kind() {}
+
+func (*Value_NumberValue) isValue_Kind() {}
+
+func (*Value_StringValue) isValue_Kind() {}
+
+func (*Value_BoolValue) isValue_Kind() {}
+
+func (*Value_StructValue) isValue_Kind() {}
+
+func (*Value_ListValue) isValue_Kind() {}
+
+func (m *Value) GetKind() isValue_Kind {
+	if m != nil {
+		return m.Kind
+	}
+	return nil
+}
+
+func (m *Value) GetNullValue() NullValue {
+	if x, ok := m.GetKind().(*Value_NullValue); ok {
+		return x.NullValue
+	}
+	return NullValue_NULL_VALUE
+}
+
+func (m *Value) GetNumberValue() float64 {
+	if x, ok := m.GetKind().(*Value_NumberValue); ok {
+		return x.NumberValue
+	}
+	return 0
+}
+
+func (m *Value) GetStringValue() string {
+	if x, ok := m.GetKind().(*Value_StringValue); ok {
+		return x.StringValue
+	}
+	return ""
+}
+
+func (m *Value) GetBoolValue() bool {
+	if x, ok := m.GetKind().(*Value_BoolValue); ok {
+		return x.BoolValue
+	}
+	return false
+}
+
+func (m *Value) GetStructValue() *Struct {
+	if x, ok := m.GetKind().(*Value_StructValue); ok {
+		return x.StructValue
+	}
+	return nil
+}
+
+func (m *Value) GetListValue() *ListValue {
+	if x, ok := m.GetKind().(*Value_ListValue); ok {
+		return x.ListValue
+	}
+	return nil
+}
+
+// XXX_OneofWrappers is for the internal use of the proto package.
+func (*Value) XXX_OneofWrappers() []interface{} {
+	return []interface{}{
+		(*Value_NullValue)(nil),
+		(*Value_NumberValue)(nil),
+		(*Value_StringValue)(nil),
+		(*Value_BoolValue)(nil),
+		(*Value_StructValue)(nil),
+		(*Value_ListValue)(nil),
+	}
+}
+
+// `ListValue` is a wrapper around a repeated field of values.
+//
+// The JSON representation for `ListValue` is JSON array.
+type ListValue struct {
+	// Repeated field of dynamically typed values.
+	Values               []*Value `protobuf:"bytes,1,rep,name=values,proto3" json:"values,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ListValue) Reset()         { *m = ListValue{} }
+func (m *ListValue) String() string { return proto.CompactTextString(m) }
+func (*ListValue) ProtoMessage()    {}
+func (*ListValue) Descriptor() ([]byte, []int) {
+	return fileDescriptor_df322afd6c9fb402, []int{2}
+}
+
+func (*ListValue) XXX_WellKnownType() string { return "ListValue" }
+
+func (m *ListValue) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ListValue.Unmarshal(m, b)
+}
+func (m *ListValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ListValue.Marshal(b, m, deterministic)
+}
+func (m *ListValue) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ListValue.Merge(m, src)
+}
+func (m *ListValue) XXX_Size() int {
+	return xxx_messageInfo_ListValue.Size(m)
+}
+func (m *ListValue) XXX_DiscardUnknown() {
+	xxx_messageInfo_ListValue.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ListValue proto.InternalMessageInfo
+
+func (m *ListValue) GetValues() []*Value {
+	if m != nil {
+		return m.Values
+	}
+	return nil
+}
+
+func init() {
+	proto.RegisterEnum("google.protobuf.NullValue", NullValue_name, NullValue_value)
+	proto.RegisterType((*Struct)(nil), "google.protobuf.Struct")
+	proto.RegisterMapType((map[string]*Value)(nil), "google.protobuf.Struct.FieldsEntry")
+	proto.RegisterType((*Value)(nil), "google.protobuf.Value")
+	proto.RegisterType((*ListValue)(nil), "google.protobuf.ListValue")
+}
+
+func init() { proto.RegisterFile("google/protobuf/struct.proto", fileDescriptor_df322afd6c9fb402) }
+
+var fileDescriptor_df322afd6c9fb402 = []byte{
+	// 417 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x92, 0x41, 0x8b, 0xd3, 0x40,
+	0x14, 0xc7, 0x3b, 0xc9, 0x36, 0x98, 0x17, 0x59, 0x97, 0x11, 0xb4, 0xac, 0xa2, 0xa1, 0x7b, 0x09,
+	0x22, 0x29, 0xd6, 0x8b, 0x18, 0x2f, 0x06, 0xd6, 0x5d, 0x30, 0x2c, 0x31, 0xba, 0x15, 0xbc, 0x94,
+	0x26, 0x4d, 0x63, 0xe8, 0x74, 0x26, 0x24, 0x33, 0x4a, 0x8f, 0x7e, 0x0b, 0xcf, 0x1e, 0x3d, 0xfa,
+	0xe9, 0x3c, 0xca, 0xcc, 0x24, 0xa9, 0xb4, 0xf4, 0x94, 0xbc, 0xf7, 0x7e, 0xef, 0x3f, 0xef, 0xff,
+	0x66, 0xe0, 0x71, 0xc1, 0x58, 0x41, 0xf2, 0x49, 0x55, 0x33, 0xce, 0x52, 0xb1, 0x9a, 0x34, 0xbc,
+	0x16, 0x19, 0xf7, 0x55, 0x8c, 0xef, 0xe9, 0xaa, 0xdf, 0x55, 0xc7, 0x3f, 0x11, 0x58, 0x1f, 0x15,
+	0x81, 0x03, 0xb0, 0x56, 0x65, 0x4e, 0x96, 0xcd, 0x08, 0xb9, 0xa6, 0xe7, 0x4c, 0x2f, 0xfc, 0x3d,
+	0xd8, 0xd7, 0xa0, 0xff, 0x4e, 0x51, 0x97, 0x94, 0xd7, 0xdb, 0xa4, 0x6d, 0x39, 0xff, 0x00, 0xce,
+	0x7f, 0x69, 0x7c, 0x06, 0xe6, 0x3a, 0xdf, 0x8e, 0x90, 0x8b, 0x3c, 0x3b, 0x91, 0xbf, 0xf8, 0x39,
+	0x0c, 0xbf, 0x2d, 0x88, 0xc8, 0x47, 0x86, 0x8b, 0x3c, 0x67, 0xfa, 0xe0, 0x40, 0x7c, 0x26, 0xab,
+	0x89, 0x86, 0x5e, 0x1b, 0xaf, 0xd0, 0xf8, 0x8f, 0x01, 0x43, 0x95, 0xc4, 0x01, 0x00, 0x15, 0x84,
+	0xcc, 0xb5, 0x80, 0x14, 0x3d, 0x9d, 0x9e, 0x1f, 0x08, 0xdc, 0x08, 0x42, 0x14, 0x7f, 0x3d, 0x48,
+	0x6c, 0xda, 0x05, 0xf8, 0x02, 0xee, 0x52, 0xb1, 0x49, 0xf3, 0x7a, 0xbe, 0x3b, 0x1f, 0x5d, 0x0f,
+	0x12, 0x47, 0x67, 0x7b, 0xa8, 0xe1, 0x75, 0x49, 0x8b, 0x16, 0x32, 0xe5, 0xe0, 0x12, 0xd2, 0x59,
+	0x0d, 0x3d, 0x05, 0x48, 0x19, 0xeb, 0xc6, 0x38, 0x71, 0x91, 0x77, 0x47, 0x1e, 0x25, 0x73, 0x1a,
+	0x78, 0xa3, 0x54, 0x44, 0xc6, 0x5b, 0x64, 0xa8, 0xac, 0x3e, 0x3c, 0xb2, 0xc7, 0x56, 0x5e, 0x64,
+	0xbc, 0x77, 0x49, 0xca, 0xa6, 0xeb, 0xb5, 0x54, 0xef, 0xa1, 0xcb, 0xa8, 0x6c, 0x78, 0xef, 0x92,
+	0x74, 0x41, 0x68, 0xc1, 0xc9, 0xba, 0xa4, 0xcb, 0x71, 0x00, 0x76, 0x4f, 0x60, 0x1f, 0x2c, 0x25,
+	0xd6, 0xdd, 0xe8, 0xb1, 0xa5, 0xb7, 0xd4, 0xb3, 0x47, 0x60, 0xf7, 0x4b, 0xc4, 0xa7, 0x00, 0x37,
+	0xb7, 0x51, 0x34, 0x9f, 0xbd, 0x8d, 0x6e, 0x2f, 0xcf, 0x06, 0xe1, 0x0f, 0x04, 0xf7, 0x33, 0xb6,
+	0xd9, 0x97, 0x08, 0x1d, 0xed, 0x26, 0x96, 0x71, 0x8c, 0xbe, 0xbc, 0x28, 0x4a, 0xfe, 0x55, 0xa4,
+	0x7e, 0xc6, 0x36, 0x93, 0x82, 0x91, 0x05, 0x2d, 0x76, 0x4f, 0xb1, 0xe2, 0xdb, 0x2a, 0x6f, 0xda,
+	0x17, 0x19, 0xe8, 0x4f, 0x95, 0xfe, 0x45, 0xe8, 0x97, 0x61, 0x5e, 0xc5, 0xe1, 0x6f, 0xe3, 0xc9,
+	0x95, 0x16, 0x8f, 0xbb, 0xf9, 0x3e, 0xe7, 0x84, 0xbc, 0xa7, 0xec, 0x3b, 0xfd, 0x24, 0x3b, 0x53,
+	0x4b, 0x49, 0xbd, 0xfc, 0x17, 0x00, 0x00, 0xff, 0xff, 0xe8, 0x1b, 0x59, 0xf8, 0xe5, 0x02, 0x00,
+	0x00,
+}
diff --git a/vendor/github.com/golang/protobuf/ptypes/struct/struct.proto b/vendor/github.com/golang/protobuf/ptypes/struct/struct.proto
new file mode 100644
index 0000000..7d7808e
--- /dev/null
+++ b/vendor/github.com/golang/protobuf/ptypes/struct/struct.proto
@@ -0,0 +1,96 @@
+// Protocol Buffers - Google's data interchange format
+// Copyright 2008 Google Inc.  All rights reserved.
+// https://developers.google.com/protocol-buffers/
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+syntax = "proto3";
+
+package google.protobuf;
+
+option csharp_namespace = "Google.Protobuf.WellKnownTypes";
+option cc_enable_arenas = true;
+option go_package = "github.com/golang/protobuf/ptypes/struct;structpb";
+option java_package = "com.google.protobuf";
+option java_outer_classname = "StructProto";
+option java_multiple_files = true;
+option objc_class_prefix = "GPB";
+
+
+// `Struct` represents a structured data value, consisting of fields
+// which map to dynamically typed values. In some languages, `Struct`
+// might be supported by a native representation. For example, in
+// scripting languages like JS a struct is represented as an
+// object. The details of that representation are described together
+// with the proto support for the language.
+//
+// The JSON representation for `Struct` is JSON object.
+message Struct {
+  // Unordered map of dynamically typed values.
+  map<string, Value> fields = 1;
+}
+
+// `Value` represents a dynamically typed value which can be either
+// null, a number, a string, a boolean, a recursive struct value, or a
+// list of values. A producer of value is expected to set one of that
+// variants, absence of any variant indicates an error.
+//
+// The JSON representation for `Value` is JSON value.
+message Value {
+  // The kind of value.
+  oneof kind {
+    // Represents a null value.
+    NullValue null_value = 1;
+    // Represents a double value.
+    double number_value = 2;
+    // Represents a string value.
+    string string_value = 3;
+    // Represents a boolean value.
+    bool bool_value = 4;
+    // Represents a structured value.
+    Struct struct_value = 5;
+    // Represents a repeated `Value`.
+    ListValue list_value = 6;
+  }
+}
+
+// `NullValue` is a singleton enumeration to represent the null value for the
+// `Value` type union.
+//
+//  The JSON representation for `NullValue` is JSON `null`.
+enum NullValue {
+  // Null value.
+  NULL_VALUE = 0;
+}
+
+// `ListValue` is a wrapper around a repeated field of values.
+//
+// The JSON representation for `ListValue` is JSON array.
+message ListValue {
+  // Repeated field of dynamically typed values.
+  repeated Value values = 1;
+}
diff --git a/vendor/github.com/golang/protobuf/ptypes/wrappers/wrappers.pb.go b/vendor/github.com/golang/protobuf/ptypes/wrappers/wrappers.pb.go
new file mode 100644
index 0000000..add19a1
--- /dev/null
+++ b/vendor/github.com/golang/protobuf/ptypes/wrappers/wrappers.pb.go
@@ -0,0 +1,461 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: google/protobuf/wrappers.proto
+
+package wrappers
+
+import (
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	math "math"
+)
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
+
+// Wrapper message for `double`.
+//
+// The JSON representation for `DoubleValue` is JSON number.
+type DoubleValue struct {
+	// The double value.
+	Value                float64  `protobuf:"fixed64,1,opt,name=value,proto3" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *DoubleValue) Reset()         { *m = DoubleValue{} }
+func (m *DoubleValue) String() string { return proto.CompactTextString(m) }
+func (*DoubleValue) ProtoMessage()    {}
+func (*DoubleValue) Descriptor() ([]byte, []int) {
+	return fileDescriptor_5377b62bda767935, []int{0}
+}
+
+func (*DoubleValue) XXX_WellKnownType() string { return "DoubleValue" }
+
+func (m *DoubleValue) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DoubleValue.Unmarshal(m, b)
+}
+func (m *DoubleValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DoubleValue.Marshal(b, m, deterministic)
+}
+func (m *DoubleValue) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DoubleValue.Merge(m, src)
+}
+func (m *DoubleValue) XXX_Size() int {
+	return xxx_messageInfo_DoubleValue.Size(m)
+}
+func (m *DoubleValue) XXX_DiscardUnknown() {
+	xxx_messageInfo_DoubleValue.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DoubleValue proto.InternalMessageInfo
+
+func (m *DoubleValue) GetValue() float64 {
+	if m != nil {
+		return m.Value
+	}
+	return 0
+}
+
+// Wrapper message for `float`.
+//
+// The JSON representation for `FloatValue` is JSON number.
+type FloatValue struct {
+	// The float value.
+	Value                float32  `protobuf:"fixed32,1,opt,name=value,proto3" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *FloatValue) Reset()         { *m = FloatValue{} }
+func (m *FloatValue) String() string { return proto.CompactTextString(m) }
+func (*FloatValue) ProtoMessage()    {}
+func (*FloatValue) Descriptor() ([]byte, []int) {
+	return fileDescriptor_5377b62bda767935, []int{1}
+}
+
+func (*FloatValue) XXX_WellKnownType() string { return "FloatValue" }
+
+func (m *FloatValue) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_FloatValue.Unmarshal(m, b)
+}
+func (m *FloatValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_FloatValue.Marshal(b, m, deterministic)
+}
+func (m *FloatValue) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_FloatValue.Merge(m, src)
+}
+func (m *FloatValue) XXX_Size() int {
+	return xxx_messageInfo_FloatValue.Size(m)
+}
+func (m *FloatValue) XXX_DiscardUnknown() {
+	xxx_messageInfo_FloatValue.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_FloatValue proto.InternalMessageInfo
+
+func (m *FloatValue) GetValue() float32 {
+	if m != nil {
+		return m.Value
+	}
+	return 0
+}
+
+// Wrapper message for `int64`.
+//
+// The JSON representation for `Int64Value` is JSON string.
+type Int64Value struct {
+	// The int64 value.
+	Value                int64    `protobuf:"varint,1,opt,name=value,proto3" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Int64Value) Reset()         { *m = Int64Value{} }
+func (m *Int64Value) String() string { return proto.CompactTextString(m) }
+func (*Int64Value) ProtoMessage()    {}
+func (*Int64Value) Descriptor() ([]byte, []int) {
+	return fileDescriptor_5377b62bda767935, []int{2}
+}
+
+func (*Int64Value) XXX_WellKnownType() string { return "Int64Value" }
+
+func (m *Int64Value) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Int64Value.Unmarshal(m, b)
+}
+func (m *Int64Value) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Int64Value.Marshal(b, m, deterministic)
+}
+func (m *Int64Value) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Int64Value.Merge(m, src)
+}
+func (m *Int64Value) XXX_Size() int {
+	return xxx_messageInfo_Int64Value.Size(m)
+}
+func (m *Int64Value) XXX_DiscardUnknown() {
+	xxx_messageInfo_Int64Value.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Int64Value proto.InternalMessageInfo
+
+func (m *Int64Value) GetValue() int64 {
+	if m != nil {
+		return m.Value
+	}
+	return 0
+}
+
+// Wrapper message for `uint64`.
+//
+// The JSON representation for `UInt64Value` is JSON string.
+type UInt64Value struct {
+	// The uint64 value.
+	Value                uint64   `protobuf:"varint,1,opt,name=value,proto3" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *UInt64Value) Reset()         { *m = UInt64Value{} }
+func (m *UInt64Value) String() string { return proto.CompactTextString(m) }
+func (*UInt64Value) ProtoMessage()    {}
+func (*UInt64Value) Descriptor() ([]byte, []int) {
+	return fileDescriptor_5377b62bda767935, []int{3}
+}
+
+func (*UInt64Value) XXX_WellKnownType() string { return "UInt64Value" }
+
+func (m *UInt64Value) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_UInt64Value.Unmarshal(m, b)
+}
+func (m *UInt64Value) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_UInt64Value.Marshal(b, m, deterministic)
+}
+func (m *UInt64Value) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_UInt64Value.Merge(m, src)
+}
+func (m *UInt64Value) XXX_Size() int {
+	return xxx_messageInfo_UInt64Value.Size(m)
+}
+func (m *UInt64Value) XXX_DiscardUnknown() {
+	xxx_messageInfo_UInt64Value.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_UInt64Value proto.InternalMessageInfo
+
+func (m *UInt64Value) GetValue() uint64 {
+	if m != nil {
+		return m.Value
+	}
+	return 0
+}
+
+// Wrapper message for `int32`.
+//
+// The JSON representation for `Int32Value` is JSON number.
+type Int32Value struct {
+	// The int32 value.
+	Value                int32    `protobuf:"varint,1,opt,name=value,proto3" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Int32Value) Reset()         { *m = Int32Value{} }
+func (m *Int32Value) String() string { return proto.CompactTextString(m) }
+func (*Int32Value) ProtoMessage()    {}
+func (*Int32Value) Descriptor() ([]byte, []int) {
+	return fileDescriptor_5377b62bda767935, []int{4}
+}
+
+func (*Int32Value) XXX_WellKnownType() string { return "Int32Value" }
+
+func (m *Int32Value) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Int32Value.Unmarshal(m, b)
+}
+func (m *Int32Value) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Int32Value.Marshal(b, m, deterministic)
+}
+func (m *Int32Value) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Int32Value.Merge(m, src)
+}
+func (m *Int32Value) XXX_Size() int {
+	return xxx_messageInfo_Int32Value.Size(m)
+}
+func (m *Int32Value) XXX_DiscardUnknown() {
+	xxx_messageInfo_Int32Value.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Int32Value proto.InternalMessageInfo
+
+func (m *Int32Value) GetValue() int32 {
+	if m != nil {
+		return m.Value
+	}
+	return 0
+}
+
+// Wrapper message for `uint32`.
+//
+// The JSON representation for `UInt32Value` is JSON number.
+type UInt32Value struct {
+	// The uint32 value.
+	Value                uint32   `protobuf:"varint,1,opt,name=value,proto3" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *UInt32Value) Reset()         { *m = UInt32Value{} }
+func (m *UInt32Value) String() string { return proto.CompactTextString(m) }
+func (*UInt32Value) ProtoMessage()    {}
+func (*UInt32Value) Descriptor() ([]byte, []int) {
+	return fileDescriptor_5377b62bda767935, []int{5}
+}
+
+func (*UInt32Value) XXX_WellKnownType() string { return "UInt32Value" }
+
+func (m *UInt32Value) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_UInt32Value.Unmarshal(m, b)
+}
+func (m *UInt32Value) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_UInt32Value.Marshal(b, m, deterministic)
+}
+func (m *UInt32Value) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_UInt32Value.Merge(m, src)
+}
+func (m *UInt32Value) XXX_Size() int {
+	return xxx_messageInfo_UInt32Value.Size(m)
+}
+func (m *UInt32Value) XXX_DiscardUnknown() {
+	xxx_messageInfo_UInt32Value.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_UInt32Value proto.InternalMessageInfo
+
+func (m *UInt32Value) GetValue() uint32 {
+	if m != nil {
+		return m.Value
+	}
+	return 0
+}
+
+// Wrapper message for `bool`.
+//
+// The JSON representation for `BoolValue` is JSON `true` and `false`.
+type BoolValue struct {
+	// The bool value.
+	Value                bool     `protobuf:"varint,1,opt,name=value,proto3" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *BoolValue) Reset()         { *m = BoolValue{} }
+func (m *BoolValue) String() string { return proto.CompactTextString(m) }
+func (*BoolValue) ProtoMessage()    {}
+func (*BoolValue) Descriptor() ([]byte, []int) {
+	return fileDescriptor_5377b62bda767935, []int{6}
+}
+
+func (*BoolValue) XXX_WellKnownType() string { return "BoolValue" }
+
+func (m *BoolValue) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_BoolValue.Unmarshal(m, b)
+}
+func (m *BoolValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_BoolValue.Marshal(b, m, deterministic)
+}
+func (m *BoolValue) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_BoolValue.Merge(m, src)
+}
+func (m *BoolValue) XXX_Size() int {
+	return xxx_messageInfo_BoolValue.Size(m)
+}
+func (m *BoolValue) XXX_DiscardUnknown() {
+	xxx_messageInfo_BoolValue.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_BoolValue proto.InternalMessageInfo
+
+func (m *BoolValue) GetValue() bool {
+	if m != nil {
+		return m.Value
+	}
+	return false
+}
+
+// Wrapper message for `string`.
+//
+// The JSON representation for `StringValue` is JSON string.
+type StringValue struct {
+	// The string value.
+	Value                string   `protobuf:"bytes,1,opt,name=value,proto3" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *StringValue) Reset()         { *m = StringValue{} }
+func (m *StringValue) String() string { return proto.CompactTextString(m) }
+func (*StringValue) ProtoMessage()    {}
+func (*StringValue) Descriptor() ([]byte, []int) {
+	return fileDescriptor_5377b62bda767935, []int{7}
+}
+
+func (*StringValue) XXX_WellKnownType() string { return "StringValue" }
+
+func (m *StringValue) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_StringValue.Unmarshal(m, b)
+}
+func (m *StringValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_StringValue.Marshal(b, m, deterministic)
+}
+func (m *StringValue) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StringValue.Merge(m, src)
+}
+func (m *StringValue) XXX_Size() int {
+	return xxx_messageInfo_StringValue.Size(m)
+}
+func (m *StringValue) XXX_DiscardUnknown() {
+	xxx_messageInfo_StringValue.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_StringValue proto.InternalMessageInfo
+
+func (m *StringValue) GetValue() string {
+	if m != nil {
+		return m.Value
+	}
+	return ""
+}
+
+// Wrapper message for `bytes`.
+//
+// The JSON representation for `BytesValue` is JSON string.
+type BytesValue struct {
+	// The bytes value.
+	Value                []byte   `protobuf:"bytes,1,opt,name=value,proto3" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *BytesValue) Reset()         { *m = BytesValue{} }
+func (m *BytesValue) String() string { return proto.CompactTextString(m) }
+func (*BytesValue) ProtoMessage()    {}
+func (*BytesValue) Descriptor() ([]byte, []int) {
+	return fileDescriptor_5377b62bda767935, []int{8}
+}
+
+func (*BytesValue) XXX_WellKnownType() string { return "BytesValue" }
+
+func (m *BytesValue) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_BytesValue.Unmarshal(m, b)
+}
+func (m *BytesValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_BytesValue.Marshal(b, m, deterministic)
+}
+func (m *BytesValue) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_BytesValue.Merge(m, src)
+}
+func (m *BytesValue) XXX_Size() int {
+	return xxx_messageInfo_BytesValue.Size(m)
+}
+func (m *BytesValue) XXX_DiscardUnknown() {
+	xxx_messageInfo_BytesValue.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_BytesValue proto.InternalMessageInfo
+
+func (m *BytesValue) GetValue() []byte {
+	if m != nil {
+		return m.Value
+	}
+	return nil
+}
+
+func init() {
+	proto.RegisterType((*DoubleValue)(nil), "google.protobuf.DoubleValue")
+	proto.RegisterType((*FloatValue)(nil), "google.protobuf.FloatValue")
+	proto.RegisterType((*Int64Value)(nil), "google.protobuf.Int64Value")
+	proto.RegisterType((*UInt64Value)(nil), "google.protobuf.UInt64Value")
+	proto.RegisterType((*Int32Value)(nil), "google.protobuf.Int32Value")
+	proto.RegisterType((*UInt32Value)(nil), "google.protobuf.UInt32Value")
+	proto.RegisterType((*BoolValue)(nil), "google.protobuf.BoolValue")
+	proto.RegisterType((*StringValue)(nil), "google.protobuf.StringValue")
+	proto.RegisterType((*BytesValue)(nil), "google.protobuf.BytesValue")
+}
+
+func init() { proto.RegisterFile("google/protobuf/wrappers.proto", fileDescriptor_5377b62bda767935) }
+
+var fileDescriptor_5377b62bda767935 = []byte{
+	// 259 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x4b, 0xcf, 0xcf, 0x4f,
+	0xcf, 0x49, 0xd5, 0x2f, 0x28, 0xca, 0x2f, 0xc9, 0x4f, 0x2a, 0x4d, 0xd3, 0x2f, 0x2f, 0x4a, 0x2c,
+	0x28, 0x48, 0x2d, 0x2a, 0xd6, 0x03, 0x8b, 0x08, 0xf1, 0x43, 0xe4, 0xf5, 0x60, 0xf2, 0x4a, 0xca,
+	0x5c, 0xdc, 0x2e, 0xf9, 0xa5, 0x49, 0x39, 0xa9, 0x61, 0x89, 0x39, 0xa5, 0xa9, 0x42, 0x22, 0x5c,
+	0xac, 0x65, 0x20, 0x86, 0x04, 0xa3, 0x02, 0xa3, 0x06, 0x63, 0x10, 0x84, 0xa3, 0xa4, 0xc4, 0xc5,
+	0xe5, 0x96, 0x93, 0x9f, 0x58, 0x82, 0x45, 0x0d, 0x13, 0x92, 0x1a, 0xcf, 0xbc, 0x12, 0x33, 0x13,
+	0x2c, 0x6a, 0x98, 0x61, 0x6a, 0x94, 0xb9, 0xb8, 0x43, 0x71, 0x29, 0x62, 0x41, 0x35, 0xc8, 0xd8,
+	0x08, 0x8b, 0x1a, 0x56, 0x34, 0x83, 0xb0, 0x2a, 0xe2, 0x85, 0x29, 0x52, 0xe4, 0xe2, 0x74, 0xca,
+	0xcf, 0xcf, 0xc1, 0xa2, 0x84, 0x03, 0xc9, 0x9c, 0xe0, 0x92, 0xa2, 0xcc, 0xbc, 0x74, 0x2c, 0x8a,
+	0x38, 0x91, 0x1c, 0xe4, 0x54, 0x59, 0x92, 0x5a, 0x8c, 0x45, 0x0d, 0x0f, 0x54, 0x8d, 0x53, 0x0d,
+	0x97, 0x70, 0x72, 0x7e, 0xae, 0x1e, 0x5a, 0xe8, 0x3a, 0xf1, 0x86, 0x43, 0x83, 0x3f, 0x00, 0x24,
+	0x12, 0xc0, 0x18, 0xa5, 0x95, 0x9e, 0x59, 0x92, 0x51, 0x9a, 0xa4, 0x97, 0x9c, 0x9f, 0xab, 0x9f,
+	0x9e, 0x9f, 0x93, 0x98, 0x97, 0x8e, 0x88, 0xaa, 0x82, 0x92, 0xca, 0x82, 0xd4, 0x62, 0x78, 0x8c,
+	0xfd, 0x60, 0x64, 0x5c, 0xc4, 0xc4, 0xec, 0x1e, 0xe0, 0xb4, 0x8a, 0x49, 0xce, 0x1d, 0x62, 0x6e,
+	0x00, 0x54, 0xa9, 0x5e, 0x78, 0x6a, 0x4e, 0x8e, 0x77, 0x5e, 0x7e, 0x79, 0x5e, 0x08, 0x48, 0x4b,
+	0x12, 0x1b, 0xd8, 0x0c, 0x63, 0x40, 0x00, 0x00, 0x00, 0xff, 0xff, 0x19, 0x6c, 0xb9, 0xb8, 0xfe,
+	0x01, 0x00, 0x00,
+}
diff --git a/vendor/github.com/golang/protobuf/ptypes/wrappers/wrappers.proto b/vendor/github.com/golang/protobuf/ptypes/wrappers/wrappers.proto
new file mode 100644
index 0000000..0194763
--- /dev/null
+++ b/vendor/github.com/golang/protobuf/ptypes/wrappers/wrappers.proto
@@ -0,0 +1,118 @@
+// Protocol Buffers - Google's data interchange format
+// Copyright 2008 Google Inc.  All rights reserved.
+// https://developers.google.com/protocol-buffers/
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+// Wrappers for primitive (non-message) types. These types are useful
+// for embedding primitives in the `google.protobuf.Any` type and for places
+// where we need to distinguish between the absence of a primitive
+// typed field and its default value.
+
+syntax = "proto3";
+
+package google.protobuf;
+
+option csharp_namespace = "Google.Protobuf.WellKnownTypes";
+option cc_enable_arenas = true;
+option go_package = "github.com/golang/protobuf/ptypes/wrappers";
+option java_package = "com.google.protobuf";
+option java_outer_classname = "WrappersProto";
+option java_multiple_files = true;
+option objc_class_prefix = "GPB";
+
+// Wrapper message for `double`.
+//
+// The JSON representation for `DoubleValue` is JSON number.
+message DoubleValue {
+  // The double value.
+  double value = 1;
+}
+
+// Wrapper message for `float`.
+//
+// The JSON representation for `FloatValue` is JSON number.
+message FloatValue {
+  // The float value.
+  float value = 1;
+}
+
+// Wrapper message for `int64`.
+//
+// The JSON representation for `Int64Value` is JSON string.
+message Int64Value {
+  // The int64 value.
+  int64 value = 1;
+}
+
+// Wrapper message for `uint64`.
+//
+// The JSON representation for `UInt64Value` is JSON string.
+message UInt64Value {
+  // The uint64 value.
+  uint64 value = 1;
+}
+
+// Wrapper message for `int32`.
+//
+// The JSON representation for `Int32Value` is JSON number.
+message Int32Value {
+  // The int32 value.
+  int32 value = 1;
+}
+
+// Wrapper message for `uint32`.
+//
+// The JSON representation for `UInt32Value` is JSON number.
+message UInt32Value {
+  // The uint32 value.
+  uint32 value = 1;
+}
+
+// Wrapper message for `bool`.
+//
+// The JSON representation for `BoolValue` is JSON `true` and `false`.
+message BoolValue {
+  // The bool value.
+  bool value = 1;
+}
+
+// Wrapper message for `string`.
+//
+// The JSON representation for `StringValue` is JSON string.
+message StringValue {
+  // The string value.
+  string value = 1;
+}
+
+// Wrapper message for `bytes`.
+//
+// The JSON representation for `BytesValue` is JSON string.
+message BytesValue {
+  // The bytes value.
+  bytes value = 1;
+}
diff --git a/vendor/github.com/google/btree/.travis.yml b/vendor/github.com/google/btree/.travis.yml
new file mode 100644
index 0000000..4f2ee4d
--- /dev/null
+++ b/vendor/github.com/google/btree/.travis.yml
@@ -0,0 +1 @@
+language: go
diff --git a/vendor/github.com/google/btree/LICENSE b/vendor/github.com/google/btree/LICENSE
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/vendor/github.com/google/btree/LICENSE
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
diff --git a/vendor/github.com/google/btree/README.md b/vendor/github.com/google/btree/README.md
new file mode 100644
index 0000000..6062a4d
--- /dev/null
+++ b/vendor/github.com/google/btree/README.md
@@ -0,0 +1,12 @@
+# BTree implementation for Go
+
+![Travis CI Build Status](https://api.travis-ci.org/google/btree.svg?branch=master)
+
+This package provides an in-memory B-Tree implementation for Go, useful as
+an ordered, mutable data structure.
+
+The API is based off of the wonderful
+http://godoc.org/github.com/petar/GoLLRB/llrb, and is meant to allow btree to
+act as a drop-in replacement for gollrb trees.
+
+See http://godoc.org/github.com/google/btree for documentation.
diff --git a/vendor/github.com/google/btree/btree.go b/vendor/github.com/google/btree/btree.go
new file mode 100644
index 0000000..6ff062f
--- /dev/null
+++ b/vendor/github.com/google/btree/btree.go
@@ -0,0 +1,890 @@
+// Copyright 2014 Google Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// Package btree implements in-memory B-Trees of arbitrary degree.
+//
+// btree implements an in-memory B-Tree for use as an ordered data structure.
+// It is not meant for persistent storage solutions.
+//
+// It has a flatter structure than an equivalent red-black or other binary tree,
+// which in some cases yields better memory usage and/or performance.
+// See some discussion on the matter here:
+//   http://google-opensource.blogspot.com/2013/01/c-containers-that-save-memory-and-time.html
+// Note, though, that this project is in no way related to the C++ B-Tree
+// implementation written about there.
+//
+// Within this tree, each node contains a slice of items and a (possibly nil)
+// slice of children.  For basic numeric values or raw structs, this can cause
+// efficiency differences when compared to equivalent C++ template code that
+// stores values in arrays within the node:
+//   * Due to the overhead of storing values as interfaces (each
+//     value needs to be stored as the value itself, then 2 words for the
+//     interface pointing to that value and its type), resulting in higher
+//     memory use.
+//   * Since interfaces can point to values anywhere in memory, values are
+//     most likely not stored in contiguous blocks, resulting in a higher
+//     number of cache misses.
+// These issues don't tend to matter, though, when working with strings or other
+// heap-allocated structures, since C++-equivalent structures also must store
+// pointers and also distribute their values across the heap.
+//
+// This implementation is designed to be a drop-in replacement to gollrb.LLRB
+// trees, (http://github.com/petar/gollrb), an excellent and probably the most
+// widely used ordered tree implementation in the Go ecosystem currently.
+// Its functions, therefore, exactly mirror those of
+// llrb.LLRB where possible.  Unlike gollrb, though, we currently don't
+// support storing multiple equivalent values.
+package btree
+
+import (
+	"fmt"
+	"io"
+	"sort"
+	"strings"
+	"sync"
+)
+
+// Item represents a single object in the tree.
+type Item interface {
+	// Less tests whether the current item is less than the given argument.
+	//
+	// This must provide a strict weak ordering.
+	// If !a.Less(b) && !b.Less(a), we treat this to mean a == b (i.e. we can only
+	// hold one of either a or b in the tree).
+	Less(than Item) bool
+}
+
+const (
+	DefaultFreeListSize = 32
+)
+
+var (
+	nilItems    = make(items, 16)
+	nilChildren = make(children, 16)
+)
+
+// FreeList represents a free list of btree nodes. By default each
+// BTree has its own FreeList, but multiple BTrees can share the same
+// FreeList.
+// Two Btrees using the same freelist are safe for concurrent write access.
+type FreeList struct {
+	mu       sync.Mutex
+	freelist []*node
+}
+
+// NewFreeList creates a new free list.
+// size is the maximum size of the returned free list.
+func NewFreeList(size int) *FreeList {
+	return &FreeList{freelist: make([]*node, 0, size)}
+}
+
+func (f *FreeList) newNode() (n *node) {
+	f.mu.Lock()
+	index := len(f.freelist) - 1
+	if index < 0 {
+		f.mu.Unlock()
+		return new(node)
+	}
+	n = f.freelist[index]
+	f.freelist[index] = nil
+	f.freelist = f.freelist[:index]
+	f.mu.Unlock()
+	return
+}
+
+// freeNode adds the given node to the list, returning true if it was added
+// and false if it was discarded.
+func (f *FreeList) freeNode(n *node) (out bool) {
+	f.mu.Lock()
+	if len(f.freelist) < cap(f.freelist) {
+		f.freelist = append(f.freelist, n)
+		out = true
+	}
+	f.mu.Unlock()
+	return
+}
+
+// ItemIterator allows callers of Ascend* to iterate in-order over portions of
+// the tree.  When this function returns false, iteration will stop and the
+// associated Ascend* function will immediately return.
+type ItemIterator func(i Item) bool
+
+// New creates a new B-Tree with the given degree.
+//
+// New(2), for example, will create a 2-3-4 tree (each node contains 1-3 items
+// and 2-4 children).
+func New(degree int) *BTree {
+	return NewWithFreeList(degree, NewFreeList(DefaultFreeListSize))
+}
+
+// NewWithFreeList creates a new B-Tree that uses the given node free list.
+func NewWithFreeList(degree int, f *FreeList) *BTree {
+	if degree <= 1 {
+		panic("bad degree")
+	}
+	return &BTree{
+		degree: degree,
+		cow:    &copyOnWriteContext{freelist: f},
+	}
+}
+
+// items stores items in a node.
+type items []Item
+
+// insertAt inserts a value into the given index, pushing all subsequent values
+// forward.
+func (s *items) insertAt(index int, item Item) {
+	*s = append(*s, nil)
+	if index < len(*s) {
+		copy((*s)[index+1:], (*s)[index:])
+	}
+	(*s)[index] = item
+}
+
+// removeAt removes a value at a given index, pulling all subsequent values
+// back.
+func (s *items) removeAt(index int) Item {
+	item := (*s)[index]
+	copy((*s)[index:], (*s)[index+1:])
+	(*s)[len(*s)-1] = nil
+	*s = (*s)[:len(*s)-1]
+	return item
+}
+
+// pop removes and returns the last element in the list.
+func (s *items) pop() (out Item) {
+	index := len(*s) - 1
+	out = (*s)[index]
+	(*s)[index] = nil
+	*s = (*s)[:index]
+	return
+}
+
+// truncate truncates this instance at index so that it contains only the
+// first index items. index must be less than or equal to length.
+func (s *items) truncate(index int) {
+	var toClear items
+	*s, toClear = (*s)[:index], (*s)[index:]
+	for len(toClear) > 0 {
+		toClear = toClear[copy(toClear, nilItems):]
+	}
+}
+
+// find returns the index where the given item should be inserted into this
+// list.  'found' is true if the item already exists in the list at the given
+// index.
+func (s items) find(item Item) (index int, found bool) {
+	i := sort.Search(len(s), func(i int) bool {
+		return item.Less(s[i])
+	})
+	if i > 0 && !s[i-1].Less(item) {
+		return i - 1, true
+	}
+	return i, false
+}
+
+// children stores child nodes in a node.
+type children []*node
+
+// insertAt inserts a value into the given index, pushing all subsequent values
+// forward.
+func (s *children) insertAt(index int, n *node) {
+	*s = append(*s, nil)
+	if index < len(*s) {
+		copy((*s)[index+1:], (*s)[index:])
+	}
+	(*s)[index] = n
+}
+
+// removeAt removes a value at a given index, pulling all subsequent values
+// back.
+func (s *children) removeAt(index int) *node {
+	n := (*s)[index]
+	copy((*s)[index:], (*s)[index+1:])
+	(*s)[len(*s)-1] = nil
+	*s = (*s)[:len(*s)-1]
+	return n
+}
+
+// pop removes and returns the last element in the list.
+func (s *children) pop() (out *node) {
+	index := len(*s) - 1
+	out = (*s)[index]
+	(*s)[index] = nil
+	*s = (*s)[:index]
+	return
+}
+
+// truncate truncates this instance at index so that it contains only the
+// first index children. index must be less than or equal to length.
+func (s *children) truncate(index int) {
+	var toClear children
+	*s, toClear = (*s)[:index], (*s)[index:]
+	for len(toClear) > 0 {
+		toClear = toClear[copy(toClear, nilChildren):]
+	}
+}
+
+// node is an internal node in a tree.
+//
+// It must at all times maintain the invariant that either
+//   * len(children) == 0, len(items) unconstrained
+//   * len(children) == len(items) + 1
+type node struct {
+	items    items
+	children children
+	cow      *copyOnWriteContext
+}
+
+func (n *node) mutableFor(cow *copyOnWriteContext) *node {
+	if n.cow == cow {
+		return n
+	}
+	out := cow.newNode()
+	if cap(out.items) >= len(n.items) {
+		out.items = out.items[:len(n.items)]
+	} else {
+		out.items = make(items, len(n.items), cap(n.items))
+	}
+	copy(out.items, n.items)
+	// Copy children
+	if cap(out.children) >= len(n.children) {
+		out.children = out.children[:len(n.children)]
+	} else {
+		out.children = make(children, len(n.children), cap(n.children))
+	}
+	copy(out.children, n.children)
+	return out
+}
+
+func (n *node) mutableChild(i int) *node {
+	c := n.children[i].mutableFor(n.cow)
+	n.children[i] = c
+	return c
+}
+
+// split splits the given node at the given index.  The current node shrinks,
+// and this function returns the item that existed at that index and a new node
+// containing all items/children after it.
+func (n *node) split(i int) (Item, *node) {
+	item := n.items[i]
+	next := n.cow.newNode()
+	next.items = append(next.items, n.items[i+1:]...)
+	n.items.truncate(i)
+	if len(n.children) > 0 {
+		next.children = append(next.children, n.children[i+1:]...)
+		n.children.truncate(i + 1)
+	}
+	return item, next
+}
+
+// maybeSplitChild checks if a child should be split, and if so splits it.
+// Returns whether or not a split occurred.
+func (n *node) maybeSplitChild(i, maxItems int) bool {
+	if len(n.children[i].items) < maxItems {
+		return false
+	}
+	first := n.mutableChild(i)
+	item, second := first.split(maxItems / 2)
+	n.items.insertAt(i, item)
+	n.children.insertAt(i+1, second)
+	return true
+}
+
+// insert inserts an item into the subtree rooted at this node, making sure
+// no nodes in the subtree exceed maxItems items.  Should an equivalent item be
+// be found/replaced by insert, it will be returned.
+func (n *node) insert(item Item, maxItems int) Item {
+	i, found := n.items.find(item)
+	if found {
+		out := n.items[i]
+		n.items[i] = item
+		return out
+	}
+	if len(n.children) == 0 {
+		n.items.insertAt(i, item)
+		return nil
+	}
+	if n.maybeSplitChild(i, maxItems) {
+		inTree := n.items[i]
+		switch {
+		case item.Less(inTree):
+			// no change, we want first split node
+		case inTree.Less(item):
+			i++ // we want second split node
+		default:
+			out := n.items[i]
+			n.items[i] = item
+			return out
+		}
+	}
+	return n.mutableChild(i).insert(item, maxItems)
+}
+
+// get finds the given key in the subtree and returns it.
+func (n *node) get(key Item) Item {
+	i, found := n.items.find(key)
+	if found {
+		return n.items[i]
+	} else if len(n.children) > 0 {
+		return n.children[i].get(key)
+	}
+	return nil
+}
+
+// min returns the first item in the subtree.
+func min(n *node) Item {
+	if n == nil {
+		return nil
+	}
+	for len(n.children) > 0 {
+		n = n.children[0]
+	}
+	if len(n.items) == 0 {
+		return nil
+	}
+	return n.items[0]
+}
+
+// max returns the last item in the subtree.
+func max(n *node) Item {
+	if n == nil {
+		return nil
+	}
+	for len(n.children) > 0 {
+		n = n.children[len(n.children)-1]
+	}
+	if len(n.items) == 0 {
+		return nil
+	}
+	return n.items[len(n.items)-1]
+}
+
+// toRemove details what item to remove in a node.remove call.
+type toRemove int
+
+const (
+	removeItem toRemove = iota // removes the given item
+	removeMin                  // removes smallest item in the subtree
+	removeMax                  // removes largest item in the subtree
+)
+
+// remove removes an item from the subtree rooted at this node.
+func (n *node) remove(item Item, minItems int, typ toRemove) Item {
+	var i int
+	var found bool
+	switch typ {
+	case removeMax:
+		if len(n.children) == 0 {
+			return n.items.pop()
+		}
+		i = len(n.items)
+	case removeMin:
+		if len(n.children) == 0 {
+			return n.items.removeAt(0)
+		}
+		i = 0
+	case removeItem:
+		i, found = n.items.find(item)
+		if len(n.children) == 0 {
+			if found {
+				return n.items.removeAt(i)
+			}
+			return nil
+		}
+	default:
+		panic("invalid type")
+	}
+	// If we get to here, we have children.
+	if len(n.children[i].items) <= minItems {
+		return n.growChildAndRemove(i, item, minItems, typ)
+	}
+	child := n.mutableChild(i)
+	// Either we had enough items to begin with, or we've done some
+	// merging/stealing, because we've got enough now and we're ready to return
+	// stuff.
+	if found {
+		// The item exists at index 'i', and the child we've selected can give us a
+		// predecessor, since if we've gotten here it's got > minItems items in it.
+		out := n.items[i]
+		// We use our special-case 'remove' call with typ=maxItem to pull the
+		// predecessor of item i (the rightmost leaf of our immediate left child)
+		// and set it into where we pulled the item from.
+		n.items[i] = child.remove(nil, minItems, removeMax)
+		return out
+	}
+	// Final recursive call.  Once we're here, we know that the item isn't in this
+	// node and that the child is big enough to remove from.
+	return child.remove(item, minItems, typ)
+}
+
+// growChildAndRemove grows child 'i' to make sure it's possible to remove an
+// item from it while keeping it at minItems, then calls remove to actually
+// remove it.
+//
+// Most documentation says we have to do two sets of special casing:
+//   1) item is in this node
+//   2) item is in child
+// In both cases, we need to handle the two subcases:
+//   A) node has enough values that it can spare one
+//   B) node doesn't have enough values
+// For the latter, we have to check:
+//   a) left sibling has node to spare
+//   b) right sibling has node to spare
+//   c) we must merge
+// To simplify our code here, we handle cases #1 and #2 the same:
+// If a node doesn't have enough items, we make sure it does (using a,b,c).
+// We then simply redo our remove call, and the second time (regardless of
+// whether we're in case 1 or 2), we'll have enough items and can guarantee
+// that we hit case A.
+func (n *node) growChildAndRemove(i int, item Item, minItems int, typ toRemove) Item {
+	if i > 0 && len(n.children[i-1].items) > minItems {
+		// Steal from left child
+		child := n.mutableChild(i)
+		stealFrom := n.mutableChild(i - 1)
+		stolenItem := stealFrom.items.pop()
+		child.items.insertAt(0, n.items[i-1])
+		n.items[i-1] = stolenItem
+		if len(stealFrom.children) > 0 {
+			child.children.insertAt(0, stealFrom.children.pop())
+		}
+	} else if i < len(n.items) && len(n.children[i+1].items) > minItems {
+		// steal from right child
+		child := n.mutableChild(i)
+		stealFrom := n.mutableChild(i + 1)
+		stolenItem := stealFrom.items.removeAt(0)
+		child.items = append(child.items, n.items[i])
+		n.items[i] = stolenItem
+		if len(stealFrom.children) > 0 {
+			child.children = append(child.children, stealFrom.children.removeAt(0))
+		}
+	} else {
+		if i >= len(n.items) {
+			i--
+		}
+		child := n.mutableChild(i)
+		// merge with right child
+		mergeItem := n.items.removeAt(i)
+		mergeChild := n.children.removeAt(i + 1)
+		child.items = append(child.items, mergeItem)
+		child.items = append(child.items, mergeChild.items...)
+		child.children = append(child.children, mergeChild.children...)
+		n.cow.freeNode(mergeChild)
+	}
+	return n.remove(item, minItems, typ)
+}
+
+type direction int
+
+const (
+	descend = direction(-1)
+	ascend  = direction(+1)
+)
+
+// iterate provides a simple method for iterating over elements in the tree.
+//
+// When ascending, the 'start' should be less than 'stop' and when descending,
+// the 'start' should be greater than 'stop'. Setting 'includeStart' to true
+// will force the iterator to include the first item when it equals 'start',
+// thus creating a "greaterOrEqual" or "lessThanEqual" rather than just a
+// "greaterThan" or "lessThan" queries.
+func (n *node) iterate(dir direction, start, stop Item, includeStart bool, hit bool, iter ItemIterator) (bool, bool) {
+	var ok, found bool
+	var index int
+	switch dir {
+	case ascend:
+		if start != nil {
+			index, _ = n.items.find(start)
+		}
+		for i := index; i < len(n.items); i++ {
+			if len(n.children) > 0 {
+				if hit, ok = n.children[i].iterate(dir, start, stop, includeStart, hit, iter); !ok {
+					return hit, false
+				}
+			}
+			if !includeStart && !hit && start != nil && !start.Less(n.items[i]) {
+				hit = true
+				continue
+			}
+			hit = true
+			if stop != nil && !n.items[i].Less(stop) {
+				return hit, false
+			}
+			if !iter(n.items[i]) {
+				return hit, false
+			}
+		}
+		if len(n.children) > 0 {
+			if hit, ok = n.children[len(n.children)-1].iterate(dir, start, stop, includeStart, hit, iter); !ok {
+				return hit, false
+			}
+		}
+	case descend:
+		if start != nil {
+			index, found = n.items.find(start)
+			if !found {
+				index = index - 1
+			}
+		} else {
+			index = len(n.items) - 1
+		}
+		for i := index; i >= 0; i-- {
+			if start != nil && !n.items[i].Less(start) {
+				if !includeStart || hit || start.Less(n.items[i]) {
+					continue
+				}
+			}
+			if len(n.children) > 0 {
+				if hit, ok = n.children[i+1].iterate(dir, start, stop, includeStart, hit, iter); !ok {
+					return hit, false
+				}
+			}
+			if stop != nil && !stop.Less(n.items[i]) {
+				return hit, false //	continue
+			}
+			hit = true
+			if !iter(n.items[i]) {
+				return hit, false
+			}
+		}
+		if len(n.children) > 0 {
+			if hit, ok = n.children[0].iterate(dir, start, stop, includeStart, hit, iter); !ok {
+				return hit, false
+			}
+		}
+	}
+	return hit, true
+}
+
+// Used for testing/debugging purposes.
+func (n *node) print(w io.Writer, level int) {
+	fmt.Fprintf(w, "%sNODE:%v\n", strings.Repeat("  ", level), n.items)
+	for _, c := range n.children {
+		c.print(w, level+1)
+	}
+}
+
+// BTree is an implementation of a B-Tree.
+//
+// BTree stores Item instances in an ordered structure, allowing easy insertion,
+// removal, and iteration.
+//
+// Write operations are not safe for concurrent mutation by multiple
+// goroutines, but Read operations are.
+type BTree struct {
+	degree int
+	length int
+	root   *node
+	cow    *copyOnWriteContext
+}
+
+// copyOnWriteContext pointers determine node ownership... a tree with a write
+// context equivalent to a node's write context is allowed to modify that node.
+// A tree whose write context does not match a node's is not allowed to modify
+// it, and must create a new, writable copy (IE: it's a Clone).
+//
+// When doing any write operation, we maintain the invariant that the current
+// node's context is equal to the context of the tree that requested the write.
+// We do this by, before we descend into any node, creating a copy with the
+// correct context if the contexts don't match.
+//
+// Since the node we're currently visiting on any write has the requesting
+// tree's context, that node is modifiable in place.  Children of that node may
+// not share context, but before we descend into them, we'll make a mutable
+// copy.
+type copyOnWriteContext struct {
+	freelist *FreeList
+}
+
+// Clone clones the btree, lazily.  Clone should not be called concurrently,
+// but the original tree (t) and the new tree (t2) can be used concurrently
+// once the Clone call completes.
+//
+// The internal tree structure of b is marked read-only and shared between t and
+// t2.  Writes to both t and t2 use copy-on-write logic, creating new nodes
+// whenever one of b's original nodes would have been modified.  Read operations
+// should have no performance degredation.  Write operations for both t and t2
+// will initially experience minor slow-downs caused by additional allocs and
+// copies due to the aforementioned copy-on-write logic, but should converge to
+// the original performance characteristics of the original tree.
+func (t *BTree) Clone() (t2 *BTree) {
+	// Create two entirely new copy-on-write contexts.
+	// This operation effectively creates three trees:
+	//   the original, shared nodes (old b.cow)
+	//   the new b.cow nodes
+	//   the new out.cow nodes
+	cow1, cow2 := *t.cow, *t.cow
+	out := *t
+	t.cow = &cow1
+	out.cow = &cow2
+	return &out
+}
+
+// maxItems returns the max number of items to allow per node.
+func (t *BTree) maxItems() int {
+	return t.degree*2 - 1
+}
+
+// minItems returns the min number of items to allow per node (ignored for the
+// root node).
+func (t *BTree) minItems() int {
+	return t.degree - 1
+}
+
+func (c *copyOnWriteContext) newNode() (n *node) {
+	n = c.freelist.newNode()
+	n.cow = c
+	return
+}
+
+type freeType int
+
+const (
+	ftFreelistFull freeType = iota // node was freed (available for GC, not stored in freelist)
+	ftStored                       // node was stored in the freelist for later use
+	ftNotOwned                     // node was ignored by COW, since it's owned by another one
+)
+
+// freeNode frees a node within a given COW context, if it's owned by that
+// context.  It returns what happened to the node (see freeType const
+// documentation).
+func (c *copyOnWriteContext) freeNode(n *node) freeType {
+	if n.cow == c {
+		// clear to allow GC
+		n.items.truncate(0)
+		n.children.truncate(0)
+		n.cow = nil
+		if c.freelist.freeNode(n) {
+			return ftStored
+		} else {
+			return ftFreelistFull
+		}
+	} else {
+		return ftNotOwned
+	}
+}
+
+// ReplaceOrInsert adds the given item to the tree.  If an item in the tree
+// already equals the given one, it is removed from the tree and returned.
+// Otherwise, nil is returned.
+//
+// nil cannot be added to the tree (will panic).
+func (t *BTree) ReplaceOrInsert(item Item) Item {
+	if item == nil {
+		panic("nil item being added to BTree")
+	}
+	if t.root == nil {
+		t.root = t.cow.newNode()
+		t.root.items = append(t.root.items, item)
+		t.length++
+		return nil
+	} else {
+		t.root = t.root.mutableFor(t.cow)
+		if len(t.root.items) >= t.maxItems() {
+			item2, second := t.root.split(t.maxItems() / 2)
+			oldroot := t.root
+			t.root = t.cow.newNode()
+			t.root.items = append(t.root.items, item2)
+			t.root.children = append(t.root.children, oldroot, second)
+		}
+	}
+	out := t.root.insert(item, t.maxItems())
+	if out == nil {
+		t.length++
+	}
+	return out
+}
+
+// Delete removes an item equal to the passed in item from the tree, returning
+// it.  If no such item exists, returns nil.
+func (t *BTree) Delete(item Item) Item {
+	return t.deleteItem(item, removeItem)
+}
+
+// DeleteMin removes the smallest item in the tree and returns it.
+// If no such item exists, returns nil.
+func (t *BTree) DeleteMin() Item {
+	return t.deleteItem(nil, removeMin)
+}
+
+// DeleteMax removes the largest item in the tree and returns it.
+// If no such item exists, returns nil.
+func (t *BTree) DeleteMax() Item {
+	return t.deleteItem(nil, removeMax)
+}
+
+func (t *BTree) deleteItem(item Item, typ toRemove) Item {
+	if t.root == nil || len(t.root.items) == 0 {
+		return nil
+	}
+	t.root = t.root.mutableFor(t.cow)
+	out := t.root.remove(item, t.minItems(), typ)
+	if len(t.root.items) == 0 && len(t.root.children) > 0 {
+		oldroot := t.root
+		t.root = t.root.children[0]
+		t.cow.freeNode(oldroot)
+	}
+	if out != nil {
+		t.length--
+	}
+	return out
+}
+
+// AscendRange calls the iterator for every value in the tree within the range
+// [greaterOrEqual, lessThan), until iterator returns false.
+func (t *BTree) AscendRange(greaterOrEqual, lessThan Item, iterator ItemIterator) {
+	if t.root == nil {
+		return
+	}
+	t.root.iterate(ascend, greaterOrEqual, lessThan, true, false, iterator)
+}
+
+// AscendLessThan calls the iterator for every value in the tree within the range
+// [first, pivot), until iterator returns false.
+func (t *BTree) AscendLessThan(pivot Item, iterator ItemIterator) {
+	if t.root == nil {
+		return
+	}
+	t.root.iterate(ascend, nil, pivot, false, false, iterator)
+}
+
+// AscendGreaterOrEqual calls the iterator for every value in the tree within
+// the range [pivot, last], until iterator returns false.
+func (t *BTree) AscendGreaterOrEqual(pivot Item, iterator ItemIterator) {
+	if t.root == nil {
+		return
+	}
+	t.root.iterate(ascend, pivot, nil, true, false, iterator)
+}
+
+// Ascend calls the iterator for every value in the tree within the range
+// [first, last], until iterator returns false.
+func (t *BTree) Ascend(iterator ItemIterator) {
+	if t.root == nil {
+		return
+	}
+	t.root.iterate(ascend, nil, nil, false, false, iterator)
+}
+
+// DescendRange calls the iterator for every value in the tree within the range
+// [lessOrEqual, greaterThan), until iterator returns false.
+func (t *BTree) DescendRange(lessOrEqual, greaterThan Item, iterator ItemIterator) {
+	if t.root == nil {
+		return
+	}
+	t.root.iterate(descend, lessOrEqual, greaterThan, true, false, iterator)
+}
+
+// DescendLessOrEqual calls the iterator for every value in the tree within the range
+// [pivot, first], until iterator returns false.
+func (t *BTree) DescendLessOrEqual(pivot Item, iterator ItemIterator) {
+	if t.root == nil {
+		return
+	}
+	t.root.iterate(descend, pivot, nil, true, false, iterator)
+}
+
+// DescendGreaterThan calls the iterator for every value in the tree within
+// the range (pivot, last], until iterator returns false.
+func (t *BTree) DescendGreaterThan(pivot Item, iterator ItemIterator) {
+	if t.root == nil {
+		return
+	}
+	t.root.iterate(descend, nil, pivot, false, false, iterator)
+}
+
+// Descend calls the iterator for every value in the tree within the range
+// [last, first], until iterator returns false.
+func (t *BTree) Descend(iterator ItemIterator) {
+	if t.root == nil {
+		return
+	}
+	t.root.iterate(descend, nil, nil, false, false, iterator)
+}
+
+// Get looks for the key item in the tree, returning it.  It returns nil if
+// unable to find that item.
+func (t *BTree) Get(key Item) Item {
+	if t.root == nil {
+		return nil
+	}
+	return t.root.get(key)
+}
+
+// Min returns the smallest item in the tree, or nil if the tree is empty.
+func (t *BTree) Min() Item {
+	return min(t.root)
+}
+
+// Max returns the largest item in the tree, or nil if the tree is empty.
+func (t *BTree) Max() Item {
+	return max(t.root)
+}
+
+// Has returns true if the given key is in the tree.
+func (t *BTree) Has(key Item) bool {
+	return t.Get(key) != nil
+}
+
+// Len returns the number of items currently in the tree.
+func (t *BTree) Len() int {
+	return t.length
+}
+
+// Clear removes all items from the btree.  If addNodesToFreelist is true,
+// t's nodes are added to its freelist as part of this call, until the freelist
+// is full.  Otherwise, the root node is simply dereferenced and the subtree
+// left to Go's normal GC processes.
+//
+// This can be much faster
+// than calling Delete on all elements, because that requires finding/removing
+// each element in the tree and updating the tree accordingly.  It also is
+// somewhat faster than creating a new tree to replace the old one, because
+// nodes from the old tree are reclaimed into the freelist for use by the new
+// one, instead of being lost to the garbage collector.
+//
+// This call takes:
+//   O(1): when addNodesToFreelist is false, this is a single operation.
+//   O(1): when the freelist is already full, it breaks out immediately
+//   O(freelist size):  when the freelist is empty and the nodes are all owned
+//       by this tree, nodes are added to the freelist until full.
+//   O(tree size):  when all nodes are owned by another tree, all nodes are
+//       iterated over looking for nodes to add to the freelist, and due to
+//       ownership, none are.
+func (t *BTree) Clear(addNodesToFreelist bool) {
+	if t.root != nil && addNodesToFreelist {
+		t.root.reset(t.cow)
+	}
+	t.root, t.length = nil, 0
+}
+
+// reset returns a subtree to the freelist.  It breaks out immediately if the
+// freelist is full, since the only benefit of iterating is to fill that
+// freelist up.  Returns true if parent reset call should continue.
+func (n *node) reset(c *copyOnWriteContext) bool {
+	for _, child := range n.children {
+		if !child.reset(c) {
+			return false
+		}
+	}
+	return c.freeNode(n) != ftFreelistFull
+}
+
+// Int implements the Item interface for integers.
+type Int int
+
+// Less returns true if int(a) < int(b).
+func (a Int) Less(b Item) bool {
+	return a < b.(Int)
+}
diff --git a/vendor/github.com/google/btree/btree_mem.go b/vendor/github.com/google/btree/btree_mem.go
new file mode 100644
index 0000000..cb95b7f
--- /dev/null
+++ b/vendor/github.com/google/btree/btree_mem.go
@@ -0,0 +1,76 @@
+// Copyright 2014 Google Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// +build ignore
+
+// This binary compares memory usage between btree and gollrb.
+package main
+
+import (
+	"flag"
+	"fmt"
+	"math/rand"
+	"runtime"
+	"time"
+
+	"github.com/google/btree"
+	"github.com/petar/GoLLRB/llrb"
+)
+
+var (
+	size   = flag.Int("size", 1000000, "size of the tree to build")
+	degree = flag.Int("degree", 8, "degree of btree")
+	gollrb = flag.Bool("llrb", false, "use llrb instead of btree")
+)
+
+func main() {
+	flag.Parse()
+	vals := rand.Perm(*size)
+	var t, v interface{}
+	v = vals
+	var stats runtime.MemStats
+	for i := 0; i < 10; i++ {
+		runtime.GC()
+	}
+	fmt.Println("-------- BEFORE ----------")
+	runtime.ReadMemStats(&stats)
+	fmt.Printf("%+v\n", stats)
+	start := time.Now()
+	if *gollrb {
+		tr := llrb.New()
+		for _, v := range vals {
+			tr.ReplaceOrInsert(llrb.Int(v))
+		}
+		t = tr // keep it around
+	} else {
+		tr := btree.New(*degree)
+		for _, v := range vals {
+			tr.ReplaceOrInsert(btree.Int(v))
+		}
+		t = tr // keep it around
+	}
+	fmt.Printf("%v inserts in %v\n", *size, time.Since(start))
+	fmt.Println("-------- AFTER ----------")
+	runtime.ReadMemStats(&stats)
+	fmt.Printf("%+v\n", stats)
+	for i := 0; i < 10; i++ {
+		runtime.GC()
+	}
+	fmt.Println("-------- AFTER GC ----------")
+	runtime.ReadMemStats(&stats)
+	fmt.Printf("%+v\n", stats)
+	if t == v {
+		fmt.Println("to make sure vals and tree aren't GC'd")
+	}
+}
diff --git a/vendor/github.com/gorilla/websocket/.gitignore b/vendor/github.com/gorilla/websocket/.gitignore
new file mode 100644
index 0000000..ac71020
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/.gitignore
@@ -0,0 +1,25 @@
+# Compiled Object files, Static and Dynamic libs (Shared Objects)
+*.o
+*.a
+*.so
+
+# Folders
+_obj
+_test
+
+# Architecture specific extensions/prefixes
+*.[568vq]
+[568vq].out
+
+*.cgo1.go
+*.cgo2.c
+_cgo_defun.c
+_cgo_gotypes.go
+_cgo_export.*
+
+_testmain.go
+
+*.exe
+
+.idea/
+*.iml
\ No newline at end of file
diff --git a/vendor/github.com/gorilla/websocket/.travis.yml b/vendor/github.com/gorilla/websocket/.travis.yml
new file mode 100644
index 0000000..9f233f9
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/.travis.yml
@@ -0,0 +1,20 @@
+language: go
+sudo: false
+
+matrix:
+  include:
+    - go: 1.4
+    - go: 1.5
+    - go: 1.6
+    - go: 1.7
+    - go: 1.8
+    - go: 1.9
+    - go: tip
+  allow_failures:
+    - go: tip
+
+script:
+  - go get -t -v ./...
+  - diff -u <(echo -n) <(gofmt -d .)
+  - go vet $(go list ./... | grep -v /vendor/)
+  - go test -v -race ./...
diff --git a/vendor/github.com/gorilla/websocket/AUTHORS b/vendor/github.com/gorilla/websocket/AUTHORS
new file mode 100644
index 0000000..b003eca
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/AUTHORS
@@ -0,0 +1,8 @@
+# This is the official list of Gorilla WebSocket authors for copyright
+# purposes.
+#
+# Please keep the list sorted.
+
+Gary Burd <gary@beagledreams.com>
+Joachim Bauch <mail@joachim-bauch.de>
+
diff --git a/vendor/github.com/gorilla/websocket/LICENSE b/vendor/github.com/gorilla/websocket/LICENSE
new file mode 100644
index 0000000..9171c97
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/LICENSE
@@ -0,0 +1,22 @@
+Copyright (c) 2013 The Gorilla WebSocket Authors. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+  Redistributions of source code must retain the above copyright notice, this
+  list of conditions and the following disclaimer.
+
+  Redistributions in binary form must reproduce the above copyright notice,
+  this list of conditions and the following disclaimer in the documentation
+  and/or other materials provided with the distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE
+FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER
+CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
+OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/vendor/github.com/gorilla/websocket/README.md b/vendor/github.com/gorilla/websocket/README.md
new file mode 100644
index 0000000..33c3d2b
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/README.md
@@ -0,0 +1,64 @@
+# Gorilla WebSocket
+
+Gorilla WebSocket is a [Go](http://golang.org/) implementation of the
+[WebSocket](http://www.rfc-editor.org/rfc/rfc6455.txt) protocol.
+
+[![Build Status](https://travis-ci.org/gorilla/websocket.svg?branch=master)](https://travis-ci.org/gorilla/websocket)
+[![GoDoc](https://godoc.org/github.com/gorilla/websocket?status.svg)](https://godoc.org/github.com/gorilla/websocket)
+
+### Documentation
+
+* [API Reference](http://godoc.org/github.com/gorilla/websocket)
+* [Chat example](https://github.com/gorilla/websocket/tree/master/examples/chat)
+* [Command example](https://github.com/gorilla/websocket/tree/master/examples/command)
+* [Client and server example](https://github.com/gorilla/websocket/tree/master/examples/echo)
+* [File watch example](https://github.com/gorilla/websocket/tree/master/examples/filewatch)
+
+### Status
+
+The Gorilla WebSocket package provides a complete and tested implementation of
+the [WebSocket](http://www.rfc-editor.org/rfc/rfc6455.txt) protocol. The
+package API is stable.
+
+### Installation
+
+    go get github.com/gorilla/websocket
+
+### Protocol Compliance
+
+The Gorilla WebSocket package passes the server tests in the [Autobahn Test
+Suite](http://autobahn.ws/testsuite) using the application in the [examples/autobahn
+subdirectory](https://github.com/gorilla/websocket/tree/master/examples/autobahn).
+
+### Gorilla WebSocket compared with other packages
+
+<table>
+<tr>
+<th></th>
+<th><a href="http://godoc.org/github.com/gorilla/websocket">github.com/gorilla</a></th>
+<th><a href="http://godoc.org/golang.org/x/net/websocket">golang.org/x/net</a></th>
+</tr>
+<tr>
+<tr><td colspan="3"><a href="http://tools.ietf.org/html/rfc6455">RFC 6455</a> Features</td></tr>
+<tr><td>Passes <a href="http://autobahn.ws/testsuite/">Autobahn Test Suite</a></td><td><a href="https://github.com/gorilla/websocket/tree/master/examples/autobahn">Yes</a></td><td>No</td></tr>
+<tr><td>Receive <a href="https://tools.ietf.org/html/rfc6455#section-5.4">fragmented</a> message<td>Yes</td><td><a href="https://code.google.com/p/go/issues/detail?id=7632">No</a>, see note 1</td></tr>
+<tr><td>Send <a href="https://tools.ietf.org/html/rfc6455#section-5.5.1">close</a> message</td><td><a href="http://godoc.org/github.com/gorilla/websocket#hdr-Control_Messages">Yes</a></td><td><a href="https://code.google.com/p/go/issues/detail?id=4588">No</a></td></tr>
+<tr><td>Send <a href="https://tools.ietf.org/html/rfc6455#section-5.5.2">pings</a> and receive <a href="https://tools.ietf.org/html/rfc6455#section-5.5.3">pongs</a></td><td><a href="http://godoc.org/github.com/gorilla/websocket#hdr-Control_Messages">Yes</a></td><td>No</td></tr>
+<tr><td>Get the <a href="https://tools.ietf.org/html/rfc6455#section-5.6">type</a> of a received data message</td><td>Yes</td><td>Yes, see note 2</td></tr>
+<tr><td colspan="3">Other Features</tr></td>
+<tr><td><a href="https://tools.ietf.org/html/rfc7692">Compression Extensions</a></td><td>Experimental</td><td>No</td></tr>
+<tr><td>Read message using io.Reader</td><td><a href="http://godoc.org/github.com/gorilla/websocket#Conn.NextReader">Yes</a></td><td>No, see note 3</td></tr>
+<tr><td>Write message using io.WriteCloser</td><td><a href="http://godoc.org/github.com/gorilla/websocket#Conn.NextWriter">Yes</a></td><td>No, see note 3</td></tr>
+</table>
+
+Notes: 
+
+1. Large messages are fragmented in [Chrome's new WebSocket implementation](http://www.ietf.org/mail-archive/web/hybi/current/msg10503.html).
+2. The application can get the type of a received data message by implementing
+   a [Codec marshal](http://godoc.org/golang.org/x/net/websocket#Codec.Marshal)
+   function.
+3. The go.net io.Reader and io.Writer operate across WebSocket frame boundaries.
+  Read returns when the input buffer is full or a frame boundary is
+  encountered. Each call to Write sends a single frame message. The Gorilla
+  io.Reader and io.WriteCloser operate on a single WebSocket message.
+
diff --git a/vendor/github.com/gorilla/websocket/client.go b/vendor/github.com/gorilla/websocket/client.go
new file mode 100644
index 0000000..43a87c7
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/client.go
@@ -0,0 +1,392 @@
+// Copyright 2013 The Gorilla WebSocket Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package websocket
+
+import (
+	"bufio"
+	"bytes"
+	"crypto/tls"
+	"encoding/base64"
+	"errors"
+	"io"
+	"io/ioutil"
+	"net"
+	"net/http"
+	"net/url"
+	"strings"
+	"time"
+)
+
+// ErrBadHandshake is returned when the server response to opening handshake is
+// invalid.
+var ErrBadHandshake = errors.New("websocket: bad handshake")
+
+var errInvalidCompression = errors.New("websocket: invalid compression negotiation")
+
+// NewClient creates a new client connection using the given net connection.
+// The URL u specifies the host and request URI. Use requestHeader to specify
+// the origin (Origin), subprotocols (Sec-WebSocket-Protocol) and cookies
+// (Cookie). Use the response.Header to get the selected subprotocol
+// (Sec-WebSocket-Protocol) and cookies (Set-Cookie).
+//
+// If the WebSocket handshake fails, ErrBadHandshake is returned along with a
+// non-nil *http.Response so that callers can handle redirects, authentication,
+// etc.
+//
+// Deprecated: Use Dialer instead.
+func NewClient(netConn net.Conn, u *url.URL, requestHeader http.Header, readBufSize, writeBufSize int) (c *Conn, response *http.Response, err error) {
+	d := Dialer{
+		ReadBufferSize:  readBufSize,
+		WriteBufferSize: writeBufSize,
+		NetDial: func(net, addr string) (net.Conn, error) {
+			return netConn, nil
+		},
+	}
+	return d.Dial(u.String(), requestHeader)
+}
+
+// A Dialer contains options for connecting to WebSocket server.
+type Dialer struct {
+	// NetDial specifies the dial function for creating TCP connections. If
+	// NetDial is nil, net.Dial is used.
+	NetDial func(network, addr string) (net.Conn, error)
+
+	// Proxy specifies a function to return a proxy for a given
+	// Request. If the function returns a non-nil error, the
+	// request is aborted with the provided error.
+	// If Proxy is nil or returns a nil *URL, no proxy is used.
+	Proxy func(*http.Request) (*url.URL, error)
+
+	// TLSClientConfig specifies the TLS configuration to use with tls.Client.
+	// If nil, the default configuration is used.
+	TLSClientConfig *tls.Config
+
+	// HandshakeTimeout specifies the duration for the handshake to complete.
+	HandshakeTimeout time.Duration
+
+	// ReadBufferSize and WriteBufferSize specify I/O buffer sizes. If a buffer
+	// size is zero, then a useful default size is used. The I/O buffer sizes
+	// do not limit the size of the messages that can be sent or received.
+	ReadBufferSize, WriteBufferSize int
+
+	// Subprotocols specifies the client's requested subprotocols.
+	Subprotocols []string
+
+	// EnableCompression specifies if the client should attempt to negotiate
+	// per message compression (RFC 7692). Setting this value to true does not
+	// guarantee that compression will be supported. Currently only "no context
+	// takeover" modes are supported.
+	EnableCompression bool
+
+	// Jar specifies the cookie jar.
+	// If Jar is nil, cookies are not sent in requests and ignored
+	// in responses.
+	Jar http.CookieJar
+}
+
+var errMalformedURL = errors.New("malformed ws or wss URL")
+
+// parseURL parses the URL.
+//
+// This function is a replacement for the standard library url.Parse function.
+// In Go 1.4 and earlier, url.Parse loses information from the path.
+func parseURL(s string) (*url.URL, error) {
+	// From the RFC:
+	//
+	// ws-URI = "ws:" "//" host [ ":" port ] path [ "?" query ]
+	// wss-URI = "wss:" "//" host [ ":" port ] path [ "?" query ]
+	var u url.URL
+	switch {
+	case strings.HasPrefix(s, "ws://"):
+		u.Scheme = "ws"
+		s = s[len("ws://"):]
+	case strings.HasPrefix(s, "wss://"):
+		u.Scheme = "wss"
+		s = s[len("wss://"):]
+	default:
+		return nil, errMalformedURL
+	}
+
+	if i := strings.Index(s, "?"); i >= 0 {
+		u.RawQuery = s[i+1:]
+		s = s[:i]
+	}
+
+	if i := strings.Index(s, "/"); i >= 0 {
+		u.Opaque = s[i:]
+		s = s[:i]
+	} else {
+		u.Opaque = "/"
+	}
+
+	u.Host = s
+
+	if strings.Contains(u.Host, "@") {
+		// Don't bother parsing user information because user information is
+		// not allowed in websocket URIs.
+		return nil, errMalformedURL
+	}
+
+	return &u, nil
+}
+
+func hostPortNoPort(u *url.URL) (hostPort, hostNoPort string) {
+	hostPort = u.Host
+	hostNoPort = u.Host
+	if i := strings.LastIndex(u.Host, ":"); i > strings.LastIndex(u.Host, "]") {
+		hostNoPort = hostNoPort[:i]
+	} else {
+		switch u.Scheme {
+		case "wss":
+			hostPort += ":443"
+		case "https":
+			hostPort += ":443"
+		default:
+			hostPort += ":80"
+		}
+	}
+	return hostPort, hostNoPort
+}
+
+// DefaultDialer is a dialer with all fields set to the default zero values.
+var DefaultDialer = &Dialer{
+	Proxy: http.ProxyFromEnvironment,
+}
+
+// Dial creates a new client connection. Use requestHeader to specify the
+// origin (Origin), subprotocols (Sec-WebSocket-Protocol) and cookies (Cookie).
+// Use the response.Header to get the selected subprotocol
+// (Sec-WebSocket-Protocol) and cookies (Set-Cookie).
+//
+// If the WebSocket handshake fails, ErrBadHandshake is returned along with a
+// non-nil *http.Response so that callers can handle redirects, authentication,
+// etcetera. The response body may not contain the entire response and does not
+// need to be closed by the application.
+func (d *Dialer) Dial(urlStr string, requestHeader http.Header) (*Conn, *http.Response, error) {
+
+	if d == nil {
+		d = &Dialer{
+			Proxy: http.ProxyFromEnvironment,
+		}
+	}
+
+	challengeKey, err := generateChallengeKey()
+	if err != nil {
+		return nil, nil, err
+	}
+
+	u, err := parseURL(urlStr)
+	if err != nil {
+		return nil, nil, err
+	}
+
+	switch u.Scheme {
+	case "ws":
+		u.Scheme = "http"
+	case "wss":
+		u.Scheme = "https"
+	default:
+		return nil, nil, errMalformedURL
+	}
+
+	if u.User != nil {
+		// User name and password are not allowed in websocket URIs.
+		return nil, nil, errMalformedURL
+	}
+
+	req := &http.Request{
+		Method:     "GET",
+		URL:        u,
+		Proto:      "HTTP/1.1",
+		ProtoMajor: 1,
+		ProtoMinor: 1,
+		Header:     make(http.Header),
+		Host:       u.Host,
+	}
+
+	// Set the cookies present in the cookie jar of the dialer
+	if d.Jar != nil {
+		for _, cookie := range d.Jar.Cookies(u) {
+			req.AddCookie(cookie)
+		}
+	}
+
+	// Set the request headers using the capitalization for names and values in
+	// RFC examples. Although the capitalization shouldn't matter, there are
+	// servers that depend on it. The Header.Set method is not used because the
+	// method canonicalizes the header names.
+	req.Header["Upgrade"] = []string{"websocket"}
+	req.Header["Connection"] = []string{"Upgrade"}
+	req.Header["Sec-WebSocket-Key"] = []string{challengeKey}
+	req.Header["Sec-WebSocket-Version"] = []string{"13"}
+	if len(d.Subprotocols) > 0 {
+		req.Header["Sec-WebSocket-Protocol"] = []string{strings.Join(d.Subprotocols, ", ")}
+	}
+	for k, vs := range requestHeader {
+		switch {
+		case k == "Host":
+			if len(vs) > 0 {
+				req.Host = vs[0]
+			}
+		case k == "Upgrade" ||
+			k == "Connection" ||
+			k == "Sec-Websocket-Key" ||
+			k == "Sec-Websocket-Version" ||
+			k == "Sec-Websocket-Extensions" ||
+			(k == "Sec-Websocket-Protocol" && len(d.Subprotocols) > 0):
+			return nil, nil, errors.New("websocket: duplicate header not allowed: " + k)
+		default:
+			req.Header[k] = vs
+		}
+	}
+
+	if d.EnableCompression {
+		req.Header.Set("Sec-Websocket-Extensions", "permessage-deflate; server_no_context_takeover; client_no_context_takeover")
+	}
+
+	hostPort, hostNoPort := hostPortNoPort(u)
+
+	var proxyURL *url.URL
+	// Check wether the proxy method has been configured
+	if d.Proxy != nil {
+		proxyURL, err = d.Proxy(req)
+	}
+	if err != nil {
+		return nil, nil, err
+	}
+
+	var targetHostPort string
+	if proxyURL != nil {
+		targetHostPort, _ = hostPortNoPort(proxyURL)
+	} else {
+		targetHostPort = hostPort
+	}
+
+	var deadline time.Time
+	if d.HandshakeTimeout != 0 {
+		deadline = time.Now().Add(d.HandshakeTimeout)
+	}
+
+	netDial := d.NetDial
+	if netDial == nil {
+		netDialer := &net.Dialer{Deadline: deadline}
+		netDial = netDialer.Dial
+	}
+
+	netConn, err := netDial("tcp", targetHostPort)
+	if err != nil {
+		return nil, nil, err
+	}
+
+	defer func() {
+		if netConn != nil {
+			netConn.Close()
+		}
+	}()
+
+	if err := netConn.SetDeadline(deadline); err != nil {
+		return nil, nil, err
+	}
+
+	if proxyURL != nil {
+		connectHeader := make(http.Header)
+		if user := proxyURL.User; user != nil {
+			proxyUser := user.Username()
+			if proxyPassword, passwordSet := user.Password(); passwordSet {
+				credential := base64.StdEncoding.EncodeToString([]byte(proxyUser + ":" + proxyPassword))
+				connectHeader.Set("Proxy-Authorization", "Basic "+credential)
+			}
+		}
+		connectReq := &http.Request{
+			Method: "CONNECT",
+			URL:    &url.URL{Opaque: hostPort},
+			Host:   hostPort,
+			Header: connectHeader,
+		}
+
+		connectReq.Write(netConn)
+
+		// Read response.
+		// Okay to use and discard buffered reader here, because
+		// TLS server will not speak until spoken to.
+		br := bufio.NewReader(netConn)
+		resp, err := http.ReadResponse(br, connectReq)
+		if err != nil {
+			return nil, nil, err
+		}
+		if resp.StatusCode != 200 {
+			f := strings.SplitN(resp.Status, " ", 2)
+			return nil, nil, errors.New(f[1])
+		}
+	}
+
+	if u.Scheme == "https" {
+		cfg := cloneTLSConfig(d.TLSClientConfig)
+		if cfg.ServerName == "" {
+			cfg.ServerName = hostNoPort
+		}
+		tlsConn := tls.Client(netConn, cfg)
+		netConn = tlsConn
+		if err := tlsConn.Handshake(); err != nil {
+			return nil, nil, err
+		}
+		if !cfg.InsecureSkipVerify {
+			if err := tlsConn.VerifyHostname(cfg.ServerName); err != nil {
+				return nil, nil, err
+			}
+		}
+	}
+
+	conn := newConn(netConn, false, d.ReadBufferSize, d.WriteBufferSize)
+
+	if err := req.Write(netConn); err != nil {
+		return nil, nil, err
+	}
+
+	resp, err := http.ReadResponse(conn.br, req)
+	if err != nil {
+		return nil, nil, err
+	}
+
+	if d.Jar != nil {
+		if rc := resp.Cookies(); len(rc) > 0 {
+			d.Jar.SetCookies(u, rc)
+		}
+	}
+
+	if resp.StatusCode != 101 ||
+		!strings.EqualFold(resp.Header.Get("Upgrade"), "websocket") ||
+		!strings.EqualFold(resp.Header.Get("Connection"), "upgrade") ||
+		resp.Header.Get("Sec-Websocket-Accept") != computeAcceptKey(challengeKey) {
+		// Before closing the network connection on return from this
+		// function, slurp up some of the response to aid application
+		// debugging.
+		buf := make([]byte, 1024)
+		n, _ := io.ReadFull(resp.Body, buf)
+		resp.Body = ioutil.NopCloser(bytes.NewReader(buf[:n]))
+		return nil, resp, ErrBadHandshake
+	}
+
+	for _, ext := range parseExtensions(resp.Header) {
+		if ext[""] != "permessage-deflate" {
+			continue
+		}
+		_, snct := ext["server_no_context_takeover"]
+		_, cnct := ext["client_no_context_takeover"]
+		if !snct || !cnct {
+			return nil, resp, errInvalidCompression
+		}
+		conn.newCompressionWriter = compressNoContextTakeover
+		conn.newDecompressionReader = decompressNoContextTakeover
+		break
+	}
+
+	resp.Body = ioutil.NopCloser(bytes.NewReader([]byte{}))
+	conn.subprotocol = resp.Header.Get("Sec-Websocket-Protocol")
+
+	netConn.SetDeadline(time.Time{})
+	netConn = nil // to avoid close in defer.
+	return conn, resp, nil
+}
diff --git a/vendor/github.com/gorilla/websocket/client_clone.go b/vendor/github.com/gorilla/websocket/client_clone.go
new file mode 100644
index 0000000..4f0d943
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/client_clone.go
@@ -0,0 +1,16 @@
+// Copyright 2013 The Gorilla WebSocket Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build go1.8
+
+package websocket
+
+import "crypto/tls"
+
+func cloneTLSConfig(cfg *tls.Config) *tls.Config {
+	if cfg == nil {
+		return &tls.Config{}
+	}
+	return cfg.Clone()
+}
diff --git a/vendor/github.com/gorilla/websocket/client_clone_legacy.go b/vendor/github.com/gorilla/websocket/client_clone_legacy.go
new file mode 100644
index 0000000..babb007
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/client_clone_legacy.go
@@ -0,0 +1,38 @@
+// Copyright 2013 The Gorilla WebSocket Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build !go1.8
+
+package websocket
+
+import "crypto/tls"
+
+// cloneTLSConfig clones all public fields except the fields
+// SessionTicketsDisabled and SessionTicketKey. This avoids copying the
+// sync.Mutex in the sync.Once and makes it safe to call cloneTLSConfig on a
+// config in active use.
+func cloneTLSConfig(cfg *tls.Config) *tls.Config {
+	if cfg == nil {
+		return &tls.Config{}
+	}
+	return &tls.Config{
+		Rand:                     cfg.Rand,
+		Time:                     cfg.Time,
+		Certificates:             cfg.Certificates,
+		NameToCertificate:        cfg.NameToCertificate,
+		GetCertificate:           cfg.GetCertificate,
+		RootCAs:                  cfg.RootCAs,
+		NextProtos:               cfg.NextProtos,
+		ServerName:               cfg.ServerName,
+		ClientAuth:               cfg.ClientAuth,
+		ClientCAs:                cfg.ClientCAs,
+		InsecureSkipVerify:       cfg.InsecureSkipVerify,
+		CipherSuites:             cfg.CipherSuites,
+		PreferServerCipherSuites: cfg.PreferServerCipherSuites,
+		ClientSessionCache:       cfg.ClientSessionCache,
+		MinVersion:               cfg.MinVersion,
+		MaxVersion:               cfg.MaxVersion,
+		CurvePreferences:         cfg.CurvePreferences,
+	}
+}
diff --git a/vendor/github.com/gorilla/websocket/compression.go b/vendor/github.com/gorilla/websocket/compression.go
new file mode 100644
index 0000000..813ffb1
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/compression.go
@@ -0,0 +1,148 @@
+// Copyright 2017 The Gorilla WebSocket Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package websocket
+
+import (
+	"compress/flate"
+	"errors"
+	"io"
+	"strings"
+	"sync"
+)
+
+const (
+	minCompressionLevel     = -2 // flate.HuffmanOnly not defined in Go < 1.6
+	maxCompressionLevel     = flate.BestCompression
+	defaultCompressionLevel = 1
+)
+
+var (
+	flateWriterPools [maxCompressionLevel - minCompressionLevel + 1]sync.Pool
+	flateReaderPool  = sync.Pool{New: func() interface{} {
+		return flate.NewReader(nil)
+	}}
+)
+
+func decompressNoContextTakeover(r io.Reader) io.ReadCloser {
+	const tail =
+	// Add four bytes as specified in RFC
+	"\x00\x00\xff\xff" +
+		// Add final block to squelch unexpected EOF error from flate reader.
+		"\x01\x00\x00\xff\xff"
+
+	fr, _ := flateReaderPool.Get().(io.ReadCloser)
+	fr.(flate.Resetter).Reset(io.MultiReader(r, strings.NewReader(tail)), nil)
+	return &flateReadWrapper{fr}
+}
+
+func isValidCompressionLevel(level int) bool {
+	return minCompressionLevel <= level && level <= maxCompressionLevel
+}
+
+func compressNoContextTakeover(w io.WriteCloser, level int) io.WriteCloser {
+	p := &flateWriterPools[level-minCompressionLevel]
+	tw := &truncWriter{w: w}
+	fw, _ := p.Get().(*flate.Writer)
+	if fw == nil {
+		fw, _ = flate.NewWriter(tw, level)
+	} else {
+		fw.Reset(tw)
+	}
+	return &flateWriteWrapper{fw: fw, tw: tw, p: p}
+}
+
+// truncWriter is an io.Writer that writes all but the last four bytes of the
+// stream to another io.Writer.
+type truncWriter struct {
+	w io.WriteCloser
+	n int
+	p [4]byte
+}
+
+func (w *truncWriter) Write(p []byte) (int, error) {
+	n := 0
+
+	// fill buffer first for simplicity.
+	if w.n < len(w.p) {
+		n = copy(w.p[w.n:], p)
+		p = p[n:]
+		w.n += n
+		if len(p) == 0 {
+			return n, nil
+		}
+	}
+
+	m := len(p)
+	if m > len(w.p) {
+		m = len(w.p)
+	}
+
+	if nn, err := w.w.Write(w.p[:m]); err != nil {
+		return n + nn, err
+	}
+
+	copy(w.p[:], w.p[m:])
+	copy(w.p[len(w.p)-m:], p[len(p)-m:])
+	nn, err := w.w.Write(p[:len(p)-m])
+	return n + nn, err
+}
+
+type flateWriteWrapper struct {
+	fw *flate.Writer
+	tw *truncWriter
+	p  *sync.Pool
+}
+
+func (w *flateWriteWrapper) Write(p []byte) (int, error) {
+	if w.fw == nil {
+		return 0, errWriteClosed
+	}
+	return w.fw.Write(p)
+}
+
+func (w *flateWriteWrapper) Close() error {
+	if w.fw == nil {
+		return errWriteClosed
+	}
+	err1 := w.fw.Flush()
+	w.p.Put(w.fw)
+	w.fw = nil
+	if w.tw.p != [4]byte{0, 0, 0xff, 0xff} {
+		return errors.New("websocket: internal error, unexpected bytes at end of flate stream")
+	}
+	err2 := w.tw.w.Close()
+	if err1 != nil {
+		return err1
+	}
+	return err2
+}
+
+type flateReadWrapper struct {
+	fr io.ReadCloser
+}
+
+func (r *flateReadWrapper) Read(p []byte) (int, error) {
+	if r.fr == nil {
+		return 0, io.ErrClosedPipe
+	}
+	n, err := r.fr.Read(p)
+	if err == io.EOF {
+		// Preemptively place the reader back in the pool. This helps with
+		// scenarios where the application does not call NextReader() soon after
+		// this final read.
+		r.Close()
+	}
+	return n, err
+}
+
+func (r *flateReadWrapper) Close() error {
+	if r.fr == nil {
+		return io.ErrClosedPipe
+	}
+	err := r.fr.Close()
+	flateReaderPool.Put(r.fr)
+	r.fr = nil
+	return err
+}
diff --git a/vendor/github.com/gorilla/websocket/conn.go b/vendor/github.com/gorilla/websocket/conn.go
new file mode 100644
index 0000000..97e1dba
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/conn.go
@@ -0,0 +1,1149 @@
+// Copyright 2013 The Gorilla WebSocket Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package websocket
+
+import (
+	"bufio"
+	"encoding/binary"
+	"errors"
+	"io"
+	"io/ioutil"
+	"math/rand"
+	"net"
+	"strconv"
+	"sync"
+	"time"
+	"unicode/utf8"
+)
+
+const (
+	// Frame header byte 0 bits from Section 5.2 of RFC 6455
+	finalBit = 1 << 7
+	rsv1Bit  = 1 << 6
+	rsv2Bit  = 1 << 5
+	rsv3Bit  = 1 << 4
+
+	// Frame header byte 1 bits from Section 5.2 of RFC 6455
+	maskBit = 1 << 7
+
+	maxFrameHeaderSize         = 2 + 8 + 4 // Fixed header + length + mask
+	maxControlFramePayloadSize = 125
+
+	writeWait = time.Second
+
+	defaultReadBufferSize  = 4096
+	defaultWriteBufferSize = 4096
+
+	continuationFrame = 0
+	noFrame           = -1
+)
+
+// Close codes defined in RFC 6455, section 11.7.
+const (
+	CloseNormalClosure           = 1000
+	CloseGoingAway               = 1001
+	CloseProtocolError           = 1002
+	CloseUnsupportedData         = 1003
+	CloseNoStatusReceived        = 1005
+	CloseAbnormalClosure         = 1006
+	CloseInvalidFramePayloadData = 1007
+	ClosePolicyViolation         = 1008
+	CloseMessageTooBig           = 1009
+	CloseMandatoryExtension      = 1010
+	CloseInternalServerErr       = 1011
+	CloseServiceRestart          = 1012
+	CloseTryAgainLater           = 1013
+	CloseTLSHandshake            = 1015
+)
+
+// The message types are defined in RFC 6455, section 11.8.
+const (
+	// TextMessage denotes a text data message. The text message payload is
+	// interpreted as UTF-8 encoded text data.
+	TextMessage = 1
+
+	// BinaryMessage denotes a binary data message.
+	BinaryMessage = 2
+
+	// CloseMessage denotes a close control message. The optional message
+	// payload contains a numeric code and text. Use the FormatCloseMessage
+	// function to format a close message payload.
+	CloseMessage = 8
+
+	// PingMessage denotes a ping control message. The optional message payload
+	// is UTF-8 encoded text.
+	PingMessage = 9
+
+	// PongMessage denotes a ping control message. The optional message payload
+	// is UTF-8 encoded text.
+	PongMessage = 10
+)
+
+// ErrCloseSent is returned when the application writes a message to the
+// connection after sending a close message.
+var ErrCloseSent = errors.New("websocket: close sent")
+
+// ErrReadLimit is returned when reading a message that is larger than the
+// read limit set for the connection.
+var ErrReadLimit = errors.New("websocket: read limit exceeded")
+
+// netError satisfies the net Error interface.
+type netError struct {
+	msg       string
+	temporary bool
+	timeout   bool
+}
+
+func (e *netError) Error() string   { return e.msg }
+func (e *netError) Temporary() bool { return e.temporary }
+func (e *netError) Timeout() bool   { return e.timeout }
+
+// CloseError represents close frame.
+type CloseError struct {
+
+	// Code is defined in RFC 6455, section 11.7.
+	Code int
+
+	// Text is the optional text payload.
+	Text string
+}
+
+func (e *CloseError) Error() string {
+	s := []byte("websocket: close ")
+	s = strconv.AppendInt(s, int64(e.Code), 10)
+	switch e.Code {
+	case CloseNormalClosure:
+		s = append(s, " (normal)"...)
+	case CloseGoingAway:
+		s = append(s, " (going away)"...)
+	case CloseProtocolError:
+		s = append(s, " (protocol error)"...)
+	case CloseUnsupportedData:
+		s = append(s, " (unsupported data)"...)
+	case CloseNoStatusReceived:
+		s = append(s, " (no status)"...)
+	case CloseAbnormalClosure:
+		s = append(s, " (abnormal closure)"...)
+	case CloseInvalidFramePayloadData:
+		s = append(s, " (invalid payload data)"...)
+	case ClosePolicyViolation:
+		s = append(s, " (policy violation)"...)
+	case CloseMessageTooBig:
+		s = append(s, " (message too big)"...)
+	case CloseMandatoryExtension:
+		s = append(s, " (mandatory extension missing)"...)
+	case CloseInternalServerErr:
+		s = append(s, " (internal server error)"...)
+	case CloseTLSHandshake:
+		s = append(s, " (TLS handshake error)"...)
+	}
+	if e.Text != "" {
+		s = append(s, ": "...)
+		s = append(s, e.Text...)
+	}
+	return string(s)
+}
+
+// IsCloseError returns boolean indicating whether the error is a *CloseError
+// with one of the specified codes.
+func IsCloseError(err error, codes ...int) bool {
+	if e, ok := err.(*CloseError); ok {
+		for _, code := range codes {
+			if e.Code == code {
+				return true
+			}
+		}
+	}
+	return false
+}
+
+// IsUnexpectedCloseError returns boolean indicating whether the error is a
+// *CloseError with a code not in the list of expected codes.
+func IsUnexpectedCloseError(err error, expectedCodes ...int) bool {
+	if e, ok := err.(*CloseError); ok {
+		for _, code := range expectedCodes {
+			if e.Code == code {
+				return false
+			}
+		}
+		return true
+	}
+	return false
+}
+
+var (
+	errWriteTimeout        = &netError{msg: "websocket: write timeout", timeout: true, temporary: true}
+	errUnexpectedEOF       = &CloseError{Code: CloseAbnormalClosure, Text: io.ErrUnexpectedEOF.Error()}
+	errBadWriteOpCode      = errors.New("websocket: bad write message type")
+	errWriteClosed         = errors.New("websocket: write closed")
+	errInvalidControlFrame = errors.New("websocket: invalid control frame")
+)
+
+func newMaskKey() [4]byte {
+	n := rand.Uint32()
+	return [4]byte{byte(n), byte(n >> 8), byte(n >> 16), byte(n >> 24)}
+}
+
+func hideTempErr(err error) error {
+	if e, ok := err.(net.Error); ok && e.Temporary() {
+		err = &netError{msg: e.Error(), timeout: e.Timeout()}
+	}
+	return err
+}
+
+func isControl(frameType int) bool {
+	return frameType == CloseMessage || frameType == PingMessage || frameType == PongMessage
+}
+
+func isData(frameType int) bool {
+	return frameType == TextMessage || frameType == BinaryMessage
+}
+
+var validReceivedCloseCodes = map[int]bool{
+	// see http://www.iana.org/assignments/websocket/websocket.xhtml#close-code-number
+
+	CloseNormalClosure:           true,
+	CloseGoingAway:               true,
+	CloseProtocolError:           true,
+	CloseUnsupportedData:         true,
+	CloseNoStatusReceived:        false,
+	CloseAbnormalClosure:         false,
+	CloseInvalidFramePayloadData: true,
+	ClosePolicyViolation:         true,
+	CloseMessageTooBig:           true,
+	CloseMandatoryExtension:      true,
+	CloseInternalServerErr:       true,
+	CloseServiceRestart:          true,
+	CloseTryAgainLater:           true,
+	CloseTLSHandshake:            false,
+}
+
+func isValidReceivedCloseCode(code int) bool {
+	return validReceivedCloseCodes[code] || (code >= 3000 && code <= 4999)
+}
+
+// The Conn type represents a WebSocket connection.
+type Conn struct {
+	conn        net.Conn
+	isServer    bool
+	subprotocol string
+
+	// Write fields
+	mu            chan bool // used as mutex to protect write to conn
+	writeBuf      []byte    // frame is constructed in this buffer.
+	writeDeadline time.Time
+	writer        io.WriteCloser // the current writer returned to the application
+	isWriting     bool           // for best-effort concurrent write detection
+
+	writeErrMu sync.Mutex
+	writeErr   error
+
+	enableWriteCompression bool
+	compressionLevel       int
+	newCompressionWriter   func(io.WriteCloser, int) io.WriteCloser
+
+	// Read fields
+	reader        io.ReadCloser // the current reader returned to the application
+	readErr       error
+	br            *bufio.Reader
+	readRemaining int64 // bytes remaining in current frame.
+	readFinal     bool  // true the current message has more frames.
+	readLength    int64 // Message size.
+	readLimit     int64 // Maximum message size.
+	readMaskPos   int
+	readMaskKey   [4]byte
+	handlePong    func(string) error
+	handlePing    func(string) error
+	handleClose   func(int, string) error
+	readErrCount  int
+	messageReader *messageReader // the current low-level reader
+
+	readDecompress         bool // whether last read frame had RSV1 set
+	newDecompressionReader func(io.Reader) io.ReadCloser
+}
+
+func newConn(conn net.Conn, isServer bool, readBufferSize, writeBufferSize int) *Conn {
+	return newConnBRW(conn, isServer, readBufferSize, writeBufferSize, nil)
+}
+
+type writeHook struct {
+	p []byte
+}
+
+func (wh *writeHook) Write(p []byte) (int, error) {
+	wh.p = p
+	return len(p), nil
+}
+
+func newConnBRW(conn net.Conn, isServer bool, readBufferSize, writeBufferSize int, brw *bufio.ReadWriter) *Conn {
+	mu := make(chan bool, 1)
+	mu <- true
+
+	var br *bufio.Reader
+	if readBufferSize == 0 && brw != nil && brw.Reader != nil {
+		// Reuse the supplied bufio.Reader if the buffer has a useful size.
+		// This code assumes that peek on a reader returns
+		// bufio.Reader.buf[:0].
+		brw.Reader.Reset(conn)
+		if p, err := brw.Reader.Peek(0); err == nil && cap(p) >= 256 {
+			br = brw.Reader
+		}
+	}
+	if br == nil {
+		if readBufferSize == 0 {
+			readBufferSize = defaultReadBufferSize
+		}
+		if readBufferSize < maxControlFramePayloadSize {
+			readBufferSize = maxControlFramePayloadSize
+		}
+		br = bufio.NewReaderSize(conn, readBufferSize)
+	}
+
+	var writeBuf []byte
+	if writeBufferSize == 0 && brw != nil && brw.Writer != nil {
+		// Use the bufio.Writer's buffer if the buffer has a useful size. This
+		// code assumes that bufio.Writer.buf[:1] is passed to the
+		// bufio.Writer's underlying writer.
+		var wh writeHook
+		brw.Writer.Reset(&wh)
+		brw.Writer.WriteByte(0)
+		brw.Flush()
+		if cap(wh.p) >= maxFrameHeaderSize+256 {
+			writeBuf = wh.p[:cap(wh.p)]
+		}
+	}
+
+	if writeBuf == nil {
+		if writeBufferSize == 0 {
+			writeBufferSize = defaultWriteBufferSize
+		}
+		writeBuf = make([]byte, writeBufferSize+maxFrameHeaderSize)
+	}
+
+	c := &Conn{
+		isServer:               isServer,
+		br:                     br,
+		conn:                   conn,
+		mu:                     mu,
+		readFinal:              true,
+		writeBuf:               writeBuf,
+		enableWriteCompression: true,
+		compressionLevel:       defaultCompressionLevel,
+	}
+	c.SetCloseHandler(nil)
+	c.SetPingHandler(nil)
+	c.SetPongHandler(nil)
+	return c
+}
+
+// Subprotocol returns the negotiated protocol for the connection.
+func (c *Conn) Subprotocol() string {
+	return c.subprotocol
+}
+
+// Close closes the underlying network connection without sending or waiting for a close frame.
+func (c *Conn) Close() error {
+	return c.conn.Close()
+}
+
+// LocalAddr returns the local network address.
+func (c *Conn) LocalAddr() net.Addr {
+	return c.conn.LocalAddr()
+}
+
+// RemoteAddr returns the remote network address.
+func (c *Conn) RemoteAddr() net.Addr {
+	return c.conn.RemoteAddr()
+}
+
+// Write methods
+
+func (c *Conn) writeFatal(err error) error {
+	err = hideTempErr(err)
+	c.writeErrMu.Lock()
+	if c.writeErr == nil {
+		c.writeErr = err
+	}
+	c.writeErrMu.Unlock()
+	return err
+}
+
+func (c *Conn) write(frameType int, deadline time.Time, bufs ...[]byte) error {
+	<-c.mu
+	defer func() { c.mu <- true }()
+
+	c.writeErrMu.Lock()
+	err := c.writeErr
+	c.writeErrMu.Unlock()
+	if err != nil {
+		return err
+	}
+
+	c.conn.SetWriteDeadline(deadline)
+	for _, buf := range bufs {
+		if len(buf) > 0 {
+			_, err := c.conn.Write(buf)
+			if err != nil {
+				return c.writeFatal(err)
+			}
+		}
+	}
+
+	if frameType == CloseMessage {
+		c.writeFatal(ErrCloseSent)
+	}
+	return nil
+}
+
+// WriteControl writes a control message with the given deadline. The allowed
+// message types are CloseMessage, PingMessage and PongMessage.
+func (c *Conn) WriteControl(messageType int, data []byte, deadline time.Time) error {
+	if !isControl(messageType) {
+		return errBadWriteOpCode
+	}
+	if len(data) > maxControlFramePayloadSize {
+		return errInvalidControlFrame
+	}
+
+	b0 := byte(messageType) | finalBit
+	b1 := byte(len(data))
+	if !c.isServer {
+		b1 |= maskBit
+	}
+
+	buf := make([]byte, 0, maxFrameHeaderSize+maxControlFramePayloadSize)
+	buf = append(buf, b0, b1)
+
+	if c.isServer {
+		buf = append(buf, data...)
+	} else {
+		key := newMaskKey()
+		buf = append(buf, key[:]...)
+		buf = append(buf, data...)
+		maskBytes(key, 0, buf[6:])
+	}
+
+	d := time.Hour * 1000
+	if !deadline.IsZero() {
+		d = deadline.Sub(time.Now())
+		if d < 0 {
+			return errWriteTimeout
+		}
+	}
+
+	timer := time.NewTimer(d)
+	select {
+	case <-c.mu:
+		timer.Stop()
+	case <-timer.C:
+		return errWriteTimeout
+	}
+	defer func() { c.mu <- true }()
+
+	c.writeErrMu.Lock()
+	err := c.writeErr
+	c.writeErrMu.Unlock()
+	if err != nil {
+		return err
+	}
+
+	c.conn.SetWriteDeadline(deadline)
+	_, err = c.conn.Write(buf)
+	if err != nil {
+		return c.writeFatal(err)
+	}
+	if messageType == CloseMessage {
+		c.writeFatal(ErrCloseSent)
+	}
+	return err
+}
+
+func (c *Conn) prepWrite(messageType int) error {
+	// Close previous writer if not already closed by the application. It's
+	// probably better to return an error in this situation, but we cannot
+	// change this without breaking existing applications.
+	if c.writer != nil {
+		c.writer.Close()
+		c.writer = nil
+	}
+
+	if !isControl(messageType) && !isData(messageType) {
+		return errBadWriteOpCode
+	}
+
+	c.writeErrMu.Lock()
+	err := c.writeErr
+	c.writeErrMu.Unlock()
+	return err
+}
+
+// NextWriter returns a writer for the next message to send. The writer's Close
+// method flushes the complete message to the network.
+//
+// There can be at most one open writer on a connection. NextWriter closes the
+// previous writer if the application has not already done so.
+func (c *Conn) NextWriter(messageType int) (io.WriteCloser, error) {
+	if err := c.prepWrite(messageType); err != nil {
+		return nil, err
+	}
+
+	mw := &messageWriter{
+		c:         c,
+		frameType: messageType,
+		pos:       maxFrameHeaderSize,
+	}
+	c.writer = mw
+	if c.newCompressionWriter != nil && c.enableWriteCompression && isData(messageType) {
+		w := c.newCompressionWriter(c.writer, c.compressionLevel)
+		mw.compress = true
+		c.writer = w
+	}
+	return c.writer, nil
+}
+
+type messageWriter struct {
+	c         *Conn
+	compress  bool // whether next call to flushFrame should set RSV1
+	pos       int  // end of data in writeBuf.
+	frameType int  // type of the current frame.
+	err       error
+}
+
+func (w *messageWriter) fatal(err error) error {
+	if w.err != nil {
+		w.err = err
+		w.c.writer = nil
+	}
+	return err
+}
+
+// flushFrame writes buffered data and extra as a frame to the network. The
+// final argument indicates that this is the last frame in the message.
+func (w *messageWriter) flushFrame(final bool, extra []byte) error {
+	c := w.c
+	length := w.pos - maxFrameHeaderSize + len(extra)
+
+	// Check for invalid control frames.
+	if isControl(w.frameType) &&
+		(!final || length > maxControlFramePayloadSize) {
+		return w.fatal(errInvalidControlFrame)
+	}
+
+	b0 := byte(w.frameType)
+	if final {
+		b0 |= finalBit
+	}
+	if w.compress {
+		b0 |= rsv1Bit
+	}
+	w.compress = false
+
+	b1 := byte(0)
+	if !c.isServer {
+		b1 |= maskBit
+	}
+
+	// Assume that the frame starts at beginning of c.writeBuf.
+	framePos := 0
+	if c.isServer {
+		// Adjust up if mask not included in the header.
+		framePos = 4
+	}
+
+	switch {
+	case length >= 65536:
+		c.writeBuf[framePos] = b0
+		c.writeBuf[framePos+1] = b1 | 127
+		binary.BigEndian.PutUint64(c.writeBuf[framePos+2:], uint64(length))
+	case length > 125:
+		framePos += 6
+		c.writeBuf[framePos] = b0
+		c.writeBuf[framePos+1] = b1 | 126
+		binary.BigEndian.PutUint16(c.writeBuf[framePos+2:], uint16(length))
+	default:
+		framePos += 8
+		c.writeBuf[framePos] = b0
+		c.writeBuf[framePos+1] = b1 | byte(length)
+	}
+
+	if !c.isServer {
+		key := newMaskKey()
+		copy(c.writeBuf[maxFrameHeaderSize-4:], key[:])
+		maskBytes(key, 0, c.writeBuf[maxFrameHeaderSize:w.pos])
+		if len(extra) > 0 {
+			return c.writeFatal(errors.New("websocket: internal error, extra used in client mode"))
+		}
+	}
+
+	// Write the buffers to the connection with best-effort detection of
+	// concurrent writes. See the concurrency section in the package
+	// documentation for more info.
+
+	if c.isWriting {
+		panic("concurrent write to websocket connection")
+	}
+	c.isWriting = true
+
+	err := c.write(w.frameType, c.writeDeadline, c.writeBuf[framePos:w.pos], extra)
+
+	if !c.isWriting {
+		panic("concurrent write to websocket connection")
+	}
+	c.isWriting = false
+
+	if err != nil {
+		return w.fatal(err)
+	}
+
+	if final {
+		c.writer = nil
+		return nil
+	}
+
+	// Setup for next frame.
+	w.pos = maxFrameHeaderSize
+	w.frameType = continuationFrame
+	return nil
+}
+
+func (w *messageWriter) ncopy(max int) (int, error) {
+	n := len(w.c.writeBuf) - w.pos
+	if n <= 0 {
+		if err := w.flushFrame(false, nil); err != nil {
+			return 0, err
+		}
+		n = len(w.c.writeBuf) - w.pos
+	}
+	if n > max {
+		n = max
+	}
+	return n, nil
+}
+
+func (w *messageWriter) Write(p []byte) (int, error) {
+	if w.err != nil {
+		return 0, w.err
+	}
+
+	if len(p) > 2*len(w.c.writeBuf) && w.c.isServer {
+		// Don't buffer large messages.
+		err := w.flushFrame(false, p)
+		if err != nil {
+			return 0, err
+		}
+		return len(p), nil
+	}
+
+	nn := len(p)
+	for len(p) > 0 {
+		n, err := w.ncopy(len(p))
+		if err != nil {
+			return 0, err
+		}
+		copy(w.c.writeBuf[w.pos:], p[:n])
+		w.pos += n
+		p = p[n:]
+	}
+	return nn, nil
+}
+
+func (w *messageWriter) WriteString(p string) (int, error) {
+	if w.err != nil {
+		return 0, w.err
+	}
+
+	nn := len(p)
+	for len(p) > 0 {
+		n, err := w.ncopy(len(p))
+		if err != nil {
+			return 0, err
+		}
+		copy(w.c.writeBuf[w.pos:], p[:n])
+		w.pos += n
+		p = p[n:]
+	}
+	return nn, nil
+}
+
+func (w *messageWriter) ReadFrom(r io.Reader) (nn int64, err error) {
+	if w.err != nil {
+		return 0, w.err
+	}
+	for {
+		if w.pos == len(w.c.writeBuf) {
+			err = w.flushFrame(false, nil)
+			if err != nil {
+				break
+			}
+		}
+		var n int
+		n, err = r.Read(w.c.writeBuf[w.pos:])
+		w.pos += n
+		nn += int64(n)
+		if err != nil {
+			if err == io.EOF {
+				err = nil
+			}
+			break
+		}
+	}
+	return nn, err
+}
+
+func (w *messageWriter) Close() error {
+	if w.err != nil {
+		return w.err
+	}
+	if err := w.flushFrame(true, nil); err != nil {
+		return err
+	}
+	w.err = errWriteClosed
+	return nil
+}
+
+// WritePreparedMessage writes prepared message into connection.
+func (c *Conn) WritePreparedMessage(pm *PreparedMessage) error {
+	frameType, frameData, err := pm.frame(prepareKey{
+		isServer:         c.isServer,
+		compress:         c.newCompressionWriter != nil && c.enableWriteCompression && isData(pm.messageType),
+		compressionLevel: c.compressionLevel,
+	})
+	if err != nil {
+		return err
+	}
+	if c.isWriting {
+		panic("concurrent write to websocket connection")
+	}
+	c.isWriting = true
+	err = c.write(frameType, c.writeDeadline, frameData, nil)
+	if !c.isWriting {
+		panic("concurrent write to websocket connection")
+	}
+	c.isWriting = false
+	return err
+}
+
+// WriteMessage is a helper method for getting a writer using NextWriter,
+// writing the message and closing the writer.
+func (c *Conn) WriteMessage(messageType int, data []byte) error {
+
+	if c.isServer && (c.newCompressionWriter == nil || !c.enableWriteCompression) {
+		// Fast path with no allocations and single frame.
+
+		if err := c.prepWrite(messageType); err != nil {
+			return err
+		}
+		mw := messageWriter{c: c, frameType: messageType, pos: maxFrameHeaderSize}
+		n := copy(c.writeBuf[mw.pos:], data)
+		mw.pos += n
+		data = data[n:]
+		return mw.flushFrame(true, data)
+	}
+
+	w, err := c.NextWriter(messageType)
+	if err != nil {
+		return err
+	}
+	if _, err = w.Write(data); err != nil {
+		return err
+	}
+	return w.Close()
+}
+
+// SetWriteDeadline sets the write deadline on the underlying network
+// connection. After a write has timed out, the websocket state is corrupt and
+// all future writes will return an error. A zero value for t means writes will
+// not time out.
+func (c *Conn) SetWriteDeadline(t time.Time) error {
+	c.writeDeadline = t
+	return nil
+}
+
+// Read methods
+
+func (c *Conn) advanceFrame() (int, error) {
+
+	// 1. Skip remainder of previous frame.
+
+	if c.readRemaining > 0 {
+		if _, err := io.CopyN(ioutil.Discard, c.br, c.readRemaining); err != nil {
+			return noFrame, err
+		}
+	}
+
+	// 2. Read and parse first two bytes of frame header.
+
+	p, err := c.read(2)
+	if err != nil {
+		return noFrame, err
+	}
+
+	final := p[0]&finalBit != 0
+	frameType := int(p[0] & 0xf)
+	mask := p[1]&maskBit != 0
+	c.readRemaining = int64(p[1] & 0x7f)
+
+	c.readDecompress = false
+	if c.newDecompressionReader != nil && (p[0]&rsv1Bit) != 0 {
+		c.readDecompress = true
+		p[0] &^= rsv1Bit
+	}
+
+	if rsv := p[0] & (rsv1Bit | rsv2Bit | rsv3Bit); rsv != 0 {
+		return noFrame, c.handleProtocolError("unexpected reserved bits 0x" + strconv.FormatInt(int64(rsv), 16))
+	}
+
+	switch frameType {
+	case CloseMessage, PingMessage, PongMessage:
+		if c.readRemaining > maxControlFramePayloadSize {
+			return noFrame, c.handleProtocolError("control frame length > 125")
+		}
+		if !final {
+			return noFrame, c.handleProtocolError("control frame not final")
+		}
+	case TextMessage, BinaryMessage:
+		if !c.readFinal {
+			return noFrame, c.handleProtocolError("message start before final message frame")
+		}
+		c.readFinal = final
+	case continuationFrame:
+		if c.readFinal {
+			return noFrame, c.handleProtocolError("continuation after final message frame")
+		}
+		c.readFinal = final
+	default:
+		return noFrame, c.handleProtocolError("unknown opcode " + strconv.Itoa(frameType))
+	}
+
+	// 3. Read and parse frame length.
+
+	switch c.readRemaining {
+	case 126:
+		p, err := c.read(2)
+		if err != nil {
+			return noFrame, err
+		}
+		c.readRemaining = int64(binary.BigEndian.Uint16(p))
+	case 127:
+		p, err := c.read(8)
+		if err != nil {
+			return noFrame, err
+		}
+		c.readRemaining = int64(binary.BigEndian.Uint64(p))
+	}
+
+	// 4. Handle frame masking.
+
+	if mask != c.isServer {
+		return noFrame, c.handleProtocolError("incorrect mask flag")
+	}
+
+	if mask {
+		c.readMaskPos = 0
+		p, err := c.read(len(c.readMaskKey))
+		if err != nil {
+			return noFrame, err
+		}
+		copy(c.readMaskKey[:], p)
+	}
+
+	// 5. For text and binary messages, enforce read limit and return.
+
+	if frameType == continuationFrame || frameType == TextMessage || frameType == BinaryMessage {
+
+		c.readLength += c.readRemaining
+		if c.readLimit > 0 && c.readLength > c.readLimit {
+			c.WriteControl(CloseMessage, FormatCloseMessage(CloseMessageTooBig, ""), time.Now().Add(writeWait))
+			return noFrame, ErrReadLimit
+		}
+
+		return frameType, nil
+	}
+
+	// 6. Read control frame payload.
+
+	var payload []byte
+	if c.readRemaining > 0 {
+		payload, err = c.read(int(c.readRemaining))
+		c.readRemaining = 0
+		if err != nil {
+			return noFrame, err
+		}
+		if c.isServer {
+			maskBytes(c.readMaskKey, 0, payload)
+		}
+	}
+
+	// 7. Process control frame payload.
+
+	switch frameType {
+	case PongMessage:
+		if err := c.handlePong(string(payload)); err != nil {
+			return noFrame, err
+		}
+	case PingMessage:
+		if err := c.handlePing(string(payload)); err != nil {
+			return noFrame, err
+		}
+	case CloseMessage:
+		closeCode := CloseNoStatusReceived
+		closeText := ""
+		if len(payload) >= 2 {
+			closeCode = int(binary.BigEndian.Uint16(payload))
+			if !isValidReceivedCloseCode(closeCode) {
+				return noFrame, c.handleProtocolError("invalid close code")
+			}
+			closeText = string(payload[2:])
+			if !utf8.ValidString(closeText) {
+				return noFrame, c.handleProtocolError("invalid utf8 payload in close frame")
+			}
+		}
+		if err := c.handleClose(closeCode, closeText); err != nil {
+			return noFrame, err
+		}
+		return noFrame, &CloseError{Code: closeCode, Text: closeText}
+	}
+
+	return frameType, nil
+}
+
+func (c *Conn) handleProtocolError(message string) error {
+	c.WriteControl(CloseMessage, FormatCloseMessage(CloseProtocolError, message), time.Now().Add(writeWait))
+	return errors.New("websocket: " + message)
+}
+
+// NextReader returns the next data message received from the peer. The
+// returned messageType is either TextMessage or BinaryMessage.
+//
+// There can be at most one open reader on a connection. NextReader discards
+// the previous message if the application has not already consumed it.
+//
+// Applications must break out of the application's read loop when this method
+// returns a non-nil error value. Errors returned from this method are
+// permanent. Once this method returns a non-nil error, all subsequent calls to
+// this method return the same error.
+func (c *Conn) NextReader() (messageType int, r io.Reader, err error) {
+	// Close previous reader, only relevant for decompression.
+	if c.reader != nil {
+		c.reader.Close()
+		c.reader = nil
+	}
+
+	c.messageReader = nil
+	c.readLength = 0
+
+	for c.readErr == nil {
+		frameType, err := c.advanceFrame()
+		if err != nil {
+			c.readErr = hideTempErr(err)
+			break
+		}
+		if frameType == TextMessage || frameType == BinaryMessage {
+			c.messageReader = &messageReader{c}
+			c.reader = c.messageReader
+			if c.readDecompress {
+				c.reader = c.newDecompressionReader(c.reader)
+			}
+			return frameType, c.reader, nil
+		}
+	}
+
+	// Applications that do handle the error returned from this method spin in
+	// tight loop on connection failure. To help application developers detect
+	// this error, panic on repeated reads to the failed connection.
+	c.readErrCount++
+	if c.readErrCount >= 1000 {
+		panic("repeated read on failed websocket connection")
+	}
+
+	return noFrame, nil, c.readErr
+}
+
+type messageReader struct{ c *Conn }
+
+func (r *messageReader) Read(b []byte) (int, error) {
+	c := r.c
+	if c.messageReader != r {
+		return 0, io.EOF
+	}
+
+	for c.readErr == nil {
+
+		if c.readRemaining > 0 {
+			if int64(len(b)) > c.readRemaining {
+				b = b[:c.readRemaining]
+			}
+			n, err := c.br.Read(b)
+			c.readErr = hideTempErr(err)
+			if c.isServer {
+				c.readMaskPos = maskBytes(c.readMaskKey, c.readMaskPos, b[:n])
+			}
+			c.readRemaining -= int64(n)
+			if c.readRemaining > 0 && c.readErr == io.EOF {
+				c.readErr = errUnexpectedEOF
+			}
+			return n, c.readErr
+		}
+
+		if c.readFinal {
+			c.messageReader = nil
+			return 0, io.EOF
+		}
+
+		frameType, err := c.advanceFrame()
+		switch {
+		case err != nil:
+			c.readErr = hideTempErr(err)
+		case frameType == TextMessage || frameType == BinaryMessage:
+			c.readErr = errors.New("websocket: internal error, unexpected text or binary in Reader")
+		}
+	}
+
+	err := c.readErr
+	if err == io.EOF && c.messageReader == r {
+		err = errUnexpectedEOF
+	}
+	return 0, err
+}
+
+func (r *messageReader) Close() error {
+	return nil
+}
+
+// ReadMessage is a helper method for getting a reader using NextReader and
+// reading from that reader to a buffer.
+func (c *Conn) ReadMessage() (messageType int, p []byte, err error) {
+	var r io.Reader
+	messageType, r, err = c.NextReader()
+	if err != nil {
+		return messageType, nil, err
+	}
+	p, err = ioutil.ReadAll(r)
+	return messageType, p, err
+}
+
+// SetReadDeadline sets the read deadline on the underlying network connection.
+// After a read has timed out, the websocket connection state is corrupt and
+// all future reads will return an error. A zero value for t means reads will
+// not time out.
+func (c *Conn) SetReadDeadline(t time.Time) error {
+	return c.conn.SetReadDeadline(t)
+}
+
+// SetReadLimit sets the maximum size for a message read from the peer. If a
+// message exceeds the limit, the connection sends a close frame to the peer
+// and returns ErrReadLimit to the application.
+func (c *Conn) SetReadLimit(limit int64) {
+	c.readLimit = limit
+}
+
+// CloseHandler returns the current close handler
+func (c *Conn) CloseHandler() func(code int, text string) error {
+	return c.handleClose
+}
+
+// SetCloseHandler sets the handler for close messages received from the peer.
+// The code argument to h is the received close code or CloseNoStatusReceived
+// if the close message is empty. The default close handler sends a close frame
+// back to the peer.
+//
+// The application must read the connection to process close messages as
+// described in the section on Control Frames above.
+//
+// The connection read methods return a CloseError when a close frame is
+// received. Most applications should handle close messages as part of their
+// normal error handling. Applications should only set a close handler when the
+// application must perform some action before sending a close frame back to
+// the peer.
+func (c *Conn) SetCloseHandler(h func(code int, text string) error) {
+	if h == nil {
+		h = func(code int, text string) error {
+			message := []byte{}
+			if code != CloseNoStatusReceived {
+				message = FormatCloseMessage(code, "")
+			}
+			c.WriteControl(CloseMessage, message, time.Now().Add(writeWait))
+			return nil
+		}
+	}
+	c.handleClose = h
+}
+
+// PingHandler returns the current ping handler
+func (c *Conn) PingHandler() func(appData string) error {
+	return c.handlePing
+}
+
+// SetPingHandler sets the handler for ping messages received from the peer.
+// The appData argument to h is the PING frame application data. The default
+// ping handler sends a pong to the peer.
+//
+// The application must read the connection to process ping messages as
+// described in the section on Control Frames above.
+func (c *Conn) SetPingHandler(h func(appData string) error) {
+	if h == nil {
+		h = func(message string) error {
+			err := c.WriteControl(PongMessage, []byte(message), time.Now().Add(writeWait))
+			if err == ErrCloseSent {
+				return nil
+			} else if e, ok := err.(net.Error); ok && e.Temporary() {
+				return nil
+			}
+			return err
+		}
+	}
+	c.handlePing = h
+}
+
+// PongHandler returns the current pong handler
+func (c *Conn) PongHandler() func(appData string) error {
+	return c.handlePong
+}
+
+// SetPongHandler sets the handler for pong messages received from the peer.
+// The appData argument to h is the PONG frame application data. The default
+// pong handler does nothing.
+//
+// The application must read the connection to process ping messages as
+// described in the section on Control Frames above.
+func (c *Conn) SetPongHandler(h func(appData string) error) {
+	if h == nil {
+		h = func(string) error { return nil }
+	}
+	c.handlePong = h
+}
+
+// UnderlyingConn returns the internal net.Conn. This can be used to further
+// modifications to connection specific flags.
+func (c *Conn) UnderlyingConn() net.Conn {
+	return c.conn
+}
+
+// EnableWriteCompression enables and disables write compression of
+// subsequent text and binary messages. This function is a noop if
+// compression was not negotiated with the peer.
+func (c *Conn) EnableWriteCompression(enable bool) {
+	c.enableWriteCompression = enable
+}
+
+// SetCompressionLevel sets the flate compression level for subsequent text and
+// binary messages. This function is a noop if compression was not negotiated
+// with the peer. See the compress/flate package for a description of
+// compression levels.
+func (c *Conn) SetCompressionLevel(level int) error {
+	if !isValidCompressionLevel(level) {
+		return errors.New("websocket: invalid compression level")
+	}
+	c.compressionLevel = level
+	return nil
+}
+
+// FormatCloseMessage formats closeCode and text as a WebSocket close message.
+func FormatCloseMessage(closeCode int, text string) []byte {
+	buf := make([]byte, 2+len(text))
+	binary.BigEndian.PutUint16(buf, uint16(closeCode))
+	copy(buf[2:], text)
+	return buf
+}
diff --git a/vendor/github.com/gorilla/websocket/conn_read.go b/vendor/github.com/gorilla/websocket/conn_read.go
new file mode 100644
index 0000000..1ea1505
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/conn_read.go
@@ -0,0 +1,18 @@
+// Copyright 2016 The Gorilla WebSocket Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build go1.5
+
+package websocket
+
+import "io"
+
+func (c *Conn) read(n int) ([]byte, error) {
+	p, err := c.br.Peek(n)
+	if err == io.EOF {
+		err = errUnexpectedEOF
+	}
+	c.br.Discard(len(p))
+	return p, err
+}
diff --git a/vendor/github.com/gorilla/websocket/conn_read_legacy.go b/vendor/github.com/gorilla/websocket/conn_read_legacy.go
new file mode 100644
index 0000000..018541c
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/conn_read_legacy.go
@@ -0,0 +1,21 @@
+// Copyright 2016 The Gorilla WebSocket Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build !go1.5
+
+package websocket
+
+import "io"
+
+func (c *Conn) read(n int) ([]byte, error) {
+	p, err := c.br.Peek(n)
+	if err == io.EOF {
+		err = errUnexpectedEOF
+	}
+	if len(p) > 0 {
+		// advance over the bytes just read
+		io.ReadFull(c.br, p)
+	}
+	return p, err
+}
diff --git a/vendor/github.com/gorilla/websocket/doc.go b/vendor/github.com/gorilla/websocket/doc.go
new file mode 100644
index 0000000..f5ff082
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/doc.go
@@ -0,0 +1,179 @@
+// Copyright 2013 The Gorilla WebSocket Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// Package websocket implements the WebSocket protocol defined in RFC 6455.
+//
+// Overview
+//
+// The Conn type represents a WebSocket connection. A server application calls
+// the Upgrader.Upgrade method from an HTTP request handler to get a *Conn:
+//
+//  var upgrader = websocket.Upgrader{
+//      ReadBufferSize:  1024,
+//      WriteBufferSize: 1024,
+//  }
+//
+//  func handler(w http.ResponseWriter, r *http.Request) {
+//      conn, err := upgrader.Upgrade(w, r, nil)
+//      if err != nil {
+//          log.Println(err)
+//          return
+//      }
+//      ... Use conn to send and receive messages.
+//  }
+//
+// Call the connection's WriteMessage and ReadMessage methods to send and
+// receive messages as a slice of bytes. This snippet of code shows how to echo
+// messages using these methods:
+//
+//  for {
+//      messageType, p, err := conn.ReadMessage()
+//      if err != nil {
+//          return
+//      }
+//      if err := conn.WriteMessage(messageType, p); err != nil {
+//          return err
+//      }
+//  }
+//
+// In above snippet of code, p is a []byte and messageType is an int with value
+// websocket.BinaryMessage or websocket.TextMessage.
+//
+// An application can also send and receive messages using the io.WriteCloser
+// and io.Reader interfaces. To send a message, call the connection NextWriter
+// method to get an io.WriteCloser, write the message to the writer and close
+// the writer when done. To receive a message, call the connection NextReader
+// method to get an io.Reader and read until io.EOF is returned. This snippet
+// shows how to echo messages using the NextWriter and NextReader methods:
+//
+//  for {
+//      messageType, r, err := conn.NextReader()
+//      if err != nil {
+//          return
+//      }
+//      w, err := conn.NextWriter(messageType)
+//      if err != nil {
+//          return err
+//      }
+//      if _, err := io.Copy(w, r); err != nil {
+//          return err
+//      }
+//      if err := w.Close(); err != nil {
+//          return err
+//      }
+//  }
+//
+// Data Messages
+//
+// The WebSocket protocol distinguishes between text and binary data messages.
+// Text messages are interpreted as UTF-8 encoded text. The interpretation of
+// binary messages is left to the application.
+//
+// This package uses the TextMessage and BinaryMessage integer constants to
+// identify the two data message types. The ReadMessage and NextReader methods
+// return the type of the received message. The messageType argument to the
+// WriteMessage and NextWriter methods specifies the type of a sent message.
+//
+// It is the application's responsibility to ensure that text messages are
+// valid UTF-8 encoded text.
+//
+// Control Messages
+//
+// The WebSocket protocol defines three types of control messages: close, ping
+// and pong. Call the connection WriteControl, WriteMessage or NextWriter
+// methods to send a control message to the peer.
+//
+// Connections handle received close messages by sending a close message to the
+// peer and returning a *CloseError from the the NextReader, ReadMessage or the
+// message Read method.
+//
+// Connections handle received ping and pong messages by invoking callback
+// functions set with SetPingHandler and SetPongHandler methods. The callback
+// functions are called from the NextReader, ReadMessage and the message Read
+// methods.
+//
+// The default ping handler sends a pong to the peer. The application's reading
+// goroutine can block for a short time while the handler writes the pong data
+// to the connection.
+//
+// The application must read the connection to process ping, pong and close
+// messages sent from the peer. If the application is not otherwise interested
+// in messages from the peer, then the application should start a goroutine to
+// read and discard messages from the peer. A simple example is:
+//
+//  func readLoop(c *websocket.Conn) {
+//      for {
+//          if _, _, err := c.NextReader(); err != nil {
+//              c.Close()
+//              break
+//          }
+//      }
+//  }
+//
+// Concurrency
+//
+// Connections support one concurrent reader and one concurrent writer.
+//
+// Applications are responsible for ensuring that no more than one goroutine
+// calls the write methods (NextWriter, SetWriteDeadline, WriteMessage,
+// WriteJSON, EnableWriteCompression, SetCompressionLevel) concurrently and
+// that no more than one goroutine calls the read methods (NextReader,
+// SetReadDeadline, ReadMessage, ReadJSON, SetPongHandler, SetPingHandler)
+// concurrently.
+//
+// The Close and WriteControl methods can be called concurrently with all other
+// methods.
+//
+// Origin Considerations
+//
+// Web browsers allow Javascript applications to open a WebSocket connection to
+// any host. It's up to the server to enforce an origin policy using the Origin
+// request header sent by the browser.
+//
+// The Upgrader calls the function specified in the CheckOrigin field to check
+// the origin. If the CheckOrigin function returns false, then the Upgrade
+// method fails the WebSocket handshake with HTTP status 403.
+//
+// If the CheckOrigin field is nil, then the Upgrader uses a safe default: fail
+// the handshake if the Origin request header is present and not equal to the
+// Host request header.
+//
+// An application can allow connections from any origin by specifying a
+// function that always returns true:
+//
+//  var upgrader = websocket.Upgrader{
+//      CheckOrigin: func(r *http.Request) bool { return true },
+//  }
+//
+// The deprecated package-level Upgrade function does not perform origin
+// checking. The application is responsible for checking the Origin header
+// before calling the Upgrade function.
+//
+// Compression EXPERIMENTAL
+//
+// Per message compression extensions (RFC 7692) are experimentally supported
+// by this package in a limited capacity. Setting the EnableCompression option
+// to true in Dialer or Upgrader will attempt to negotiate per message deflate
+// support.
+//
+//  var upgrader = websocket.Upgrader{
+//      EnableCompression: true,
+//  }
+//
+// If compression was successfully negotiated with the connection's peer, any
+// message received in compressed form will be automatically decompressed.
+// All Read methods will return uncompressed bytes.
+//
+// Per message compression of messages written to a connection can be enabled
+// or disabled by calling the corresponding Conn method:
+//
+//  conn.EnableWriteCompression(false)
+//
+// Currently this package does not support compression with "context takeover".
+// This means that messages must be compressed and decompressed in isolation,
+// without retaining sliding window or dictionary state across messages. For
+// more details refer to RFC 7692.
+//
+// Use of compression is experimental and may result in decreased performance.
+package websocket
diff --git a/vendor/github.com/gorilla/websocket/json.go b/vendor/github.com/gorilla/websocket/json.go
new file mode 100644
index 0000000..dc2c1f6
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/json.go
@@ -0,0 +1,60 @@
+// Copyright 2013 The Gorilla WebSocket Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package websocket
+
+import (
+	"encoding/json"
+	"io"
+)
+
+// WriteJSON writes the JSON encoding of v as a message.
+//
+// Deprecated: Use c.WriteJSON instead.
+func WriteJSON(c *Conn, v interface{}) error {
+	return c.WriteJSON(v)
+}
+
+// WriteJSON writes the JSON encoding of v as a message.
+//
+// See the documentation for encoding/json Marshal for details about the
+// conversion of Go values to JSON.
+func (c *Conn) WriteJSON(v interface{}) error {
+	w, err := c.NextWriter(TextMessage)
+	if err != nil {
+		return err
+	}
+	err1 := json.NewEncoder(w).Encode(v)
+	err2 := w.Close()
+	if err1 != nil {
+		return err1
+	}
+	return err2
+}
+
+// ReadJSON reads the next JSON-encoded message from the connection and stores
+// it in the value pointed to by v.
+//
+// Deprecated: Use c.ReadJSON instead.
+func ReadJSON(c *Conn, v interface{}) error {
+	return c.ReadJSON(v)
+}
+
+// ReadJSON reads the next JSON-encoded message from the connection and stores
+// it in the value pointed to by v.
+//
+// See the documentation for the encoding/json Unmarshal function for details
+// about the conversion of JSON to a Go value.
+func (c *Conn) ReadJSON(v interface{}) error {
+	_, r, err := c.NextReader()
+	if err != nil {
+		return err
+	}
+	err = json.NewDecoder(r).Decode(v)
+	if err == io.EOF {
+		// One value is expected in the message.
+		err = io.ErrUnexpectedEOF
+	}
+	return err
+}
diff --git a/vendor/github.com/gorilla/websocket/mask.go b/vendor/github.com/gorilla/websocket/mask.go
new file mode 100644
index 0000000..6a88bbc
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/mask.go
@@ -0,0 +1,55 @@
+// Copyright 2016 The Gorilla WebSocket Authors. All rights reserved.  Use of
+// this source code is governed by a BSD-style license that can be found in the
+// LICENSE file.
+
+// +build !appengine
+
+package websocket
+
+import "unsafe"
+
+const wordSize = int(unsafe.Sizeof(uintptr(0)))
+
+func maskBytes(key [4]byte, pos int, b []byte) int {
+
+	// Mask one byte at a time for small buffers.
+	if len(b) < 2*wordSize {
+		for i := range b {
+			b[i] ^= key[pos&3]
+			pos++
+		}
+		return pos & 3
+	}
+
+	// Mask one byte at a time to word boundary.
+	if n := int(uintptr(unsafe.Pointer(&b[0]))) % wordSize; n != 0 {
+		n = wordSize - n
+		for i := range b[:n] {
+			b[i] ^= key[pos&3]
+			pos++
+		}
+		b = b[n:]
+	}
+
+	// Create aligned word size key.
+	var k [wordSize]byte
+	for i := range k {
+		k[i] = key[(pos+i)&3]
+	}
+	kw := *(*uintptr)(unsafe.Pointer(&k))
+
+	// Mask one word at a time.
+	n := (len(b) / wordSize) * wordSize
+	for i := 0; i < n; i += wordSize {
+		*(*uintptr)(unsafe.Pointer(uintptr(unsafe.Pointer(&b[0])) + uintptr(i))) ^= kw
+	}
+
+	// Mask one byte at a time for remaining bytes.
+	b = b[n:]
+	for i := range b {
+		b[i] ^= key[pos&3]
+		pos++
+	}
+
+	return pos & 3
+}
diff --git a/vendor/github.com/gorilla/websocket/mask_safe.go b/vendor/github.com/gorilla/websocket/mask_safe.go
new file mode 100644
index 0000000..2aac060
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/mask_safe.go
@@ -0,0 +1,15 @@
+// Copyright 2016 The Gorilla WebSocket Authors. All rights reserved.  Use of
+// this source code is governed by a BSD-style license that can be found in the
+// LICENSE file.
+
+// +build appengine
+
+package websocket
+
+func maskBytes(key [4]byte, pos int, b []byte) int {
+	for i := range b {
+		b[i] ^= key[pos&3]
+		pos++
+	}
+	return pos & 3
+}
diff --git a/vendor/github.com/gorilla/websocket/prepared.go b/vendor/github.com/gorilla/websocket/prepared.go
new file mode 100644
index 0000000..1efffbd
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/prepared.go
@@ -0,0 +1,103 @@
+// Copyright 2017 The Gorilla WebSocket Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package websocket
+
+import (
+	"bytes"
+	"net"
+	"sync"
+	"time"
+)
+
+// PreparedMessage caches on the wire representations of a message payload.
+// Use PreparedMessage to efficiently send a message payload to multiple
+// connections. PreparedMessage is especially useful when compression is used
+// because the CPU and memory expensive compression operation can be executed
+// once for a given set of compression options.
+type PreparedMessage struct {
+	messageType int
+	data        []byte
+	err         error
+	mu          sync.Mutex
+	frames      map[prepareKey]*preparedFrame
+}
+
+// prepareKey defines a unique set of options to cache prepared frames in PreparedMessage.
+type prepareKey struct {
+	isServer         bool
+	compress         bool
+	compressionLevel int
+}
+
+// preparedFrame contains data in wire representation.
+type preparedFrame struct {
+	once sync.Once
+	data []byte
+}
+
+// NewPreparedMessage returns an initialized PreparedMessage. You can then send
+// it to connection using WritePreparedMessage method. Valid wire
+// representation will be calculated lazily only once for a set of current
+// connection options.
+func NewPreparedMessage(messageType int, data []byte) (*PreparedMessage, error) {
+	pm := &PreparedMessage{
+		messageType: messageType,
+		frames:      make(map[prepareKey]*preparedFrame),
+		data:        data,
+	}
+
+	// Prepare a plain server frame.
+	_, frameData, err := pm.frame(prepareKey{isServer: true, compress: false})
+	if err != nil {
+		return nil, err
+	}
+
+	// To protect against caller modifying the data argument, remember the data
+	// copied to the plain server frame.
+	pm.data = frameData[len(frameData)-len(data):]
+	return pm, nil
+}
+
+func (pm *PreparedMessage) frame(key prepareKey) (int, []byte, error) {
+	pm.mu.Lock()
+	frame, ok := pm.frames[key]
+	if !ok {
+		frame = &preparedFrame{}
+		pm.frames[key] = frame
+	}
+	pm.mu.Unlock()
+
+	var err error
+	frame.once.Do(func() {
+		// Prepare a frame using a 'fake' connection.
+		// TODO: Refactor code in conn.go to allow more direct construction of
+		// the frame.
+		mu := make(chan bool, 1)
+		mu <- true
+		var nc prepareConn
+		c := &Conn{
+			conn:                   &nc,
+			mu:                     mu,
+			isServer:               key.isServer,
+			compressionLevel:       key.compressionLevel,
+			enableWriteCompression: true,
+			writeBuf:               make([]byte, defaultWriteBufferSize+maxFrameHeaderSize),
+		}
+		if key.compress {
+			c.newCompressionWriter = compressNoContextTakeover
+		}
+		err = c.WriteMessage(pm.messageType, pm.data)
+		frame.data = nc.buf.Bytes()
+	})
+	return pm.messageType, frame.data, err
+}
+
+type prepareConn struct {
+	buf bytes.Buffer
+	net.Conn
+}
+
+func (pc *prepareConn) Write(p []byte) (int, error)        { return pc.buf.Write(p) }
+func (pc *prepareConn) SetWriteDeadline(t time.Time) error { return nil }
diff --git a/vendor/github.com/gorilla/websocket/server.go b/vendor/github.com/gorilla/websocket/server.go
new file mode 100644
index 0000000..6ae97c5
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/server.go
@@ -0,0 +1,292 @@
+// Copyright 2013 The Gorilla WebSocket Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package websocket
+
+import (
+	"bufio"
+	"errors"
+	"net"
+	"net/http"
+	"net/url"
+	"strings"
+	"time"
+)
+
+// HandshakeError describes an error with the handshake from the peer.
+type HandshakeError struct {
+	message string
+}
+
+func (e HandshakeError) Error() string { return e.message }
+
+// Upgrader specifies parameters for upgrading an HTTP connection to a
+// WebSocket connection.
+type Upgrader struct {
+	// HandshakeTimeout specifies the duration for the handshake to complete.
+	HandshakeTimeout time.Duration
+
+	// ReadBufferSize and WriteBufferSize specify I/O buffer sizes. If a buffer
+	// size is zero, then buffers allocated by the HTTP server are used. The
+	// I/O buffer sizes do not limit the size of the messages that can be sent
+	// or received.
+	ReadBufferSize, WriteBufferSize int
+
+	// Subprotocols specifies the server's supported protocols in order of
+	// preference. If this field is set, then the Upgrade method negotiates a
+	// subprotocol by selecting the first match in this list with a protocol
+	// requested by the client.
+	Subprotocols []string
+
+	// Error specifies the function for generating HTTP error responses. If Error
+	// is nil, then http.Error is used to generate the HTTP response.
+	Error func(w http.ResponseWriter, r *http.Request, status int, reason error)
+
+	// CheckOrigin returns true if the request Origin header is acceptable. If
+	// CheckOrigin is nil, the host in the Origin header must not be set or
+	// must match the host of the request.
+	CheckOrigin func(r *http.Request) bool
+
+	// EnableCompression specify if the server should attempt to negotiate per
+	// message compression (RFC 7692). Setting this value to true does not
+	// guarantee that compression will be supported. Currently only "no context
+	// takeover" modes are supported.
+	EnableCompression bool
+}
+
+func (u *Upgrader) returnError(w http.ResponseWriter, r *http.Request, status int, reason string) (*Conn, error) {
+	err := HandshakeError{reason}
+	if u.Error != nil {
+		u.Error(w, r, status, err)
+	} else {
+		w.Header().Set("Sec-Websocket-Version", "13")
+		http.Error(w, http.StatusText(status), status)
+	}
+	return nil, err
+}
+
+// checkSameOrigin returns true if the origin is not set or is equal to the request host.
+func checkSameOrigin(r *http.Request) bool {
+	origin := r.Header["Origin"]
+	if len(origin) == 0 {
+		return true
+	}
+	u, err := url.Parse(origin[0])
+	if err != nil {
+		return false
+	}
+	return u.Host == r.Host
+}
+
+func (u *Upgrader) selectSubprotocol(r *http.Request, responseHeader http.Header) string {
+	if u.Subprotocols != nil {
+		clientProtocols := Subprotocols(r)
+		for _, serverProtocol := range u.Subprotocols {
+			for _, clientProtocol := range clientProtocols {
+				if clientProtocol == serverProtocol {
+					return clientProtocol
+				}
+			}
+		}
+	} else if responseHeader != nil {
+		return responseHeader.Get("Sec-Websocket-Protocol")
+	}
+	return ""
+}
+
+// Upgrade upgrades the HTTP server connection to the WebSocket protocol.
+//
+// The responseHeader is included in the response to the client's upgrade
+// request. Use the responseHeader to specify cookies (Set-Cookie) and the
+// application negotiated subprotocol (Sec-Websocket-Protocol).
+//
+// If the upgrade fails, then Upgrade replies to the client with an HTTP error
+// response.
+func (u *Upgrader) Upgrade(w http.ResponseWriter, r *http.Request, responseHeader http.Header) (*Conn, error) {
+	if r.Method != "GET" {
+		return u.returnError(w, r, http.StatusMethodNotAllowed, "websocket: not a websocket handshake: request method is not GET")
+	}
+
+	if _, ok := responseHeader["Sec-Websocket-Extensions"]; ok {
+		return u.returnError(w, r, http.StatusInternalServerError, "websocket: application specific 'Sec-Websocket-Extensions' headers are unsupported")
+	}
+
+	if !tokenListContainsValue(r.Header, "Connection", "upgrade") {
+		return u.returnError(w, r, http.StatusBadRequest, "websocket: not a websocket handshake: 'upgrade' token not found in 'Connection' header")
+	}
+
+	if !tokenListContainsValue(r.Header, "Upgrade", "websocket") {
+		return u.returnError(w, r, http.StatusBadRequest, "websocket: not a websocket handshake: 'websocket' token not found in 'Upgrade' header")
+	}
+
+	if !tokenListContainsValue(r.Header, "Sec-Websocket-Version", "13") {
+		return u.returnError(w, r, http.StatusBadRequest, "websocket: unsupported version: 13 not found in 'Sec-Websocket-Version' header")
+	}
+
+	checkOrigin := u.CheckOrigin
+	if checkOrigin == nil {
+		checkOrigin = checkSameOrigin
+	}
+	if !checkOrigin(r) {
+		return u.returnError(w, r, http.StatusForbidden, "websocket: 'Origin' header value not allowed")
+	}
+
+	challengeKey := r.Header.Get("Sec-Websocket-Key")
+	if challengeKey == "" {
+		return u.returnError(w, r, http.StatusBadRequest, "websocket: not a websocket handshake: `Sec-Websocket-Key' header is missing or blank")
+	}
+
+	subprotocol := u.selectSubprotocol(r, responseHeader)
+
+	// Negotiate PMCE
+	var compress bool
+	if u.EnableCompression {
+		for _, ext := range parseExtensions(r.Header) {
+			if ext[""] != "permessage-deflate" {
+				continue
+			}
+			compress = true
+			break
+		}
+	}
+
+	var (
+		netConn net.Conn
+		err     error
+	)
+
+	h, ok := w.(http.Hijacker)
+	if !ok {
+		return u.returnError(w, r, http.StatusInternalServerError, "websocket: response does not implement http.Hijacker")
+	}
+	var brw *bufio.ReadWriter
+	netConn, brw, err = h.Hijack()
+	if err != nil {
+		return u.returnError(w, r, http.StatusInternalServerError, err.Error())
+	}
+
+	if brw.Reader.Buffered() > 0 {
+		netConn.Close()
+		return nil, errors.New("websocket: client sent data before handshake is complete")
+	}
+
+	c := newConnBRW(netConn, true, u.ReadBufferSize, u.WriteBufferSize, brw)
+	c.subprotocol = subprotocol
+
+	if compress {
+		c.newCompressionWriter = compressNoContextTakeover
+		c.newDecompressionReader = decompressNoContextTakeover
+	}
+
+	p := c.writeBuf[:0]
+	p = append(p, "HTTP/1.1 101 Switching Protocols\r\nUpgrade: websocket\r\nConnection: Upgrade\r\nSec-WebSocket-Accept: "...)
+	p = append(p, computeAcceptKey(challengeKey)...)
+	p = append(p, "\r\n"...)
+	if c.subprotocol != "" {
+		p = append(p, "Sec-Websocket-Protocol: "...)
+		p = append(p, c.subprotocol...)
+		p = append(p, "\r\n"...)
+	}
+	if compress {
+		p = append(p, "Sec-Websocket-Extensions: permessage-deflate; server_no_context_takeover; client_no_context_takeover\r\n"...)
+	}
+	for k, vs := range responseHeader {
+		if k == "Sec-Websocket-Protocol" {
+			continue
+		}
+		for _, v := range vs {
+			p = append(p, k...)
+			p = append(p, ": "...)
+			for i := 0; i < len(v); i++ {
+				b := v[i]
+				if b <= 31 {
+					// prevent response splitting.
+					b = ' '
+				}
+				p = append(p, b)
+			}
+			p = append(p, "\r\n"...)
+		}
+	}
+	p = append(p, "\r\n"...)
+
+	// Clear deadlines set by HTTP server.
+	netConn.SetDeadline(time.Time{})
+
+	if u.HandshakeTimeout > 0 {
+		netConn.SetWriteDeadline(time.Now().Add(u.HandshakeTimeout))
+	}
+	if _, err = netConn.Write(p); err != nil {
+		netConn.Close()
+		return nil, err
+	}
+	if u.HandshakeTimeout > 0 {
+		netConn.SetWriteDeadline(time.Time{})
+	}
+
+	return c, nil
+}
+
+// Upgrade upgrades the HTTP server connection to the WebSocket protocol.
+//
+// Deprecated: Use websocket.Upgrader instead.
+//
+// Upgrade does not perform origin checking. The application is responsible for
+// checking the Origin header before calling Upgrade. An example implementation
+// of the same origin policy check is:
+//
+//	if req.Header.Get("Origin") != "http://"+req.Host {
+//		http.Error(w, "Origin not allowed", 403)
+//		return
+//	}
+//
+// If the endpoint supports subprotocols, then the application is responsible
+// for negotiating the protocol used on the connection. Use the Subprotocols()
+// function to get the subprotocols requested by the client. Use the
+// Sec-Websocket-Protocol response header to specify the subprotocol selected
+// by the application.
+//
+// The responseHeader is included in the response to the client's upgrade
+// request. Use the responseHeader to specify cookies (Set-Cookie) and the
+// negotiated subprotocol (Sec-Websocket-Protocol).
+//
+// The connection buffers IO to the underlying network connection. The
+// readBufSize and writeBufSize parameters specify the size of the buffers to
+// use. Messages can be larger than the buffers.
+//
+// If the request is not a valid WebSocket handshake, then Upgrade returns an
+// error of type HandshakeError. Applications should handle this error by
+// replying to the client with an HTTP error response.
+func Upgrade(w http.ResponseWriter, r *http.Request, responseHeader http.Header, readBufSize, writeBufSize int) (*Conn, error) {
+	u := Upgrader{ReadBufferSize: readBufSize, WriteBufferSize: writeBufSize}
+	u.Error = func(w http.ResponseWriter, r *http.Request, status int, reason error) {
+		// don't return errors to maintain backwards compatibility
+	}
+	u.CheckOrigin = func(r *http.Request) bool {
+		// allow all connections by default
+		return true
+	}
+	return u.Upgrade(w, r, responseHeader)
+}
+
+// Subprotocols returns the subprotocols requested by the client in the
+// Sec-Websocket-Protocol header.
+func Subprotocols(r *http.Request) []string {
+	h := strings.TrimSpace(r.Header.Get("Sec-Websocket-Protocol"))
+	if h == "" {
+		return nil
+	}
+	protocols := strings.Split(h, ",")
+	for i := range protocols {
+		protocols[i] = strings.TrimSpace(protocols[i])
+	}
+	return protocols
+}
+
+// IsWebSocketUpgrade returns true if the client requested upgrade to the
+// WebSocket protocol.
+func IsWebSocketUpgrade(r *http.Request) bool {
+	return tokenListContainsValue(r.Header, "Connection", "upgrade") &&
+		tokenListContainsValue(r.Header, "Upgrade", "websocket")
+}
diff --git a/vendor/github.com/gorilla/websocket/util.go b/vendor/github.com/gorilla/websocket/util.go
new file mode 100644
index 0000000..262e647
--- /dev/null
+++ b/vendor/github.com/gorilla/websocket/util.go
@@ -0,0 +1,214 @@
+// Copyright 2013 The Gorilla WebSocket Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package websocket
+
+import (
+	"crypto/rand"
+	"crypto/sha1"
+	"encoding/base64"
+	"io"
+	"net/http"
+	"strings"
+)
+
+var keyGUID = []byte("258EAFA5-E914-47DA-95CA-C5AB0DC85B11")
+
+func computeAcceptKey(challengeKey string) string {
+	h := sha1.New()
+	h.Write([]byte(challengeKey))
+	h.Write(keyGUID)
+	return base64.StdEncoding.EncodeToString(h.Sum(nil))
+}
+
+func generateChallengeKey() (string, error) {
+	p := make([]byte, 16)
+	if _, err := io.ReadFull(rand.Reader, p); err != nil {
+		return "", err
+	}
+	return base64.StdEncoding.EncodeToString(p), nil
+}
+
+// Octet types from RFC 2616.
+var octetTypes [256]byte
+
+const (
+	isTokenOctet = 1 << iota
+	isSpaceOctet
+)
+
+func init() {
+	// From RFC 2616
+	//
+	// OCTET      = <any 8-bit sequence of data>
+	// CHAR       = <any US-ASCII character (octets 0 - 127)>
+	// CTL        = <any US-ASCII control character (octets 0 - 31) and DEL (127)>
+	// CR         = <US-ASCII CR, carriage return (13)>
+	// LF         = <US-ASCII LF, linefeed (10)>
+	// SP         = <US-ASCII SP, space (32)>
+	// HT         = <US-ASCII HT, horizontal-tab (9)>
+	// <">        = <US-ASCII double-quote mark (34)>
+	// CRLF       = CR LF
+	// LWS        = [CRLF] 1*( SP | HT )
+	// TEXT       = <any OCTET except CTLs, but including LWS>
+	// separators = "(" | ")" | "<" | ">" | "@" | "," | ";" | ":" | "\" | <">
+	//              | "/" | "[" | "]" | "?" | "=" | "{" | "}" | SP | HT
+	// token      = 1*<any CHAR except CTLs or separators>
+	// qdtext     = <any TEXT except <">>
+
+	for c := 0; c < 256; c++ {
+		var t byte
+		isCtl := c <= 31 || c == 127
+		isChar := 0 <= c && c <= 127
+		isSeparator := strings.IndexRune(" \t\"(),/:;<=>?@[]\\{}", rune(c)) >= 0
+		if strings.IndexRune(" \t\r\n", rune(c)) >= 0 {
+			t |= isSpaceOctet
+		}
+		if isChar && !isCtl && !isSeparator {
+			t |= isTokenOctet
+		}
+		octetTypes[c] = t
+	}
+}
+
+func skipSpace(s string) (rest string) {
+	i := 0
+	for ; i < len(s); i++ {
+		if octetTypes[s[i]]&isSpaceOctet == 0 {
+			break
+		}
+	}
+	return s[i:]
+}
+
+func nextToken(s string) (token, rest string) {
+	i := 0
+	for ; i < len(s); i++ {
+		if octetTypes[s[i]]&isTokenOctet == 0 {
+			break
+		}
+	}
+	return s[:i], s[i:]
+}
+
+func nextTokenOrQuoted(s string) (value string, rest string) {
+	if !strings.HasPrefix(s, "\"") {
+		return nextToken(s)
+	}
+	s = s[1:]
+	for i := 0; i < len(s); i++ {
+		switch s[i] {
+		case '"':
+			return s[:i], s[i+1:]
+		case '\\':
+			p := make([]byte, len(s)-1)
+			j := copy(p, s[:i])
+			escape := true
+			for i = i + 1; i < len(s); i++ {
+				b := s[i]
+				switch {
+				case escape:
+					escape = false
+					p[j] = b
+					j++
+				case b == '\\':
+					escape = true
+				case b == '"':
+					return string(p[:j]), s[i+1:]
+				default:
+					p[j] = b
+					j++
+				}
+			}
+			return "", ""
+		}
+	}
+	return "", ""
+}
+
+// tokenListContainsValue returns true if the 1#token header with the given
+// name contains token.
+func tokenListContainsValue(header http.Header, name string, value string) bool {
+headers:
+	for _, s := range header[name] {
+		for {
+			var t string
+			t, s = nextToken(skipSpace(s))
+			if t == "" {
+				continue headers
+			}
+			s = skipSpace(s)
+			if s != "" && s[0] != ',' {
+				continue headers
+			}
+			if strings.EqualFold(t, value) {
+				return true
+			}
+			if s == "" {
+				continue headers
+			}
+			s = s[1:]
+		}
+	}
+	return false
+}
+
+// parseExtensiosn parses WebSocket extensions from a header.
+func parseExtensions(header http.Header) []map[string]string {
+
+	// From RFC 6455:
+	//
+	//  Sec-WebSocket-Extensions = extension-list
+	//  extension-list = 1#extension
+	//  extension = extension-token *( ";" extension-param )
+	//  extension-token = registered-token
+	//  registered-token = token
+	//  extension-param = token [ "=" (token | quoted-string) ]
+	//     ;When using the quoted-string syntax variant, the value
+	//     ;after quoted-string unescaping MUST conform to the
+	//     ;'token' ABNF.
+
+	var result []map[string]string
+headers:
+	for _, s := range header["Sec-Websocket-Extensions"] {
+		for {
+			var t string
+			t, s = nextToken(skipSpace(s))
+			if t == "" {
+				continue headers
+			}
+			ext := map[string]string{"": t}
+			for {
+				s = skipSpace(s)
+				if !strings.HasPrefix(s, ";") {
+					break
+				}
+				var k string
+				k, s = nextToken(skipSpace(s[1:]))
+				if k == "" {
+					continue headers
+				}
+				s = skipSpace(s)
+				var v string
+				if strings.HasPrefix(s, "=") {
+					v, s = nextTokenOrQuoted(skipSpace(s[1:]))
+					s = skipSpace(s)
+				}
+				if s != "" && s[0] != ',' && s[0] != ';' {
+					continue headers
+				}
+				ext[k] = v
+			}
+			if s != "" && s[0] != ',' {
+				continue headers
+			}
+			result = append(result, ext)
+			if s == "" {
+				continue headers
+			}
+			s = s[1:]
+		}
+	}
+	return result
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/.gitignore b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/.gitignore
new file mode 100644
index 0000000..c198e6a
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/.gitignore
@@ -0,0 +1,202 @@
+# Created by .ignore support plugin (hsz.mobi)
+### Go template
+# Compiled Object files, Static and Dynamic libs (Shared Objects)
+*.o
+*.a
+*.so
+
+# Folders
+_obj
+_test
+
+# Architecture specific extensions/prefixes
+*.[568vq]
+[568vq].out
+
+*.cgo1.go
+*.cgo2.c
+_cgo_defun.c
+_cgo_gotypes.go
+_cgo_export.*
+
+_testmain.go
+
+*.exe
+*.test
+*.prof
+### Windows template
+# Windows image file caches
+Thumbs.db
+ehthumbs.db
+
+# Folder config file
+Desktop.ini
+
+# Recycle Bin used on file shares
+$RECYCLE.BIN/
+
+# Windows Installer files
+*.cab
+*.msi
+*.msm
+*.msp
+
+# Windows shortcuts
+*.lnk
+### Kate template
+# Swap Files #
+.*.kate-swp
+.swp.*
+### SublimeText template
+# cache files for sublime text
+*.tmlanguage.cache
+*.tmPreferences.cache
+*.stTheme.cache
+
+# workspace files are user-specific
+*.sublime-workspace
+
+# project files should be checked into the repository, unless a significant
+# proportion of contributors will probably not be using SublimeText
+# *.sublime-project
+
+# sftp configuration file
+sftp-config.json
+### Linux template
+*~
+
+# temporary files which can be created if a process still has a handle open of a deleted file
+.fuse_hidden*
+
+# KDE directory preferences
+.directory
+
+# Linux trash folder which might appear on any partition or disk
+.Trash-*
+### JetBrains template
+# Covers JetBrains IDEs: IntelliJ, RubyMine, PhpStorm, AppCode, PyCharm, CLion, Android Studio and Webstorm
+# Reference: https://intellij-support.jetbrains.com/hc/en-us/articles/206544839
+
+# User-specific stuff:
+.idea
+.idea/tasks.xml
+.idea/dictionaries
+.idea/vcs.xml
+.idea/jsLibraryMappings.xml
+
+# Sensitive or high-churn files:
+.idea/dataSources.ids
+.idea/dataSources.xml
+.idea/dataSources.local.xml
+.idea/sqlDataSources.xml
+.idea/dynamic.xml
+.idea/uiDesigner.xml
+
+# Gradle:
+.idea/gradle.xml
+.idea/libraries
+
+# Mongo Explorer plugin:
+.idea/mongoSettings.xml
+
+## File-based project format:
+*.iws
+
+## Plugin-specific files:
+
+# IntelliJ
+/out/
+
+# mpeltonen/sbt-idea plugin
+.idea_modules/
+
+# JIRA plugin
+atlassian-ide-plugin.xml
+
+# Crashlytics plugin (for Android Studio and IntelliJ)
+com_crashlytics_export_strings.xml
+crashlytics.properties
+crashlytics-build.properties
+fabric.properties
+### Xcode template
+# Xcode
+#
+# gitignore contributors: remember to update Global/Xcode.gitignore, Objective-C.gitignore & Swift.gitignore
+
+## Build generated
+build/
+DerivedData/
+
+## Various settings
+*.pbxuser
+!default.pbxuser
+*.mode1v3
+!default.mode1v3
+*.mode2v3
+!default.mode2v3
+*.perspectivev3
+!default.perspectivev3
+xcuserdata/
+
+## Other
+*.moved-aside
+*.xccheckout
+*.xcscmblueprint
+### Eclipse template
+
+.metadata
+bin/
+tmp/
+*.tmp
+*.bak
+*.swp
+*~.nib
+local.properties
+.settings/
+.loadpath
+.recommenders
+
+# Eclipse Core
+.project
+
+# External tool builders
+.externalToolBuilders/
+
+# Locally stored "Eclipse launch configurations"
+*.launch
+
+# PyDev specific (Python IDE for Eclipse)
+*.pydevproject
+
+# CDT-specific (C/C++ Development Tooling)
+.cproject
+
+# JDT-specific (Eclipse Java Development Tools)
+.classpath
+
+# Java annotation processor (APT)
+.factorypath
+
+# PDT-specific (PHP Development Tools)
+.buildpath
+
+# sbteclipse plugin
+.target
+
+# Tern plugin
+.tern-project
+
+# TeXlipse plugin
+.texlipse
+
+# STS (Spring Tool Suite)
+.springBeans
+
+# Code Recommenders
+.recommenders/
+
+
+coverage.txt
+
+#vendor
+vendor/
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/.travis.yml b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/.travis.yml
new file mode 100644
index 0000000..702fa5b
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/.travis.yml
@@ -0,0 +1,21 @@
+sudo: false
+language: go
+go:
+  - 1.8.x
+env:
+  - DEP_VERSION="0.3.2"
+
+before_install:
+  # Download the binary to bin folder in $GOPATH
+  - curl -L -s https://github.com/golang/dep/releases/download/v${DEP_VERSION}/dep-linux-amd64 -o $GOPATH/bin/dep
+  # Make the binary executable
+  - chmod +x $GOPATH/bin/dep
+
+install:
+  - dep ensure
+
+script:
+ - make test
+ 
+after_success:
+  - bash <(curl -s https://codecov.io/bash)
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/CHANGELOG.md b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/CHANGELOG.md
new file mode 100644
index 0000000..0e64822
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/CHANGELOG.md
@@ -0,0 +1,30 @@
+# Changelog
+All notable changes to this project will be documented in this file.
+
+The format is based on [Keep a Changelog](http://keepachangelog.com/en/1.0.0/)
+and this project adheres to [Semantic Versioning](http://semver.org/spec/v2.0.0.html).
+
+Types of changes:
+- `Added` for new features.
+- `Changed` for changes in existing functionality.
+- `Deprecated` for soon-to-be removed features.
+- `Removed` for now removed features.
+- `Fixed` for any bug fixes.
+- `Security` in case of vulnerabilities.
+
+## [Unreleased]
+### Added
+- This CHANGELOG file to keep track of changes.
+
+## 1.0.0 - 2018-05-08
+### Added
+- grpc_auth 
+- grpc_ctxtags
+- grpc_zap
+- grpc_logrus
+- grpc_opentracing
+- grpc_retry
+- grpc_validator
+- grpc_recovery
+
+[Unreleased]: https://github.com/grpc-ecosystem/go-grpc-middleware/compare/v1.0.0...HEAD 
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/CONTRIBUTING.md b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/CONTRIBUTING.md
new file mode 100644
index 0000000..dd52ab8
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/CONTRIBUTING.md
@@ -0,0 +1,20 @@
+# Contributing
+
+We would love to have people submit pull requests and help make `grpc-ecosystem/go-grpc-middleware` even better 👍.
+
+Fork, then clone the repo:
+
+```bash
+git clone git@github.com:your-username/go-grpc-middleware.git
+```    
+
+Before checking in please run the following:
+
+```bash
+make all
+```
+
+This will `vet`, `fmt`, regenerate documentation and run all tests.
+
+
+Push to your fork and open a pull request.
\ No newline at end of file
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/Gopkg.lock b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/Gopkg.lock
new file mode 100644
index 0000000..ebdcb75
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/Gopkg.lock
@@ -0,0 +1,123 @@
+# This file is autogenerated, do not edit; changes may be undone by the next 'dep ensure'.
+
+
+[[projects]]
+  name = "cloud.google.com/go"
+  packages = ["compute/metadata"]
+  revision = "2d3a6656c17a60b0815b7e06ab0be04eacb6e613"
+  version = "v0.16.0"
+
+[[projects]]
+  name = "github.com/davecgh/go-spew"
+  packages = ["spew"]
+  revision = "346938d642f2ec3594ed81d874461961cd0faa76"
+  version = "v1.1.0"
+
+[[projects]]
+  name = "github.com/gogo/protobuf"
+  packages = ["gogoproto","proto","protoc-gen-gogo/descriptor"]
+  revision = "342cbe0a04158f6dcb03ca0079991a51a4248c02"
+  version = "v0.5"
+
+[[projects]]
+  branch = "master"
+  name = "github.com/golang/protobuf"
+  packages = ["jsonpb","proto","ptypes","ptypes/any","ptypes/duration","ptypes/struct","ptypes/timestamp"]
+  revision = "1e59b77b52bf8e4b449a57e6f79f21226d571845"
+
+[[projects]]
+  name = "github.com/opentracing/opentracing-go"
+  packages = [".","ext","log","mocktracer"]
+  revision = "1949ddbfd147afd4d964a9f00b24eb291e0e7c38"
+  version = "v1.0.2"
+
+[[projects]]
+  name = "github.com/pmezard/go-difflib"
+  packages = ["difflib"]
+  revision = "792786c7400a136282c1664665ae0a8db921c6c2"
+  version = "v1.0.0"
+
+[[projects]]
+  name = "github.com/sirupsen/logrus"
+  packages = ["."]
+  revision = "f006c2ac4710855cf0f916dd6b77acf6b048dc6e"
+  version = "v1.0.3"
+
+[[projects]]
+  name = "github.com/stretchr/testify"
+  packages = ["assert","require","suite"]
+  revision = "69483b4bd14f5845b5a1e55bca19e954e827f1d0"
+  version = "v1.1.4"
+
+[[projects]]
+  name = "go.uber.org/atomic"
+  packages = ["."]
+  revision = "8474b86a5a6f79c443ce4b2992817ff32cf208b8"
+  version = "v1.3.1"
+
+[[projects]]
+  name = "go.uber.org/multierr"
+  packages = ["."]
+  revision = "3c4937480c32f4c13a875a1829af76c98ca3d40a"
+  version = "v1.1.0"
+
+[[projects]]
+  name = "go.uber.org/zap"
+  packages = [".","buffer","internal/bufferpool","internal/color","internal/exit","zapcore"]
+  revision = "35aad584952c3e7020db7b839f6b102de6271f89"
+  version = "v1.7.1"
+
+[[projects]]
+  branch = "master"
+  name = "golang.org/x/crypto"
+  packages = ["ssh/terminal"]
+  revision = "94eea52f7b742c7cbe0b03b22f0c4c8631ece122"
+
+[[projects]]
+  branch = "master"
+  name = "golang.org/x/net"
+  packages = ["context","context/ctxhttp","http2","http2/hpack","idna","internal/timeseries","lex/httplex","trace"]
+  revision = "a8b9294777976932365dabb6640cf1468d95c70f"
+
+[[projects]]
+  branch = "master"
+  name = "golang.org/x/oauth2"
+  packages = [".","google","internal","jws","jwt"]
+  revision = "f95fa95eaa936d9d87489b15d1d18b97c1ba9c28"
+
+[[projects]]
+  branch = "master"
+  name = "golang.org/x/sys"
+  packages = ["unix","windows"]
+  revision = "13fcbd661c8ececa8807a29b48407d674b1d8ed8"
+
+[[projects]]
+  branch = "master"
+  name = "golang.org/x/text"
+  packages = ["collate","collate/build","internal/colltab","internal/gen","internal/tag","internal/triegen","internal/ucd","language","secure/bidirule","transform","unicode/bidi","unicode/cldr","unicode/norm","unicode/rangetable"]
+  revision = "75cc3cad82b5f47d3fb229ddda8c5167da14f294"
+
+[[projects]]
+  name = "google.golang.org/appengine"
+  packages = [".","internal","internal/app_identity","internal/base","internal/datastore","internal/log","internal/modules","internal/remote_api","internal/urlfetch","urlfetch"]
+  revision = "150dc57a1b433e64154302bdc40b6bb8aefa313a"
+  version = "v1.0.0"
+
+[[projects]]
+  branch = "master"
+  name = "google.golang.org/genproto"
+  packages = ["googleapis/rpc/status"]
+  revision = "7f0da29060c682909f650ad8ed4e515bd74fa12a"
+
+[[projects]]
+  name = "google.golang.org/grpc"
+  packages = [".","balancer","balancer/roundrobin","codes","connectivity","credentials","credentials/oauth","encoding","grpclb/grpc_lb_v1/messages","grpclog","internal","keepalive","metadata","naming","peer","resolver","resolver/dns","resolver/passthrough","stats","status","tap","transport"]
+  revision = "5a9f7b402fe85096d2e1d0383435ee1876e863d0"
+  version = "v1.8.0"
+
+[solve-meta]
+  analyzer-name = "dep"
+  analyzer-version = 1
+  inputs-digest = "b24c6670412eb0bc44ed1db77fecc52333f8725f3e3272bdc568f5683a63031f"
+  solver-name = "gps-cdcl"
+  solver-version = 1
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/Gopkg.toml b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/Gopkg.toml
new file mode 100644
index 0000000..0a7d4c1
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/Gopkg.toml
@@ -0,0 +1,35 @@
+[[constraint]]
+  name = "github.com/gogo/protobuf"
+  version = "0.5.0"
+
+[[constraint]]
+  branch = "master"
+  name = "github.com/golang/protobuf"
+
+[[constraint]]
+  name = "github.com/opentracing/opentracing-go"
+  version = "1.0.2"
+
+[[constraint]]
+  name = "github.com/sirupsen/logrus"
+  version = "1.0.3"
+
+[[constraint]]
+  name = "github.com/stretchr/testify"
+  version = "1.1.4"
+
+[[constraint]]
+  name = "go.uber.org/zap"
+  version = "1.7.1"
+
+[[constraint]]
+  branch = "master"
+  name = "golang.org/x/net"
+
+[[constraint]]
+  branch = "master"
+  name = "golang.org/x/oauth2"
+
+[[constraint]]
+  name = "google.golang.org/grpc"
+  version = "1.8.0"
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/LICENSE b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/LICENSE
new file mode 100644
index 0000000..b2b0650
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/LICENSE
@@ -0,0 +1,201 @@
+                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
\ No newline at end of file
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/README.md b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/README.md
new file mode 100644
index 0000000..224069b
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/README.md
@@ -0,0 +1,84 @@
+# Go gRPC Middleware
+
+[![Travis Build](https://travis-ci.org/grpc-ecosystem/go-grpc-middleware.svg?branch=master)](https://travis-ci.org/grpc-ecosystem/go-grpc-middleware)
+[![Go Report Card](https://goreportcard.com/badge/github.com/grpc-ecosystem/go-grpc-middleware)](https://goreportcard.com/report/github.com/grpc-ecosystem/go-grpc-middleware)
+[![GoDoc](http://img.shields.io/badge/GoDoc-Reference-blue.svg)](https://godoc.org/github.com/grpc-ecosystem/go-grpc-middleware)
+[![SourceGraph](https://sourcegraph.com/github.com/grpc-ecosystem/go-grpc-middleware/-/badge.svg)](https://sourcegraph.com/github.com/grpc-ecosystem/go-grpc-middleware/?badge)
+[![codecov](https://codecov.io/gh/grpc-ecosystem/go-grpc-middleware/branch/master/graph/badge.svg)](https://codecov.io/gh/grpc-ecosystem/go-grpc-middleware)
+[![Apache 2.0 License](https://img.shields.io/badge/License-Apache%202.0-blue.svg)](LICENSE)
+[![quality: production](https://img.shields.io/badge/quality-production-orange.svg)](#status)
+[![Slack](slack.png)](https://join.slack.com/t/improbable-eng/shared_invite/enQtMzQ1ODcyMzQ5MjM4LWY5ZWZmNGM2ODc5MmViNmQ3ZTA3ZTY3NzQwOTBlMTkzZmIxZTIxODk0OWU3YjZhNWVlNDU3MDlkZGViZjhkMjc)
+
+[gRPC Go](https://github.com/grpc/grpc-go) Middleware: interceptors, helpers, utilities.
+
+## Middleware
+
+[gRPC Go](https://github.com/grpc/grpc-go) recently acquired support for
+Interceptors, i.e. [middleware](https://medium.com/@matryer/writing-middleware-in-golang-and-how-go-makes-it-so-much-fun-4375c1246e81#.gv7tdlghs) 
+that is executed either on the gRPC Server before the request is passed onto the user's application logic, or on the gRPC client either around the user call. It is a perfect way to implement
+common patterns: auth, logging, message, validation, retries or monitoring.
+
+These are generic building blocks that make it easy to build multiple microservices easily.
+The purpose of this repository is to act as a go-to point for such reusable functionality. It contains
+some of them itself, but also will link to useful external repos.
+
+`grpc_middleware` itself provides support for chaining interceptors, here's an example:
+
+```go
+import "github.com/grpc-ecosystem/go-grpc-middleware"
+
+myServer := grpc.NewServer(
+    grpc.StreamInterceptor(grpc_middleware.ChainStreamServer(
+        grpc_ctxtags.StreamServerInterceptor(),
+        grpc_opentracing.StreamServerInterceptor(),
+        grpc_prometheus.StreamServerInterceptor,
+        grpc_zap.StreamServerInterceptor(zapLogger),
+        grpc_auth.StreamServerInterceptor(myAuthFunction),
+        grpc_recovery.StreamServerInterceptor(),
+    )),
+    grpc.UnaryInterceptor(grpc_middleware.ChainUnaryServer(
+        grpc_ctxtags.UnaryServerInterceptor(),
+        grpc_opentracing.UnaryServerInterceptor(),
+        grpc_prometheus.UnaryServerInterceptor,
+        grpc_zap.UnaryServerInterceptor(zapLogger),
+        grpc_auth.UnaryServerInterceptor(myAuthFunction),
+        grpc_recovery.UnaryServerInterceptor(),
+    )),
+)
+```
+
+## Interceptors
+
+*Please send a PR to add new interceptors or middleware to this list*
+
+#### Auth
+   * [`grpc_auth`](auth) - a customizable (via `AuthFunc`) piece of auth middleware 
+
+#### Logging
+   * [`grpc_ctxtags`](tags/) - a library that adds a `Tag` map to context, with data populated from request body
+   * [`grpc_zap`](logging/zap/) - integration of [zap](https://github.com/uber-go/zap) logging library into gRPC handlers.
+   * [`grpc_logrus`](logging/logrus/) - integration of [logrus](https://github.com/sirupsen/logrus) logging library into gRPC handlers.
+
+
+#### Monitoring
+   * [`grpc_prometheus`⚡](https://github.com/grpc-ecosystem/go-grpc-prometheus) - Prometheus client-side and server-side monitoring middleware
+   * [`otgrpc`⚡](https://github.com/grpc-ecosystem/grpc-opentracing/tree/master/go/otgrpc) - [OpenTracing](http://opentracing.io/) client-side and server-side interceptors
+   * [`grpc_opentracing`](tracing/opentracing) - [OpenTracing](http://opentracing.io/) client-side and server-side interceptors with support for streaming and handler-returned tags
+
+#### Client
+   * [`grpc_retry`](retry/) - a generic gRPC response code retry mechanism, client-side middleware
+
+#### Server
+   * [`grpc_validator`](validator/) - codegen inbound message validation from `.proto` options
+   * [`grpc_recovery`](recovery/) - turn panics into gRPC errors
+
+
+## Status
+
+This code has been running in *production* since May 2016 as the basis of the gRPC micro services stack at [Improbable](https://improbable.io).
+
+Additional tooling will be added, and contributions are welcome.
+
+## License
+
+`go-grpc-middleware` is released under the Apache 2.0 license. See the [LICENSE](LICENSE) file for details.
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/chain.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/chain.go
new file mode 100644
index 0000000..45a2f5f
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/chain.go
@@ -0,0 +1,183 @@
+// Copyright 2016 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+// gRPC Server Interceptor chaining middleware.
+
+package grpc_middleware
+
+import (
+	"golang.org/x/net/context"
+	"google.golang.org/grpc"
+)
+
+// ChainUnaryServer creates a single interceptor out of a chain of many interceptors.
+//
+// Execution is done in left-to-right order, including passing of context.
+// For example ChainUnaryServer(one, two, three) will execute one before two before three, and three
+// will see context changes of one and two.
+func ChainUnaryServer(interceptors ...grpc.UnaryServerInterceptor) grpc.UnaryServerInterceptor {
+	n := len(interceptors)
+
+	if n > 1 {
+		lastI := n - 1
+		return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
+			var (
+				chainHandler grpc.UnaryHandler
+				curI         int
+			)
+
+			chainHandler = func(currentCtx context.Context, currentReq interface{}) (interface{}, error) {
+				if curI == lastI {
+					return handler(currentCtx, currentReq)
+				}
+				curI++
+				resp, err := interceptors[curI](currentCtx, currentReq, info, chainHandler)
+				curI--
+				return resp, err
+			}
+
+			return interceptors[0](ctx, req, info, chainHandler)
+		}
+	}
+
+	if n == 1 {
+		return interceptors[0]
+	}
+
+	// n == 0; Dummy interceptor maintained for backward compatibility to avoid returning nil.
+	return func(ctx context.Context, req interface{}, _ *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
+		return handler(ctx, req)
+	}
+}
+
+// ChainStreamServer creates a single interceptor out of a chain of many interceptors.
+//
+// Execution is done in left-to-right order, including passing of context.
+// For example ChainUnaryServer(one, two, three) will execute one before two before three.
+// If you want to pass context between interceptors, use WrapServerStream.
+func ChainStreamServer(interceptors ...grpc.StreamServerInterceptor) grpc.StreamServerInterceptor {
+	n := len(interceptors)
+
+	if n > 1 {
+		lastI := n - 1
+		return func(srv interface{}, stream grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
+			var (
+				chainHandler grpc.StreamHandler
+				curI         int
+			)
+
+			chainHandler = func(currentSrv interface{}, currentStream grpc.ServerStream) error {
+				if curI == lastI {
+					return handler(currentSrv, currentStream)
+				}
+				curI++
+				err := interceptors[curI](currentSrv, currentStream, info, chainHandler)
+				curI--
+				return err
+			}
+
+			return interceptors[0](srv, stream, info, chainHandler)
+		}
+	}
+
+	if n == 1 {
+		return interceptors[0]
+	}
+
+	// n == 0; Dummy interceptor maintained for backward compatibility to avoid returning nil.
+	return func(srv interface{}, stream grpc.ServerStream, _ *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
+		return handler(srv, stream)
+	}
+}
+
+// ChainUnaryClient creates a single interceptor out of a chain of many interceptors.
+//
+// Execution is done in left-to-right order, including passing of context.
+// For example ChainUnaryClient(one, two, three) will execute one before two before three.
+func ChainUnaryClient(interceptors ...grpc.UnaryClientInterceptor) grpc.UnaryClientInterceptor {
+	n := len(interceptors)
+
+	if n > 1 {
+		lastI := n - 1
+		return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
+			var (
+				chainHandler grpc.UnaryInvoker
+				curI         int
+			)
+
+			chainHandler = func(currentCtx context.Context, currentMethod string, currentReq, currentRepl interface{}, currentConn *grpc.ClientConn, currentOpts ...grpc.CallOption) error {
+				if curI == lastI {
+					return invoker(currentCtx, currentMethod, currentReq, currentRepl, currentConn, currentOpts...)
+				}
+				curI++
+				err := interceptors[curI](currentCtx, currentMethod, currentReq, currentRepl, currentConn, chainHandler, currentOpts...)
+				curI--
+				return err
+			}
+
+			return interceptors[0](ctx, method, req, reply, cc, chainHandler, opts...)
+		}
+	}
+
+	if n == 1 {
+		return interceptors[0]
+	}
+
+	// n == 0; Dummy interceptor maintained for backward compatibility to avoid returning nil.
+	return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
+		return invoker(ctx, method, req, reply, cc, opts...)
+	}
+}
+
+// ChainStreamClient creates a single interceptor out of a chain of many interceptors.
+//
+// Execution is done in left-to-right order, including passing of context.
+// For example ChainStreamClient(one, two, three) will execute one before two before three.
+func ChainStreamClient(interceptors ...grpc.StreamClientInterceptor) grpc.StreamClientInterceptor {
+	n := len(interceptors)
+
+	if n > 1 {
+		lastI := n - 1
+		return func(ctx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) {
+			var (
+				chainHandler grpc.Streamer
+				curI         int
+			)
+
+			chainHandler = func(currentCtx context.Context, currentDesc *grpc.StreamDesc, currentConn *grpc.ClientConn, currentMethod string, currentOpts ...grpc.CallOption) (grpc.ClientStream, error) {
+				if curI == lastI {
+					return streamer(currentCtx, currentDesc, currentConn, currentMethod, currentOpts...)
+				}
+				curI++
+				stream, err := interceptors[curI](currentCtx, currentDesc, currentConn, currentMethod, chainHandler, currentOpts...)
+				curI--
+				return stream, err
+			}
+
+			return interceptors[0](ctx, desc, cc, method, chainHandler, opts...)
+		}
+	}
+
+	if n == 1 {
+		return interceptors[0]
+	}
+
+	// n == 0; Dummy interceptor maintained for backward compatibility to avoid returning nil.
+	return func(ctx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) {
+		return streamer(ctx, desc, cc, method, opts...)
+	}
+}
+
+// Chain creates a single interceptor out of a chain of many interceptors.
+//
+// WithUnaryServerChain is a grpc.Server config option that accepts multiple unary interceptors.
+// Basically syntactic sugar.
+func WithUnaryServerChain(interceptors ...grpc.UnaryServerInterceptor) grpc.ServerOption {
+	return grpc.UnaryInterceptor(ChainUnaryServer(interceptors...))
+}
+
+// WithStreamServerChain is a grpc.Server config option that accepts multiple stream interceptors.
+// Basically syntactic sugar.
+func WithStreamServerChain(interceptors ...grpc.StreamServerInterceptor) grpc.ServerOption {
+	return grpc.StreamInterceptor(ChainStreamServer(interceptors...))
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/doc.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/doc.go
new file mode 100644
index 0000000..7168950
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/doc.go
@@ -0,0 +1,69 @@
+// Copyright 2016 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+/*
+`grpc_middleware` is a collection of gRPC middleware packages: interceptors, helpers and tools.
+
+Middleware
+
+gRPC is a fantastic RPC middleware, which sees a lot of adoption in the Golang world. However, the
+upstream gRPC codebase is relatively bare bones.
+
+This package, and most of its child packages provides commonly needed middleware for gRPC:
+client-side interceptors for retires, server-side interceptors for input validation and auth,
+functions for chaining said interceptors, metadata convenience methods and more.
+
+Chaining
+
+By default, gRPC doesn't allow one to have more than one interceptor either on the client nor on
+the server side. `grpc_middleware` provides convenient chaining methods
+
+Simple way of turning a multiple interceptors into a single interceptor. Here's an example for
+server chaining:
+
+	myServer := grpc.NewServer(
+	    grpc.StreamInterceptor(grpc_middleware.ChainStreamServer(loggingStream, monitoringStream, authStream)),
+	    grpc.UnaryInterceptor(grpc_middleware.ChainUnaryServer(loggingUnary, monitoringUnary, authUnary),
+	)
+
+These interceptors will be executed from left to right: logging, monitoring and auth.
+
+Here's an example for client side chaining:
+
+	clientConn, err = grpc.Dial(
+	    address,
+	        grpc.WithUnaryInterceptor(grpc_middleware.ChainUnaryClient(monitoringClientUnary, retryUnary)),
+	        grpc.WithStreamInterceptor(grpc_middleware.ChainStreamClient(monitoringClientStream, retryStream)),
+	)
+	client = pb_testproto.NewTestServiceClient(clientConn)
+	resp, err := client.PingEmpty(s.ctx, &myservice.Request{Msg: "hello"})
+
+These interceptors will be executed from left to right: monitoring and then retry logic.
+
+The retry interceptor will call every interceptor that follows it whenever when a retry happens.
+
+Writing Your Own
+
+Implementing your own interceptor is pretty trivial: there are interfaces for that. But the interesting
+bit exposing common data to handlers (and other middleware), similarly to HTTP Middleware design.
+For example, you may want to pass the identity of the caller from the auth interceptor all the way
+to the handling function.
+
+For example, a client side interceptor example for auth looks like:
+
+	func FakeAuthUnaryInterceptor(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
+	   newCtx := context.WithValue(ctx, "user_id", "john@example.com")
+	   return handler(newCtx, req)
+	}
+
+Unfortunately, it's not as easy for streaming RPCs. These have the `context.Context` embedded within
+the `grpc.ServerStream` object. To pass values through context, a wrapper (`WrappedServerStream`) is
+needed. For example:
+
+	func FakeAuthStreamingInterceptor(srv interface{}, stream grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
+	   newStream := grpc_middleware.WrapServerStream(stream)
+	   newStream.WrappedContext = context.WithValue(ctx, "user_id", "john@example.com")
+	   return handler(srv, stream)
+	}
+*/
+package grpc_middleware
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/makefile b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/makefile
new file mode 100644
index 0000000..51dc5b8
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/makefile
@@ -0,0 +1,16 @@
+SHELL=/bin/bash
+
+GOFILES_NOVENDOR = $(shell go list ./... | grep -v /vendor/)
+
+all: vet fmt test
+
+fmt:
+	go fmt $(GOFILES_NOVENDOR)
+
+vet:
+	go vet $(GOFILES_NOVENDOR)
+
+test: vet
+	./scripts/test_all.sh
+
+.PHONY: all test
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/slack.png b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/slack.png
new file mode 100644
index 0000000..cc8f9a6
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/slack.png
Binary files differ
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-middleware/wrappers.go b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/wrappers.go
new file mode 100644
index 0000000..597b862
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-middleware/wrappers.go
@@ -0,0 +1,29 @@
+// Copyright 2016 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+package grpc_middleware
+
+import (
+	"golang.org/x/net/context"
+	"google.golang.org/grpc"
+)
+
+// WrappedServerStream is a thin wrapper around grpc.ServerStream that allows modifying context.
+type WrappedServerStream struct {
+	grpc.ServerStream
+	// WrappedContext is the wrapper's own Context. You can assign it.
+	WrappedContext context.Context
+}
+
+// Context returns the wrapper's WrappedContext, overwriting the nested grpc.ServerStream.Context()
+func (w *WrappedServerStream) Context() context.Context {
+	return w.WrappedContext
+}
+
+// WrapServerStream returns a ServerStream that has the ability to overwrite context.
+func WrapServerStream(stream grpc.ServerStream) *WrappedServerStream {
+	if existing, ok := stream.(*WrappedServerStream); ok {
+		return existing
+	}
+	return &WrappedServerStream{ServerStream: stream, WrappedContext: stream.Context()}
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/.gitignore b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/.gitignore
new file mode 100644
index 0000000..2233cff
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/.gitignore
@@ -0,0 +1,201 @@
+#vendor
+vendor/
+
+# Created by .ignore support plugin (hsz.mobi)
+coverage.txt
+### Go template
+# Compiled Object files, Static and Dynamic libs (Shared Objects)
+*.o
+*.a
+*.so
+
+# Folders
+_obj
+_test
+
+# Architecture specific extensions/prefixes
+*.[568vq]
+[568vq].out
+
+*.cgo1.go
+*.cgo2.c
+_cgo_defun.c
+_cgo_gotypes.go
+_cgo_export.*
+
+_testmain.go
+
+*.exe
+*.test
+*.prof
+### Windows template
+# Windows image file caches
+Thumbs.db
+ehthumbs.db
+
+# Folder config file
+Desktop.ini
+
+# Recycle Bin used on file shares
+$RECYCLE.BIN/
+
+# Windows Installer files
+*.cab
+*.msi
+*.msm
+*.msp
+
+# Windows shortcuts
+*.lnk
+### Kate template
+# Swap Files #
+.*.kate-swp
+.swp.*
+### SublimeText template
+# cache files for sublime text
+*.tmlanguage.cache
+*.tmPreferences.cache
+*.stTheme.cache
+
+# workspace files are user-specific
+*.sublime-workspace
+
+# project files should be checked into the repository, unless a significant
+# proportion of contributors will probably not be using SublimeText
+# *.sublime-project
+
+# sftp configuration file
+sftp-config.json
+### Linux template
+*~
+
+# temporary files which can be created if a process still has a handle open of a deleted file
+.fuse_hidden*
+
+# KDE directory preferences
+.directory
+
+# Linux trash folder which might appear on any partition or disk
+.Trash-*
+### JetBrains template
+# Covers JetBrains IDEs: IntelliJ, RubyMine, PhpStorm, AppCode, PyCharm, CLion, Android Studio and Webstorm
+# Reference: https://intellij-support.jetbrains.com/hc/en-us/articles/206544839
+
+# User-specific stuff:
+.idea
+.idea/tasks.xml
+.idea/dictionaries
+.idea/vcs.xml
+.idea/jsLibraryMappings.xml
+
+# Sensitive or high-churn files:
+.idea/dataSources.ids
+.idea/dataSources.xml
+.idea/dataSources.local.xml
+.idea/sqlDataSources.xml
+.idea/dynamic.xml
+.idea/uiDesigner.xml
+
+# Gradle:
+.idea/gradle.xml
+.idea/libraries
+
+# Mongo Explorer plugin:
+.idea/mongoSettings.xml
+
+## File-based project format:
+*.iws
+
+## Plugin-specific files:
+
+# IntelliJ
+/out/
+
+# mpeltonen/sbt-idea plugin
+.idea_modules/
+
+# JIRA plugin
+atlassian-ide-plugin.xml
+
+# Crashlytics plugin (for Android Studio and IntelliJ)
+com_crashlytics_export_strings.xml
+crashlytics.properties
+crashlytics-build.properties
+fabric.properties
+### Xcode template
+# Xcode
+#
+# gitignore contributors: remember to update Global/Xcode.gitignore, Objective-C.gitignore & Swift.gitignore
+
+## Build generated
+build/
+DerivedData/
+
+## Various settings
+*.pbxuser
+!default.pbxuser
+*.mode1v3
+!default.mode1v3
+*.mode2v3
+!default.mode2v3
+*.perspectivev3
+!default.perspectivev3
+xcuserdata/
+
+## Other
+*.moved-aside
+*.xccheckout
+*.xcscmblueprint
+### Eclipse template
+
+.metadata
+bin/
+tmp/
+*.tmp
+*.bak
+*.swp
+*~.nib
+local.properties
+.settings/
+.loadpath
+.recommenders
+
+# Eclipse Core
+.project
+
+# External tool builders
+.externalToolBuilders/
+
+# Locally stored "Eclipse launch configurations"
+*.launch
+
+# PyDev specific (Python IDE for Eclipse)
+*.pydevproject
+
+# CDT-specific (C/C++ Development Tooling)
+.cproject
+
+# JDT-specific (Eclipse Java Development Tools)
+.classpath
+
+# Java annotation processor (APT)
+.factorypath
+
+# PDT-specific (PHP Development Tools)
+.buildpath
+
+# sbteclipse plugin
+.target
+
+# Tern plugin
+.tern-project
+
+# TeXlipse plugin
+.texlipse
+
+# STS (Spring Tool Suite)
+.springBeans
+
+# Code Recommenders
+.recommenders/
+
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/.travis.yml b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/.travis.yml
new file mode 100644
index 0000000..2a845b9
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/.travis.yml
@@ -0,0 +1,25 @@
+sudo: false
+language: go
+# * github.com/grpc/grpc-go still supports go1.6
+#   - When we drop support for go1.6 we can remove golang.org/x/net/context
+#     below as it is part of the Go std library since go1.7
+# * github.com/prometheus/client_golang already requires at least go1.7 since
+#   September 2017
+go:
+  - 1.6.x
+  - 1.7.x
+  - 1.8.x
+  - 1.9.x
+  - 1.10.x
+  - master
+
+install:
+  - go get github.com/prometheus/client_golang/prometheus
+  - go get google.golang.org/grpc
+  - go get golang.org/x/net/context
+  - go get github.com/stretchr/testify
+script:
+ - make test 
+
+after_success:
+  - bash <(curl -s https://codecov.io/bash)
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/CHANGELOG.md b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/CHANGELOG.md
new file mode 100644
index 0000000..19a8059
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/CHANGELOG.md
@@ -0,0 +1,24 @@
+# Changelog
+All notable changes to this project will be documented in this file.
+
+The format is based on [Keep a Changelog](http://keepachangelog.com/en/1.0.0/)
+and this project adheres to [Semantic Versioning](http://semver.org/spec/v2.0.0.html).
+
+## [Unreleased]
+
+## [1.2.0](https://github.com/grpc-ecosystem/go-grpc-prometheus/releases/tag/v1.2.0) - 2018-06-04
+
+### Added
+
+* Provide metrics object as `prometheus.Collector`, for conventional metric registration.
+* Support non-default/global Prometheus registry.
+* Allow configuring counters with `prometheus.CounterOpts`.
+
+### Changed
+
+* Remove usage of deprecated `grpc.Code()`.
+* Remove usage of deprecated `grpc.Errorf` and replace with `status.Errorf`.
+
+---
+
+This changelog was started with version `v1.2.0`, for earlier versions refer to the respective [GitHub releases](https://github.com/grpc-ecosystem/go-grpc-prometheus/releases).
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/LICENSE b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/LICENSE
new file mode 100644
index 0000000..b2b0650
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/LICENSE
@@ -0,0 +1,201 @@
+                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
\ No newline at end of file
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/README.md b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/README.md
new file mode 100644
index 0000000..499c583
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/README.md
@@ -0,0 +1,247 @@
+# Go gRPC Interceptors for Prometheus monitoring 
+
+[![Travis Build](https://travis-ci.org/grpc-ecosystem/go-grpc-prometheus.svg)](https://travis-ci.org/grpc-ecosystem/go-grpc-prometheus)
+[![Go Report Card](https://goreportcard.com/badge/github.com/grpc-ecosystem/go-grpc-prometheus)](http://goreportcard.com/report/grpc-ecosystem/go-grpc-prometheus)
+[![GoDoc](http://img.shields.io/badge/GoDoc-Reference-blue.svg)](https://godoc.org/github.com/grpc-ecosystem/go-grpc-prometheus)
+[![SourceGraph](https://sourcegraph.com/github.com/grpc-ecosystem/go-grpc-prometheus/-/badge.svg)](https://sourcegraph.com/github.com/grpc-ecosystem/go-grpc-prometheus/?badge)
+[![codecov](https://codecov.io/gh/grpc-ecosystem/go-grpc-prometheus/branch/master/graph/badge.svg)](https://codecov.io/gh/grpc-ecosystem/go-grpc-prometheus)
+[![Apache 2.0 License](https://img.shields.io/badge/License-Apache%202.0-blue.svg)](LICENSE)
+
+[Prometheus](https://prometheus.io/) monitoring for your [gRPC Go](https://github.com/grpc/grpc-go) servers and clients.
+
+A sister implementation for [gRPC Java](https://github.com/grpc/grpc-java) (same metrics, same semantics) is in [grpc-ecosystem/java-grpc-prometheus](https://github.com/grpc-ecosystem/java-grpc-prometheus).
+
+## Interceptors
+
+[gRPC Go](https://github.com/grpc/grpc-go) recently acquired support for Interceptors, i.e. middleware that is executed
+by a gRPC Server before the request is passed onto the user's application logic. It is a perfect way to implement
+common patterns: auth, logging and... monitoring.
+
+To use Interceptors in chains, please see [`go-grpc-middleware`](https://github.com/mwitkow/go-grpc-middleware).
+
+## Usage
+
+There are two types of interceptors: client-side and server-side. This package provides monitoring Interceptors for both.
+
+### Server-side
+
+```go
+import "github.com/grpc-ecosystem/go-grpc-prometheus"
+...
+    // Initialize your gRPC server's interceptor.
+    myServer := grpc.NewServer(
+        grpc.StreamInterceptor(grpc_prometheus.StreamServerInterceptor),
+        grpc.UnaryInterceptor(grpc_prometheus.UnaryServerInterceptor),
+    )
+    // Register your gRPC service implementations.
+    myservice.RegisterMyServiceServer(s.server, &myServiceImpl{})
+    // After all your registrations, make sure all of the Prometheus metrics are initialized.
+    grpc_prometheus.Register(myServer)
+    // Register Prometheus metrics handler.    
+    http.Handle("/metrics", promhttp.Handler())
+...
+```
+
+### Client-side
+
+```go
+import "github.com/grpc-ecosystem/go-grpc-prometheus"
+...
+   clientConn, err = grpc.Dial(
+       address,
+		   grpc.WithUnaryInterceptor(grpc_prometheus.UnaryClientInterceptor),
+		   grpc.WithStreamInterceptor(grpc_prometheus.StreamClientInterceptor)
+   )
+   client = pb_testproto.NewTestServiceClient(clientConn)
+   resp, err := client.PingEmpty(s.ctx, &myservice.Request{Msg: "hello"})
+...
+```
+
+# Metrics
+
+## Labels
+
+All server-side metrics start with `grpc_server` as Prometheus subsystem name. All client-side metrics start with `grpc_client`. Both of them have mirror-concepts. Similarly all methods
+contain the same rich labels:
+  
+  * `grpc_service` - the [gRPC service](http://www.grpc.io/docs/#defining-a-service) name, which is the combination of protobuf `package` and
+    the `grpc_service` section name. E.g. for `package = mwitkow.testproto` and 
+     `service TestService` the label will be `grpc_service="mwitkow.testproto.TestService"`
+  * `grpc_method` - the name of the method called on the gRPC service. E.g.  
+    `grpc_method="Ping"`
+  * `grpc_type` - the gRPC [type of request](http://www.grpc.io/docs/guides/concepts.html#rpc-life-cycle). 
+    Differentiating between the two is important especially for latency measurements.
+
+     - `unary` is single request, single response RPC
+     - `client_stream` is a multi-request, single response RPC
+     - `server_stream` is a single request, multi-response RPC
+     - `bidi_stream` is a multi-request, multi-response RPC
+    
+
+Additionally for completed RPCs, the following labels are used:
+
+  * `grpc_code` - the human-readable [gRPC status code](https://github.com/grpc/grpc-go/blob/master/codes/codes.go).
+    The list of all statuses is to long, but here are some common ones:
+      
+      - `OK` - means the RPC was successful
+      - `IllegalArgument` - RPC contained bad values
+      - `Internal` - server-side error not disclosed to the clients
+      
+## Counters
+
+The counters and their up to date documentation is in [server_reporter.go](server_reporter.go) and [client_reporter.go](client_reporter.go) 
+the respective Prometheus handler (usually `/metrics`). 
+
+For the purpose of this documentation we will only discuss `grpc_server` metrics. The `grpc_client` ones contain mirror concepts.
+
+For simplicity, let's assume we're tracking a single server-side RPC call of [`mwitkow.testproto.TestService`](examples/testproto/test.proto),
+calling the method `PingList`. The call succeeds and returns 20 messages in the stream.
+
+First, immediately after the server receives the call it will increment the
+`grpc_server_started_total` and start the handling time clock (if histograms are enabled). 
+
+```jsoniq
+grpc_server_started_total{grpc_method="PingList",grpc_service="mwitkow.testproto.TestService",grpc_type="server_stream"} 1
+```
+
+Then the user logic gets invoked. It receives one message from the client containing the request 
+(it's a `server_stream`):
+
+```jsoniq
+grpc_server_msg_received_total{grpc_method="PingList",grpc_service="mwitkow.testproto.TestService",grpc_type="server_stream"} 1
+```
+
+The user logic may return an error, or send multiple messages back to the client. In this case, on 
+each of the 20 messages sent back, a counter will be incremented:
+
+```jsoniq
+grpc_server_msg_sent_total{grpc_method="PingList",grpc_service="mwitkow.testproto.TestService",grpc_type="server_stream"} 20
+```
+
+After the call completes, its status (`OK` or other [gRPC status code](https://github.com/grpc/grpc-go/blob/master/codes/codes.go)) 
+and the relevant call labels increment the `grpc_server_handled_total` counter.
+
+```jsoniq
+grpc_server_handled_total{grpc_code="OK",grpc_method="PingList",grpc_service="mwitkow.testproto.TestService",grpc_type="server_stream"} 1
+```
+
+## Histograms
+
+[Prometheus histograms](https://prometheus.io/docs/concepts/metric_types/#histogram) are a great way
+to measure latency distributions of your RPCs. However, since it is bad practice to have metrics
+of [high cardinality](https://prometheus.io/docs/practices/instrumentation/#do-not-overuse-labels)
+the latency monitoring metrics are disabled by default. To enable them please call the following
+in your server initialization code:
+
+```jsoniq
+grpc_prometheus.EnableHandlingTimeHistogram()
+```
+
+After the call completes, its handling time will be recorded in a [Prometheus histogram](https://prometheus.io/docs/concepts/metric_types/#histogram)
+variable `grpc_server_handling_seconds`. The histogram variable contains three sub-metrics:
+
+ * `grpc_server_handling_seconds_count` - the count of all completed RPCs by status and method 
+ * `grpc_server_handling_seconds_sum` - cumulative time of RPCs by status and method, useful for 
+   calculating average handling times
+ * `grpc_server_handling_seconds_bucket` - contains the counts of RPCs by status and method in respective
+   handling-time buckets. These buckets can be used by Prometheus to estimate SLAs (see [here](https://prometheus.io/docs/practices/histograms/))
+
+The counter values will look as follows:
+
+```jsoniq
+grpc_server_handling_seconds_bucket{grpc_code="OK",grpc_method="PingList",grpc_service="mwitkow.testproto.TestService",grpc_type="server_stream",le="0.005"} 1
+grpc_server_handling_seconds_bucket{grpc_code="OK",grpc_method="PingList",grpc_service="mwitkow.testproto.TestService",grpc_type="server_stream",le="0.01"} 1
+grpc_server_handling_seconds_bucket{grpc_code="OK",grpc_method="PingList",grpc_service="mwitkow.testproto.TestService",grpc_type="server_stream",le="0.025"} 1
+grpc_server_handling_seconds_bucket{grpc_code="OK",grpc_method="PingList",grpc_service="mwitkow.testproto.TestService",grpc_type="server_stream",le="0.05"} 1
+grpc_server_handling_seconds_bucket{grpc_code="OK",grpc_method="PingList",grpc_service="mwitkow.testproto.TestService",grpc_type="server_stream",le="0.1"} 1
+grpc_server_handling_seconds_bucket{grpc_code="OK",grpc_method="PingList",grpc_service="mwitkow.testproto.TestService",grpc_type="server_stream",le="0.25"} 1
+grpc_server_handling_seconds_bucket{grpc_code="OK",grpc_method="PingList",grpc_service="mwitkow.testproto.TestService",grpc_type="server_stream",le="0.5"} 1
+grpc_server_handling_seconds_bucket{grpc_code="OK",grpc_method="PingList",grpc_service="mwitkow.testproto.TestService",grpc_type="server_stream",le="1"} 1
+grpc_server_handling_seconds_bucket{grpc_code="OK",grpc_method="PingList",grpc_service="mwitkow.testproto.TestService",grpc_type="server_stream",le="2.5"} 1
+grpc_server_handling_seconds_bucket{grpc_code="OK",grpc_method="PingList",grpc_service="mwitkow.testproto.TestService",grpc_type="server_stream",le="5"} 1
+grpc_server_handling_seconds_bucket{grpc_code="OK",grpc_method="PingList",grpc_service="mwitkow.testproto.TestService",grpc_type="server_stream",le="10"} 1
+grpc_server_handling_seconds_bucket{grpc_code="OK",grpc_method="PingList",grpc_service="mwitkow.testproto.TestService",grpc_type="server_stream",le="+Inf"} 1
+grpc_server_handling_seconds_sum{grpc_code="OK",grpc_method="PingList",grpc_service="mwitkow.testproto.TestService",grpc_type="server_stream"} 0.0003866430000000001
+grpc_server_handling_seconds_count{grpc_code="OK",grpc_method="PingList",grpc_service="mwitkow.testproto.TestService",grpc_type="server_stream"} 1
+```
+
+
+## Useful query examples
+
+Prometheus philosophy is to provide raw metrics to the monitoring system, and
+let the aggregations be handled there. The verbosity of above metrics make it possible to have that
+flexibility. Here's a couple of useful monitoring queries:
+
+
+### request inbound rate
+```jsoniq
+sum(rate(grpc_server_started_total{job="foo"}[1m])) by (grpc_service)
+```
+For `job="foo"` (common label to differentiate between Prometheus monitoring targets), calculate the
+rate of requests per second (1 minute window) for each gRPC `grpc_service` that the job has. Please note
+how the `grpc_method` is being omitted here: all methods of a given gRPC service will be summed together.
+
+### unary request error rate
+```jsoniq
+sum(rate(grpc_server_handled_total{job="foo",grpc_type="unary",grpc_code!="OK"}[1m])) by (grpc_service)
+```
+For `job="foo"`, calculate the per-`grpc_service` rate of `unary` (1:1) RPCs that failed, i.e. the 
+ones that didn't finish with `OK` code.
+
+### unary request error percentage
+```jsoniq
+sum(rate(grpc_server_handled_total{job="foo",grpc_type="unary",grpc_code!="OK"}[1m])) by (grpc_service)
+ / 
+sum(rate(grpc_server_started_total{job="foo",grpc_type="unary"}[1m])) by (grpc_service)
+ * 100.0
+```
+For `job="foo"`, calculate the percentage of failed requests by service. It's easy to notice that
+this is a combination of the two above examples. This is an example of a query you would like to
+[alert on](https://prometheus.io/docs/alerting/rules/) in your system for SLA violations, e.g.
+"no more than 1% requests should fail".
+
+### average response stream size
+```jsoniq
+sum(rate(grpc_server_msg_sent_total{job="foo",grpc_type="server_stream"}[10m])) by (grpc_service)
+ /
+sum(rate(grpc_server_started_total{job="foo",grpc_type="server_stream"}[10m])) by (grpc_service)
+```
+For `job="foo"` what is the `grpc_service`-wide `10m` average of messages returned for all `
+server_stream` RPCs. This allows you to track the stream sizes returned by your system, e.g. allows 
+you to track when clients started to send "wide" queries that ret
+Note the divisor is the number of started RPCs, in order to account for in-flight requests.
+
+### 99%-tile latency of unary requests
+```jsoniq
+histogram_quantile(0.99, 
+  sum(rate(grpc_server_handling_seconds_bucket{job="foo",grpc_type="unary"}[5m])) by (grpc_service,le)
+)
+```
+For `job="foo"`, returns an 99%-tile [quantile estimation](https://prometheus.io/docs/practices/histograms/#quantiles)
+of the handling time of RPCs per service. Please note the `5m` rate, this means that the quantile
+estimation will take samples in a rolling `5m` window. When combined with other quantiles
+(e.g. 50%, 90%), this query gives you tremendous insight into the responsiveness of your system 
+(e.g. impact of caching).
+
+### percentage of slow unary queries (>250ms)
+```jsoniq
+100.0 - (
+sum(rate(grpc_server_handling_seconds_bucket{job="foo",grpc_type="unary",le="0.25"}[5m])) by (grpc_service)
+ / 
+sum(rate(grpc_server_handling_seconds_count{job="foo",grpc_type="unary"}[5m])) by (grpc_service)
+) * 100.0
+```
+For `job="foo"` calculate the by-`grpc_service` fraction of slow requests that took longer than `0.25` 
+seconds. This query is relatively complex, since the Prometheus aggregations use `le` (less or equal)
+buckets, meaning that counting "fast" requests fractions is easier. However, simple maths helps.
+This is an example of a query you would like to alert on in your system for SLA violations, 
+e.g. "less than 1% of requests are slower than 250ms".
+
+
+## Status
+
+This code has been used since August 2015 as the basis for monitoring of *production* gRPC micro services  at [Improbable](https://improbable.io).
+
+## License
+
+`go-grpc-prometheus` is released under the Apache 2.0 license. See the [LICENSE](LICENSE) file for details.
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/client.go b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/client.go
new file mode 100644
index 0000000..751a4c7
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/client.go
@@ -0,0 +1,39 @@
+// Copyright 2016 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+// gRPC Prometheus monitoring interceptors for client-side gRPC.
+
+package grpc_prometheus
+
+import (
+	prom "github.com/prometheus/client_golang/prometheus"
+)
+
+var (
+	// DefaultClientMetrics is the default instance of ClientMetrics. It is
+	// intended to be used in conjunction the default Prometheus metrics
+	// registry.
+	DefaultClientMetrics = NewClientMetrics()
+
+	// UnaryClientInterceptor is a gRPC client-side interceptor that provides Prometheus monitoring for Unary RPCs.
+	UnaryClientInterceptor = DefaultClientMetrics.UnaryClientInterceptor()
+
+	// StreamClientInterceptor is a gRPC client-side interceptor that provides Prometheus monitoring for Streaming RPCs.
+	StreamClientInterceptor = DefaultClientMetrics.StreamClientInterceptor()
+)
+
+func init() {
+	prom.MustRegister(DefaultClientMetrics.clientStartedCounter)
+	prom.MustRegister(DefaultClientMetrics.clientHandledCounter)
+	prom.MustRegister(DefaultClientMetrics.clientStreamMsgReceived)
+	prom.MustRegister(DefaultClientMetrics.clientStreamMsgSent)
+}
+
+// EnableClientHandlingTimeHistogram turns on recording of handling time of
+// RPCs. Histogram metrics can be very expensive for Prometheus to retain and
+// query. This function acts on the DefaultClientMetrics variable and the
+// default Prometheus metrics registry.
+func EnableClientHandlingTimeHistogram(opts ...HistogramOption) {
+	DefaultClientMetrics.EnableClientHandlingTimeHistogram(opts...)
+	prom.Register(DefaultClientMetrics.clientHandledHistogram)
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/client_metrics.go b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/client_metrics.go
new file mode 100644
index 0000000..9b476f9
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/client_metrics.go
@@ -0,0 +1,170 @@
+package grpc_prometheus
+
+import (
+	"io"
+
+	prom "github.com/prometheus/client_golang/prometheus"
+	"golang.org/x/net/context"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+)
+
+// ClientMetrics represents a collection of metrics to be registered on a
+// Prometheus metrics registry for a gRPC client.
+type ClientMetrics struct {
+	clientStartedCounter          *prom.CounterVec
+	clientHandledCounter          *prom.CounterVec
+	clientStreamMsgReceived       *prom.CounterVec
+	clientStreamMsgSent           *prom.CounterVec
+	clientHandledHistogramEnabled bool
+	clientHandledHistogramOpts    prom.HistogramOpts
+	clientHandledHistogram        *prom.HistogramVec
+}
+
+// NewClientMetrics returns a ClientMetrics object. Use a new instance of
+// ClientMetrics when not using the default Prometheus metrics registry, for
+// example when wanting to control which metrics are added to a registry as
+// opposed to automatically adding metrics via init functions.
+func NewClientMetrics(counterOpts ...CounterOption) *ClientMetrics {
+	opts := counterOptions(counterOpts)
+	return &ClientMetrics{
+		clientStartedCounter: prom.NewCounterVec(
+			opts.apply(prom.CounterOpts{
+				Name: "grpc_client_started_total",
+				Help: "Total number of RPCs started on the client.",
+			}), []string{"grpc_type", "grpc_service", "grpc_method"}),
+
+		clientHandledCounter: prom.NewCounterVec(
+			opts.apply(prom.CounterOpts{
+				Name: "grpc_client_handled_total",
+				Help: "Total number of RPCs completed by the client, regardless of success or failure.",
+			}), []string{"grpc_type", "grpc_service", "grpc_method", "grpc_code"}),
+
+		clientStreamMsgReceived: prom.NewCounterVec(
+			opts.apply(prom.CounterOpts{
+				Name: "grpc_client_msg_received_total",
+				Help: "Total number of RPC stream messages received by the client.",
+			}), []string{"grpc_type", "grpc_service", "grpc_method"}),
+
+		clientStreamMsgSent: prom.NewCounterVec(
+			opts.apply(prom.CounterOpts{
+				Name: "grpc_client_msg_sent_total",
+				Help: "Total number of gRPC stream messages sent by the client.",
+			}), []string{"grpc_type", "grpc_service", "grpc_method"}),
+
+		clientHandledHistogramEnabled: false,
+		clientHandledHistogramOpts: prom.HistogramOpts{
+			Name:    "grpc_client_handling_seconds",
+			Help:    "Histogram of response latency (seconds) of the gRPC until it is finished by the application.",
+			Buckets: prom.DefBuckets,
+		},
+		clientHandledHistogram: nil,
+	}
+}
+
+// Describe sends the super-set of all possible descriptors of metrics
+// collected by this Collector to the provided channel and returns once
+// the last descriptor has been sent.
+func (m *ClientMetrics) Describe(ch chan<- *prom.Desc) {
+	m.clientStartedCounter.Describe(ch)
+	m.clientHandledCounter.Describe(ch)
+	m.clientStreamMsgReceived.Describe(ch)
+	m.clientStreamMsgSent.Describe(ch)
+	if m.clientHandledHistogramEnabled {
+		m.clientHandledHistogram.Describe(ch)
+	}
+}
+
+// Collect is called by the Prometheus registry when collecting
+// metrics. The implementation sends each collected metric via the
+// provided channel and returns once the last metric has been sent.
+func (m *ClientMetrics) Collect(ch chan<- prom.Metric) {
+	m.clientStartedCounter.Collect(ch)
+	m.clientHandledCounter.Collect(ch)
+	m.clientStreamMsgReceived.Collect(ch)
+	m.clientStreamMsgSent.Collect(ch)
+	if m.clientHandledHistogramEnabled {
+		m.clientHandledHistogram.Collect(ch)
+	}
+}
+
+// EnableClientHandlingTimeHistogram turns on recording of handling time of RPCs.
+// Histogram metrics can be very expensive for Prometheus to retain and query.
+func (m *ClientMetrics) EnableClientHandlingTimeHistogram(opts ...HistogramOption) {
+	for _, o := range opts {
+		o(&m.clientHandledHistogramOpts)
+	}
+	if !m.clientHandledHistogramEnabled {
+		m.clientHandledHistogram = prom.NewHistogramVec(
+			m.clientHandledHistogramOpts,
+			[]string{"grpc_type", "grpc_service", "grpc_method"},
+		)
+	}
+	m.clientHandledHistogramEnabled = true
+}
+
+// UnaryClientInterceptor is a gRPC client-side interceptor that provides Prometheus monitoring for Unary RPCs.
+func (m *ClientMetrics) UnaryClientInterceptor() func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
+	return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error {
+		monitor := newClientReporter(m, Unary, method)
+		monitor.SentMessage()
+		err := invoker(ctx, method, req, reply, cc, opts...)
+		if err != nil {
+			monitor.ReceivedMessage()
+		}
+		st, _ := status.FromError(err)
+		monitor.Handled(st.Code())
+		return err
+	}
+}
+
+// StreamClientInterceptor is a gRPC client-side interceptor that provides Prometheus monitoring for Streaming RPCs.
+func (m *ClientMetrics) StreamClientInterceptor() func(ctx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) {
+	return func(ctx context.Context, desc *grpc.StreamDesc, cc *grpc.ClientConn, method string, streamer grpc.Streamer, opts ...grpc.CallOption) (grpc.ClientStream, error) {
+		monitor := newClientReporter(m, clientStreamType(desc), method)
+		clientStream, err := streamer(ctx, desc, cc, method, opts...)
+		if err != nil {
+			st, _ := status.FromError(err)
+			monitor.Handled(st.Code())
+			return nil, err
+		}
+		return &monitoredClientStream{clientStream, monitor}, nil
+	}
+}
+
+func clientStreamType(desc *grpc.StreamDesc) grpcType {
+	if desc.ClientStreams && !desc.ServerStreams {
+		return ClientStream
+	} else if !desc.ClientStreams && desc.ServerStreams {
+		return ServerStream
+	}
+	return BidiStream
+}
+
+// monitoredClientStream wraps grpc.ClientStream allowing each Sent/Recv of message to increment counters.
+type monitoredClientStream struct {
+	grpc.ClientStream
+	monitor *clientReporter
+}
+
+func (s *monitoredClientStream) SendMsg(m interface{}) error {
+	err := s.ClientStream.SendMsg(m)
+	if err == nil {
+		s.monitor.SentMessage()
+	}
+	return err
+}
+
+func (s *monitoredClientStream) RecvMsg(m interface{}) error {
+	err := s.ClientStream.RecvMsg(m)
+	if err == nil {
+		s.monitor.ReceivedMessage()
+	} else if err == io.EOF {
+		s.monitor.Handled(codes.OK)
+	} else {
+		st, _ := status.FromError(err)
+		s.monitor.Handled(st.Code())
+	}
+	return err
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/client_reporter.go b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/client_reporter.go
new file mode 100644
index 0000000..cbf1532
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/client_reporter.go
@@ -0,0 +1,46 @@
+// Copyright 2016 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+package grpc_prometheus
+
+import (
+	"time"
+
+	"google.golang.org/grpc/codes"
+)
+
+type clientReporter struct {
+	metrics     *ClientMetrics
+	rpcType     grpcType
+	serviceName string
+	methodName  string
+	startTime   time.Time
+}
+
+func newClientReporter(m *ClientMetrics, rpcType grpcType, fullMethod string) *clientReporter {
+	r := &clientReporter{
+		metrics: m,
+		rpcType: rpcType,
+	}
+	if r.metrics.clientHandledHistogramEnabled {
+		r.startTime = time.Now()
+	}
+	r.serviceName, r.methodName = splitMethodName(fullMethod)
+	r.metrics.clientStartedCounter.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Inc()
+	return r
+}
+
+func (r *clientReporter) ReceivedMessage() {
+	r.metrics.clientStreamMsgReceived.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Inc()
+}
+
+func (r *clientReporter) SentMessage() {
+	r.metrics.clientStreamMsgSent.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Inc()
+}
+
+func (r *clientReporter) Handled(code codes.Code) {
+	r.metrics.clientHandledCounter.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName, code.String()).Inc()
+	if r.metrics.clientHandledHistogramEnabled {
+		r.metrics.clientHandledHistogram.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Observe(time.Since(r.startTime).Seconds())
+	}
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/makefile b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/makefile
new file mode 100644
index 0000000..74c0842
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/makefile
@@ -0,0 +1,16 @@
+SHELL="/bin/bash"
+
+GOFILES_NOVENDOR = $(shell go list ./... | grep -v /vendor/)
+
+all: vet fmt test
+
+fmt:
+	go fmt $(GOFILES_NOVENDOR)
+
+vet:
+	go vet $(GOFILES_NOVENDOR)
+
+test: vet
+	./scripts/test_all.sh
+
+.PHONY: all vet test
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/metric_options.go b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/metric_options.go
new file mode 100644
index 0000000..9d51aec
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/metric_options.go
@@ -0,0 +1,41 @@
+package grpc_prometheus
+
+import (
+	prom "github.com/prometheus/client_golang/prometheus"
+)
+
+// A CounterOption lets you add options to Counter metrics using With* funcs.
+type CounterOption func(*prom.CounterOpts)
+
+type counterOptions []CounterOption
+
+func (co counterOptions) apply(o prom.CounterOpts) prom.CounterOpts {
+	for _, f := range co {
+		f(&o)
+	}
+	return o
+}
+
+// WithConstLabels allows you to add ConstLabels to Counter metrics.
+func WithConstLabels(labels prom.Labels) CounterOption {
+	return func(o *prom.CounterOpts) {
+		o.ConstLabels = labels
+	}
+}
+
+// A HistogramOption lets you add options to Histogram metrics using With*
+// funcs.
+type HistogramOption func(*prom.HistogramOpts)
+
+// WithHistogramBuckets allows you to specify custom bucket ranges for histograms if EnableHandlingTimeHistogram is on.
+func WithHistogramBuckets(buckets []float64) HistogramOption {
+	return func(o *prom.HistogramOpts) { o.Buckets = buckets }
+}
+
+// WithHistogramConstLabels allows you to add custom ConstLabels to
+// histograms metrics.
+func WithHistogramConstLabels(labels prom.Labels) HistogramOption {
+	return func(o *prom.HistogramOpts) {
+		o.ConstLabels = labels
+	}
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/server.go b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/server.go
new file mode 100644
index 0000000..322f990
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/server.go
@@ -0,0 +1,48 @@
+// Copyright 2016 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+// gRPC Prometheus monitoring interceptors for server-side gRPC.
+
+package grpc_prometheus
+
+import (
+	prom "github.com/prometheus/client_golang/prometheus"
+	"google.golang.org/grpc"
+)
+
+var (
+	// DefaultServerMetrics is the default instance of ServerMetrics. It is
+	// intended to be used in conjunction the default Prometheus metrics
+	// registry.
+	DefaultServerMetrics = NewServerMetrics()
+
+	// UnaryServerInterceptor is a gRPC server-side interceptor that provides Prometheus monitoring for Unary RPCs.
+	UnaryServerInterceptor = DefaultServerMetrics.UnaryServerInterceptor()
+
+	// StreamServerInterceptor is a gRPC server-side interceptor that provides Prometheus monitoring for Streaming RPCs.
+	StreamServerInterceptor = DefaultServerMetrics.StreamServerInterceptor()
+)
+
+func init() {
+	prom.MustRegister(DefaultServerMetrics.serverStartedCounter)
+	prom.MustRegister(DefaultServerMetrics.serverHandledCounter)
+	prom.MustRegister(DefaultServerMetrics.serverStreamMsgReceived)
+	prom.MustRegister(DefaultServerMetrics.serverStreamMsgSent)
+}
+
+// Register takes a gRPC server and pre-initializes all counters to 0. This
+// allows for easier monitoring in Prometheus (no missing metrics), and should
+// be called *after* all services have been registered with the server. This
+// function acts on the DefaultServerMetrics variable.
+func Register(server *grpc.Server) {
+	DefaultServerMetrics.InitializeMetrics(server)
+}
+
+// EnableHandlingTimeHistogram turns on recording of handling time
+// of RPCs. Histogram metrics can be very expensive for Prometheus
+// to retain and query. This function acts on the DefaultServerMetrics
+// variable and the default Prometheus metrics registry.
+func EnableHandlingTimeHistogram(opts ...HistogramOption) {
+	DefaultServerMetrics.EnableHandlingTimeHistogram(opts...)
+	prom.Register(DefaultServerMetrics.serverHandledHistogram)
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/server_metrics.go b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/server_metrics.go
new file mode 100644
index 0000000..5b1467e
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/server_metrics.go
@@ -0,0 +1,185 @@
+package grpc_prometheus
+
+import (
+	prom "github.com/prometheus/client_golang/prometheus"
+	"golang.org/x/net/context"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/status"
+)
+
+// ServerMetrics represents a collection of metrics to be registered on a
+// Prometheus metrics registry for a gRPC server.
+type ServerMetrics struct {
+	serverStartedCounter          *prom.CounterVec
+	serverHandledCounter          *prom.CounterVec
+	serverStreamMsgReceived       *prom.CounterVec
+	serverStreamMsgSent           *prom.CounterVec
+	serverHandledHistogramEnabled bool
+	serverHandledHistogramOpts    prom.HistogramOpts
+	serverHandledHistogram        *prom.HistogramVec
+}
+
+// NewServerMetrics returns a ServerMetrics object. Use a new instance of
+// ServerMetrics when not using the default Prometheus metrics registry, for
+// example when wanting to control which metrics are added to a registry as
+// opposed to automatically adding metrics via init functions.
+func NewServerMetrics(counterOpts ...CounterOption) *ServerMetrics {
+	opts := counterOptions(counterOpts)
+	return &ServerMetrics{
+		serverStartedCounter: prom.NewCounterVec(
+			opts.apply(prom.CounterOpts{
+				Name: "grpc_server_started_total",
+				Help: "Total number of RPCs started on the server.",
+			}), []string{"grpc_type", "grpc_service", "grpc_method"}),
+		serverHandledCounter: prom.NewCounterVec(
+			opts.apply(prom.CounterOpts{
+				Name: "grpc_server_handled_total",
+				Help: "Total number of RPCs completed on the server, regardless of success or failure.",
+			}), []string{"grpc_type", "grpc_service", "grpc_method", "grpc_code"}),
+		serverStreamMsgReceived: prom.NewCounterVec(
+			opts.apply(prom.CounterOpts{
+				Name: "grpc_server_msg_received_total",
+				Help: "Total number of RPC stream messages received on the server.",
+			}), []string{"grpc_type", "grpc_service", "grpc_method"}),
+		serverStreamMsgSent: prom.NewCounterVec(
+			opts.apply(prom.CounterOpts{
+				Name: "grpc_server_msg_sent_total",
+				Help: "Total number of gRPC stream messages sent by the server.",
+			}), []string{"grpc_type", "grpc_service", "grpc_method"}),
+		serverHandledHistogramEnabled: false,
+		serverHandledHistogramOpts: prom.HistogramOpts{
+			Name:    "grpc_server_handling_seconds",
+			Help:    "Histogram of response latency (seconds) of gRPC that had been application-level handled by the server.",
+			Buckets: prom.DefBuckets,
+		},
+		serverHandledHistogram: nil,
+	}
+}
+
+// EnableHandlingTimeHistogram enables histograms being registered when
+// registering the ServerMetrics on a Prometheus registry. Histograms can be
+// expensive on Prometheus servers. It takes options to configure histogram
+// options such as the defined buckets.
+func (m *ServerMetrics) EnableHandlingTimeHistogram(opts ...HistogramOption) {
+	for _, o := range opts {
+		o(&m.serverHandledHistogramOpts)
+	}
+	if !m.serverHandledHistogramEnabled {
+		m.serverHandledHistogram = prom.NewHistogramVec(
+			m.serverHandledHistogramOpts,
+			[]string{"grpc_type", "grpc_service", "grpc_method"},
+		)
+	}
+	m.serverHandledHistogramEnabled = true
+}
+
+// Describe sends the super-set of all possible descriptors of metrics
+// collected by this Collector to the provided channel and returns once
+// the last descriptor has been sent.
+func (m *ServerMetrics) Describe(ch chan<- *prom.Desc) {
+	m.serverStartedCounter.Describe(ch)
+	m.serverHandledCounter.Describe(ch)
+	m.serverStreamMsgReceived.Describe(ch)
+	m.serverStreamMsgSent.Describe(ch)
+	if m.serverHandledHistogramEnabled {
+		m.serverHandledHistogram.Describe(ch)
+	}
+}
+
+// Collect is called by the Prometheus registry when collecting
+// metrics. The implementation sends each collected metric via the
+// provided channel and returns once the last metric has been sent.
+func (m *ServerMetrics) Collect(ch chan<- prom.Metric) {
+	m.serverStartedCounter.Collect(ch)
+	m.serverHandledCounter.Collect(ch)
+	m.serverStreamMsgReceived.Collect(ch)
+	m.serverStreamMsgSent.Collect(ch)
+	if m.serverHandledHistogramEnabled {
+		m.serverHandledHistogram.Collect(ch)
+	}
+}
+
+// UnaryServerInterceptor is a gRPC server-side interceptor that provides Prometheus monitoring for Unary RPCs.
+func (m *ServerMetrics) UnaryServerInterceptor() func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
+	return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
+		monitor := newServerReporter(m, Unary, info.FullMethod)
+		monitor.ReceivedMessage()
+		resp, err := handler(ctx, req)
+		st, _ := status.FromError(err)
+		monitor.Handled(st.Code())
+		if err == nil {
+			monitor.SentMessage()
+		}
+		return resp, err
+	}
+}
+
+// StreamServerInterceptor is a gRPC server-side interceptor that provides Prometheus monitoring for Streaming RPCs.
+func (m *ServerMetrics) StreamServerInterceptor() func(srv interface{}, ss grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
+	return func(srv interface{}, ss grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
+		monitor := newServerReporter(m, streamRPCType(info), info.FullMethod)
+		err := handler(srv, &monitoredServerStream{ss, monitor})
+		st, _ := status.FromError(err)
+		monitor.Handled(st.Code())
+		return err
+	}
+}
+
+// InitializeMetrics initializes all metrics, with their appropriate null
+// value, for all gRPC methods registered on a gRPC server. This is useful, to
+// ensure that all metrics exist when collecting and querying.
+func (m *ServerMetrics) InitializeMetrics(server *grpc.Server) {
+	serviceInfo := server.GetServiceInfo()
+	for serviceName, info := range serviceInfo {
+		for _, mInfo := range info.Methods {
+			preRegisterMethod(m, serviceName, &mInfo)
+		}
+	}
+}
+
+func streamRPCType(info *grpc.StreamServerInfo) grpcType {
+	if info.IsClientStream && !info.IsServerStream {
+		return ClientStream
+	} else if !info.IsClientStream && info.IsServerStream {
+		return ServerStream
+	}
+	return BidiStream
+}
+
+// monitoredStream wraps grpc.ServerStream allowing each Sent/Recv of message to increment counters.
+type monitoredServerStream struct {
+	grpc.ServerStream
+	monitor *serverReporter
+}
+
+func (s *monitoredServerStream) SendMsg(m interface{}) error {
+	err := s.ServerStream.SendMsg(m)
+	if err == nil {
+		s.monitor.SentMessage()
+	}
+	return err
+}
+
+func (s *monitoredServerStream) RecvMsg(m interface{}) error {
+	err := s.ServerStream.RecvMsg(m)
+	if err == nil {
+		s.monitor.ReceivedMessage()
+	}
+	return err
+}
+
+// preRegisterMethod is invoked on Register of a Server, allowing all gRPC services labels to be pre-populated.
+func preRegisterMethod(metrics *ServerMetrics, serviceName string, mInfo *grpc.MethodInfo) {
+	methodName := mInfo.Name
+	methodType := string(typeFromMethodInfo(mInfo))
+	// These are just references (no increments), as just referencing will create the labels but not set values.
+	metrics.serverStartedCounter.GetMetricWithLabelValues(methodType, serviceName, methodName)
+	metrics.serverStreamMsgReceived.GetMetricWithLabelValues(methodType, serviceName, methodName)
+	metrics.serverStreamMsgSent.GetMetricWithLabelValues(methodType, serviceName, methodName)
+	if metrics.serverHandledHistogramEnabled {
+		metrics.serverHandledHistogram.GetMetricWithLabelValues(methodType, serviceName, methodName)
+	}
+	for _, code := range allCodes {
+		metrics.serverHandledCounter.GetMetricWithLabelValues(methodType, serviceName, methodName, code.String())
+	}
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/server_reporter.go b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/server_reporter.go
new file mode 100644
index 0000000..aa9db54
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/server_reporter.go
@@ -0,0 +1,46 @@
+// Copyright 2016 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+package grpc_prometheus
+
+import (
+	"time"
+
+	"google.golang.org/grpc/codes"
+)
+
+type serverReporter struct {
+	metrics     *ServerMetrics
+	rpcType     grpcType
+	serviceName string
+	methodName  string
+	startTime   time.Time
+}
+
+func newServerReporter(m *ServerMetrics, rpcType grpcType, fullMethod string) *serverReporter {
+	r := &serverReporter{
+		metrics: m,
+		rpcType: rpcType,
+	}
+	if r.metrics.serverHandledHistogramEnabled {
+		r.startTime = time.Now()
+	}
+	r.serviceName, r.methodName = splitMethodName(fullMethod)
+	r.metrics.serverStartedCounter.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Inc()
+	return r
+}
+
+func (r *serverReporter) ReceivedMessage() {
+	r.metrics.serverStreamMsgReceived.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Inc()
+}
+
+func (r *serverReporter) SentMessage() {
+	r.metrics.serverStreamMsgSent.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Inc()
+}
+
+func (r *serverReporter) Handled(code codes.Code) {
+	r.metrics.serverHandledCounter.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName, code.String()).Inc()
+	if r.metrics.serverHandledHistogramEnabled {
+		r.metrics.serverHandledHistogram.WithLabelValues(string(r.rpcType), r.serviceName, r.methodName).Observe(time.Since(r.startTime).Seconds())
+	}
+}
diff --git a/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/util.go b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/util.go
new file mode 100644
index 0000000..7987de3
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/go-grpc-prometheus/util.go
@@ -0,0 +1,50 @@
+// Copyright 2016 Michal Witkowski. All Rights Reserved.
+// See LICENSE for licensing terms.
+
+package grpc_prometheus
+
+import (
+	"strings"
+
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/codes"
+)
+
+type grpcType string
+
+const (
+	Unary        grpcType = "unary"
+	ClientStream grpcType = "client_stream"
+	ServerStream grpcType = "server_stream"
+	BidiStream   grpcType = "bidi_stream"
+)
+
+var (
+	allCodes = []codes.Code{
+		codes.OK, codes.Canceled, codes.Unknown, codes.InvalidArgument, codes.DeadlineExceeded, codes.NotFound,
+		codes.AlreadyExists, codes.PermissionDenied, codes.Unauthenticated, codes.ResourceExhausted,
+		codes.FailedPrecondition, codes.Aborted, codes.OutOfRange, codes.Unimplemented, codes.Internal,
+		codes.Unavailable, codes.DataLoss,
+	}
+)
+
+func splitMethodName(fullMethodName string) (string, string) {
+	fullMethodName = strings.TrimPrefix(fullMethodName, "/") // remove leading slash
+	if i := strings.Index(fullMethodName, "/"); i >= 0 {
+		return fullMethodName[:i], fullMethodName[i+1:]
+	}
+	return "unknown", "unknown"
+}
+
+func typeFromMethodInfo(mInfo *grpc.MethodInfo) grpcType {
+	if !mInfo.IsClientStream && !mInfo.IsServerStream {
+		return Unary
+	}
+	if mInfo.IsClientStream && !mInfo.IsServerStream {
+		return ClientStream
+	}
+	if !mInfo.IsClientStream && mInfo.IsServerStream {
+		return ServerStream
+	}
+	return BidiStream
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/LICENSE.txt b/vendor/github.com/grpc-ecosystem/grpc-gateway/LICENSE.txt
new file mode 100644
index 0000000..3645162
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/LICENSE.txt
@@ -0,0 +1,27 @@
+Copyright (c) 2015, Gengo, Inc.
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright notice,
+      this list of conditions and the following disclaimer.
+
+    * Redistributions in binary form must reproduce the above copyright notice,
+      this list of conditions and the following disclaimer in the documentation
+      and/or other materials provided with the distribution.
+
+    * Neither the name of Gengo, Inc. nor the names of its
+      contributors may be used to endorse or promote products derived from this
+      software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/internal/BUILD.bazel b/vendor/github.com/grpc-ecosystem/grpc-gateway/internal/BUILD.bazel
new file mode 100644
index 0000000..76cafe6
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/internal/BUILD.bazel
@@ -0,0 +1,22 @@
+load("@io_bazel_rules_go//go:def.bzl", "go_library")
+load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")
+
+package(default_visibility = ["//visibility:public"])
+
+proto_library(
+    name = "internal_proto",
+    srcs = ["stream_chunk.proto"],
+    deps = ["@com_google_protobuf//:any_proto"],
+)
+
+go_proto_library(
+    name = "internal_go_proto",
+    importpath = "github.com/grpc-ecosystem/grpc-gateway/internal",
+    proto = ":internal_proto",
+)
+
+go_library(
+    name = "go_default_library",
+    embed = [":internal_go_proto"],
+    importpath = "github.com/grpc-ecosystem/grpc-gateway/internal",
+)
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/internal/stream_chunk.pb.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/internal/stream_chunk.pb.go
new file mode 100644
index 0000000..8858f06
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/internal/stream_chunk.pb.go
@@ -0,0 +1,118 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: internal/stream_chunk.proto
+
+package internal
+
+import proto "github.com/golang/protobuf/proto"
+import fmt "fmt"
+import math "math"
+import any "github.com/golang/protobuf/ptypes/any"
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
+
+// StreamError is a response type which is returned when
+// streaming rpc returns an error.
+type StreamError struct {
+	GrpcCode             int32      `protobuf:"varint,1,opt,name=grpc_code,json=grpcCode,proto3" json:"grpc_code,omitempty"`
+	HttpCode             int32      `protobuf:"varint,2,opt,name=http_code,json=httpCode,proto3" json:"http_code,omitempty"`
+	Message              string     `protobuf:"bytes,3,opt,name=message,proto3" json:"message,omitempty"`
+	HttpStatus           string     `protobuf:"bytes,4,opt,name=http_status,json=httpStatus,proto3" json:"http_status,omitempty"`
+	Details              []*any.Any `protobuf:"bytes,5,rep,name=details,proto3" json:"details,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
+	XXX_unrecognized     []byte     `json:"-"`
+	XXX_sizecache        int32      `json:"-"`
+}
+
+func (m *StreamError) Reset()         { *m = StreamError{} }
+func (m *StreamError) String() string { return proto.CompactTextString(m) }
+func (*StreamError) ProtoMessage()    {}
+func (*StreamError) Descriptor() ([]byte, []int) {
+	return fileDescriptor_stream_chunk_a2afb657504565d7, []int{0}
+}
+func (m *StreamError) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_StreamError.Unmarshal(m, b)
+}
+func (m *StreamError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_StreamError.Marshal(b, m, deterministic)
+}
+func (dst *StreamError) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StreamError.Merge(dst, src)
+}
+func (m *StreamError) XXX_Size() int {
+	return xxx_messageInfo_StreamError.Size(m)
+}
+func (m *StreamError) XXX_DiscardUnknown() {
+	xxx_messageInfo_StreamError.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_StreamError proto.InternalMessageInfo
+
+func (m *StreamError) GetGrpcCode() int32 {
+	if m != nil {
+		return m.GrpcCode
+	}
+	return 0
+}
+
+func (m *StreamError) GetHttpCode() int32 {
+	if m != nil {
+		return m.HttpCode
+	}
+	return 0
+}
+
+func (m *StreamError) GetMessage() string {
+	if m != nil {
+		return m.Message
+	}
+	return ""
+}
+
+func (m *StreamError) GetHttpStatus() string {
+	if m != nil {
+		return m.HttpStatus
+	}
+	return ""
+}
+
+func (m *StreamError) GetDetails() []*any.Any {
+	if m != nil {
+		return m.Details
+	}
+	return nil
+}
+
+func init() {
+	proto.RegisterType((*StreamError)(nil), "grpc.gateway.runtime.StreamError")
+}
+
+func init() {
+	proto.RegisterFile("internal/stream_chunk.proto", fileDescriptor_stream_chunk_a2afb657504565d7)
+}
+
+var fileDescriptor_stream_chunk_a2afb657504565d7 = []byte{
+	// 223 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x34, 0x90, 0x41, 0x4e, 0xc3, 0x30,
+	0x10, 0x45, 0x15, 0x4a, 0x69, 0x3b, 0xd9, 0x45, 0x5d, 0x18, 0xba, 0x20, 0x62, 0x95, 0x95, 0x23,
+	0xc1, 0x09, 0x00, 0x71, 0x81, 0x74, 0xc7, 0xa6, 0x9a, 0x26, 0x83, 0x13, 0x91, 0xd8, 0xd1, 0x78,
+	0x22, 0x94, 0x6b, 0x71, 0xc2, 0xca, 0x8e, 0xb2, 0xf4, 0x7b, 0x7f, 0xbe, 0xbe, 0x0c, 0xa7, 0xce,
+	0x0a, 0xb1, 0xc5, 0xbe, 0xf4, 0xc2, 0x84, 0xc3, 0xa5, 0x6e, 0x27, 0xfb, 0xab, 0x47, 0x76, 0xe2,
+	0xb2, 0xa3, 0xe1, 0xb1, 0xd6, 0x06, 0x85, 0xfe, 0x70, 0xd6, 0x3c, 0x59, 0xe9, 0x06, 0x7a, 0x7a,
+	0x34, 0xce, 0x99, 0x9e, 0xca, 0x98, 0xb9, 0x4e, 0x3f, 0x25, 0xda, 0x79, 0x39, 0x78, 0xf9, 0x4f,
+	0x20, 0x3d, 0xc7, 0x9e, 0x2f, 0x66, 0xc7, 0xd9, 0x09, 0x0e, 0xa1, 0xe2, 0x52, 0xbb, 0x86, 0x54,
+	0x92, 0x27, 0xc5, 0xb6, 0xda, 0x07, 0xf0, 0xe9, 0x1a, 0x0a, 0xb2, 0x15, 0x19, 0x17, 0x79, 0xb7,
+	0xc8, 0x00, 0xa2, 0x54, 0xb0, 0x1b, 0xc8, 0x7b, 0x34, 0xa4, 0x36, 0x79, 0x52, 0x1c, 0xaa, 0xf5,
+	0x99, 0x3d, 0x43, 0x1a, 0xcf, 0xbc, 0xa0, 0x4c, 0x5e, 0xdd, 0x47, 0x0b, 0x01, 0x9d, 0x23, 0xc9,
+	0x34, 0xec, 0x1a, 0x12, 0xec, 0x7a, 0xaf, 0xb6, 0xf9, 0xa6, 0x48, 0x5f, 0x8f, 0x7a, 0x59, 0xac,
+	0xd7, 0xc5, 0xfa, 0xdd, 0xce, 0xd5, 0x1a, 0xfa, 0x80, 0xef, 0xfd, 0xfa, 0x09, 0xd7, 0x87, 0x18,
+	0x79, 0xbb, 0x05, 0x00, 0x00, 0xff, 0xff, 0x0d, 0x7d, 0xa5, 0x18, 0x17, 0x01, 0x00, 0x00,
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/internal/stream_chunk.proto b/vendor/github.com/grpc-ecosystem/grpc-gateway/internal/stream_chunk.proto
new file mode 100644
index 0000000..55f42ce
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/internal/stream_chunk.proto
@@ -0,0 +1,15 @@
+syntax = "proto3";
+package grpc.gateway.runtime;
+option go_package = "internal";
+
+import "google/protobuf/any.proto";
+
+// StreamError is a response type which is returned when
+// streaming rpc returns an error.
+message StreamError {
+	int32 grpc_code = 1;
+	int32 http_code = 2;
+	string message = 3;
+	string http_status = 4;
+	repeated google.protobuf.Any details = 5;
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/BUILD.bazel b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/BUILD.bazel
new file mode 100644
index 0000000..2086222
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/BUILD.bazel
@@ -0,0 +1,84 @@
+load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
+
+package(default_visibility = ["//visibility:public"])
+
+go_library(
+    name = "go_default_library",
+    srcs = [
+        "context.go",
+        "convert.go",
+        "doc.go",
+        "errors.go",
+        "fieldmask.go",
+        "handler.go",
+        "marshal_httpbodyproto.go",
+        "marshal_json.go",
+        "marshal_jsonpb.go",
+        "marshal_proto.go",
+        "marshaler.go",
+        "marshaler_registry.go",
+        "mux.go",
+        "pattern.go",
+        "proto2_convert.go",
+        "proto_errors.go",
+        "query.go",
+    ],
+    importpath = "github.com/grpc-ecosystem/grpc-gateway/runtime",
+    deps = [
+        "//internal:go_default_library",
+        "//utilities:go_default_library",
+        "@com_github_golang_protobuf//jsonpb:go_default_library_gen",
+        "@com_github_golang_protobuf//proto:go_default_library",
+        "@com_github_golang_protobuf//protoc-gen-go/generator:go_default_library_gen",
+        "@go_googleapis//google/api:httpbody_go_proto",
+        "@io_bazel_rules_go//proto/wkt:any_go_proto",
+        "@io_bazel_rules_go//proto/wkt:duration_go_proto",
+        "@io_bazel_rules_go//proto/wkt:field_mask_go_proto",
+        "@io_bazel_rules_go//proto/wkt:timestamp_go_proto",
+        "@io_bazel_rules_go//proto/wkt:wrappers_go_proto",
+        "@org_golang_google_grpc//codes:go_default_library",
+        "@org_golang_google_grpc//grpclog:go_default_library",
+        "@org_golang_google_grpc//metadata:go_default_library",
+        "@org_golang_google_grpc//status:go_default_library",
+    ],
+)
+
+go_test(
+    name = "go_default_test",
+    size = "small",
+    srcs = [
+        "context_test.go",
+        "errors_test.go",
+        "fieldmask_test.go",
+        "handler_test.go",
+        "marshal_httpbodyproto_test.go",
+        "marshal_json_test.go",
+        "marshal_jsonpb_test.go",
+        "marshal_proto_test.go",
+        "marshaler_registry_test.go",
+        "mux_test.go",
+        "pattern_test.go",
+        "query_test.go",
+    ],
+    embed = [":go_default_library"],
+    deps = [
+        "//examples/proto/examplepb:go_default_library",
+        "//internal:go_default_library",
+        "//utilities:go_default_library",
+        "@com_github_golang_protobuf//jsonpb:go_default_library_gen",
+        "@com_github_golang_protobuf//proto:go_default_library",
+        "@com_github_golang_protobuf//ptypes:go_default_library_gen",
+        "@go_googleapis//google/api:httpbody_go_proto",
+        "@go_googleapis//google/rpc:errdetails_go_proto",
+        "@io_bazel_rules_go//proto/wkt:duration_go_proto",
+        "@io_bazel_rules_go//proto/wkt:empty_go_proto",
+        "@io_bazel_rules_go//proto/wkt:field_mask_go_proto",
+        "@io_bazel_rules_go//proto/wkt:struct_go_proto",
+        "@io_bazel_rules_go//proto/wkt:timestamp_go_proto",
+        "@io_bazel_rules_go//proto/wkt:wrappers_go_proto",
+        "@org_golang_google_grpc//:go_default_library",
+        "@org_golang_google_grpc//codes:go_default_library",
+        "@org_golang_google_grpc//metadata:go_default_library",
+        "@org_golang_google_grpc//status:go_default_library",
+    ],
+)
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/context.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/context.go
new file mode 100644
index 0000000..896057e
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/context.go
@@ -0,0 +1,210 @@
+package runtime
+
+import (
+	"context"
+	"encoding/base64"
+	"fmt"
+	"net"
+	"net/http"
+	"net/textproto"
+	"strconv"
+	"strings"
+	"time"
+
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/grpclog"
+	"google.golang.org/grpc/metadata"
+	"google.golang.org/grpc/status"
+)
+
+// MetadataHeaderPrefix is the http prefix that represents custom metadata
+// parameters to or from a gRPC call.
+const MetadataHeaderPrefix = "Grpc-Metadata-"
+
+// MetadataPrefix is prepended to permanent HTTP header keys (as specified
+// by the IANA) when added to the gRPC context.
+const MetadataPrefix = "grpcgateway-"
+
+// MetadataTrailerPrefix is prepended to gRPC metadata as it is converted to
+// HTTP headers in a response handled by grpc-gateway
+const MetadataTrailerPrefix = "Grpc-Trailer-"
+
+const metadataGrpcTimeout = "Grpc-Timeout"
+const metadataHeaderBinarySuffix = "-Bin"
+
+const xForwardedFor = "X-Forwarded-For"
+const xForwardedHost = "X-Forwarded-Host"
+
+var (
+	// DefaultContextTimeout is used for gRPC call context.WithTimeout whenever a Grpc-Timeout inbound
+	// header isn't present. If the value is 0 the sent `context` will not have a timeout.
+	DefaultContextTimeout = 0 * time.Second
+)
+
+func decodeBinHeader(v string) ([]byte, error) {
+	if len(v)%4 == 0 {
+		// Input was padded, or padding was not necessary.
+		return base64.StdEncoding.DecodeString(v)
+	}
+	return base64.RawStdEncoding.DecodeString(v)
+}
+
+/*
+AnnotateContext adds context information such as metadata from the request.
+
+At a minimum, the RemoteAddr is included in the fashion of "X-Forwarded-For",
+except that the forwarded destination is not another HTTP service but rather
+a gRPC service.
+*/
+func AnnotateContext(ctx context.Context, mux *ServeMux, req *http.Request) (context.Context, error) {
+	var pairs []string
+	timeout := DefaultContextTimeout
+	if tm := req.Header.Get(metadataGrpcTimeout); tm != "" {
+		var err error
+		timeout, err = timeoutDecode(tm)
+		if err != nil {
+			return nil, status.Errorf(codes.InvalidArgument, "invalid grpc-timeout: %s", tm)
+		}
+	}
+
+	for key, vals := range req.Header {
+		for _, val := range vals {
+			key = textproto.CanonicalMIMEHeaderKey(key)
+			// For backwards-compatibility, pass through 'authorization' header with no prefix.
+			if key == "Authorization" {
+				pairs = append(pairs, "authorization", val)
+			}
+			if h, ok := mux.incomingHeaderMatcher(key); ok {
+				// Handles "-bin" metadata in grpc, since grpc will do another base64
+				// encode before sending to server, we need to decode it first.
+				if strings.HasSuffix(key, metadataHeaderBinarySuffix) {
+					b, err := decodeBinHeader(val)
+					if err != nil {
+						return nil, status.Errorf(codes.InvalidArgument, "invalid binary header %s: %s", key, err)
+					}
+
+					val = string(b)
+				}
+				pairs = append(pairs, h, val)
+			}
+		}
+	}
+	if host := req.Header.Get(xForwardedHost); host != "" {
+		pairs = append(pairs, strings.ToLower(xForwardedHost), host)
+	} else if req.Host != "" {
+		pairs = append(pairs, strings.ToLower(xForwardedHost), req.Host)
+	}
+
+	if addr := req.RemoteAddr; addr != "" {
+		if remoteIP, _, err := net.SplitHostPort(addr); err == nil {
+			if fwd := req.Header.Get(xForwardedFor); fwd == "" {
+				pairs = append(pairs, strings.ToLower(xForwardedFor), remoteIP)
+			} else {
+				pairs = append(pairs, strings.ToLower(xForwardedFor), fmt.Sprintf("%s, %s", fwd, remoteIP))
+			}
+		} else {
+			grpclog.Infof("invalid remote addr: %s", addr)
+		}
+	}
+
+	if timeout != 0 {
+		ctx, _ = context.WithTimeout(ctx, timeout)
+	}
+	if len(pairs) == 0 {
+		return ctx, nil
+	}
+	md := metadata.Pairs(pairs...)
+	for _, mda := range mux.metadataAnnotators {
+		md = metadata.Join(md, mda(ctx, req))
+	}
+	return metadata.NewOutgoingContext(ctx, md), nil
+}
+
+// ServerMetadata consists of metadata sent from gRPC server.
+type ServerMetadata struct {
+	HeaderMD  metadata.MD
+	TrailerMD metadata.MD
+}
+
+type serverMetadataKey struct{}
+
+// NewServerMetadataContext creates a new context with ServerMetadata
+func NewServerMetadataContext(ctx context.Context, md ServerMetadata) context.Context {
+	return context.WithValue(ctx, serverMetadataKey{}, md)
+}
+
+// ServerMetadataFromContext returns the ServerMetadata in ctx
+func ServerMetadataFromContext(ctx context.Context) (md ServerMetadata, ok bool) {
+	md, ok = ctx.Value(serverMetadataKey{}).(ServerMetadata)
+	return
+}
+
+func timeoutDecode(s string) (time.Duration, error) {
+	size := len(s)
+	if size < 2 {
+		return 0, fmt.Errorf("timeout string is too short: %q", s)
+	}
+	d, ok := timeoutUnitToDuration(s[size-1])
+	if !ok {
+		return 0, fmt.Errorf("timeout unit is not recognized: %q", s)
+	}
+	t, err := strconv.ParseInt(s[:size-1], 10, 64)
+	if err != nil {
+		return 0, err
+	}
+	return d * time.Duration(t), nil
+}
+
+func timeoutUnitToDuration(u uint8) (d time.Duration, ok bool) {
+	switch u {
+	case 'H':
+		return time.Hour, true
+	case 'M':
+		return time.Minute, true
+	case 'S':
+		return time.Second, true
+	case 'm':
+		return time.Millisecond, true
+	case 'u':
+		return time.Microsecond, true
+	case 'n':
+		return time.Nanosecond, true
+	default:
+	}
+	return
+}
+
+// isPermanentHTTPHeader checks whether hdr belongs to the list of
+// permenant request headers maintained by IANA.
+// http://www.iana.org/assignments/message-headers/message-headers.xml
+func isPermanentHTTPHeader(hdr string) bool {
+	switch hdr {
+	case
+		"Accept",
+		"Accept-Charset",
+		"Accept-Language",
+		"Accept-Ranges",
+		"Authorization",
+		"Cache-Control",
+		"Content-Type",
+		"Cookie",
+		"Date",
+		"Expect",
+		"From",
+		"Host",
+		"If-Match",
+		"If-Modified-Since",
+		"If-None-Match",
+		"If-Schedule-Tag-Match",
+		"If-Unmodified-Since",
+		"Max-Forwards",
+		"Origin",
+		"Pragma",
+		"Referer",
+		"User-Agent",
+		"Via",
+		"Warning":
+		return true
+	}
+	return false
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/convert.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/convert.go
new file mode 100644
index 0000000..a5b3bd6
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/convert.go
@@ -0,0 +1,312 @@
+package runtime
+
+import (
+	"encoding/base64"
+	"fmt"
+	"strconv"
+	"strings"
+
+	"github.com/golang/protobuf/jsonpb"
+	"github.com/golang/protobuf/ptypes/duration"
+	"github.com/golang/protobuf/ptypes/timestamp"
+	"github.com/golang/protobuf/ptypes/wrappers"
+)
+
+// String just returns the given string.
+// It is just for compatibility to other types.
+func String(val string) (string, error) {
+	return val, nil
+}
+
+// StringSlice converts 'val' where individual strings are separated by
+// 'sep' into a string slice.
+func StringSlice(val, sep string) ([]string, error) {
+	return strings.Split(val, sep), nil
+}
+
+// Bool converts the given string representation of a boolean value into bool.
+func Bool(val string) (bool, error) {
+	return strconv.ParseBool(val)
+}
+
+// BoolSlice converts 'val' where individual booleans are separated by
+// 'sep' into a bool slice.
+func BoolSlice(val, sep string) ([]bool, error) {
+	s := strings.Split(val, sep)
+	values := make([]bool, len(s))
+	for i, v := range s {
+		value, err := Bool(v)
+		if err != nil {
+			return values, err
+		}
+		values[i] = value
+	}
+	return values, nil
+}
+
+// Float64 converts the given string representation into representation of a floating point number into float64.
+func Float64(val string) (float64, error) {
+	return strconv.ParseFloat(val, 64)
+}
+
+// Float64Slice converts 'val' where individual floating point numbers are separated by
+// 'sep' into a float64 slice.
+func Float64Slice(val, sep string) ([]float64, error) {
+	s := strings.Split(val, sep)
+	values := make([]float64, len(s))
+	for i, v := range s {
+		value, err := Float64(v)
+		if err != nil {
+			return values, err
+		}
+		values[i] = value
+	}
+	return values, nil
+}
+
+// Float32 converts the given string representation of a floating point number into float32.
+func Float32(val string) (float32, error) {
+	f, err := strconv.ParseFloat(val, 32)
+	if err != nil {
+		return 0, err
+	}
+	return float32(f), nil
+}
+
+// Float32Slice converts 'val' where individual floating point numbers are separated by
+// 'sep' into a float32 slice.
+func Float32Slice(val, sep string) ([]float32, error) {
+	s := strings.Split(val, sep)
+	values := make([]float32, len(s))
+	for i, v := range s {
+		value, err := Float32(v)
+		if err != nil {
+			return values, err
+		}
+		values[i] = value
+	}
+	return values, nil
+}
+
+// Int64 converts the given string representation of an integer into int64.
+func Int64(val string) (int64, error) {
+	return strconv.ParseInt(val, 0, 64)
+}
+
+// Int64Slice converts 'val' where individual integers are separated by
+// 'sep' into a int64 slice.
+func Int64Slice(val, sep string) ([]int64, error) {
+	s := strings.Split(val, sep)
+	values := make([]int64, len(s))
+	for i, v := range s {
+		value, err := Int64(v)
+		if err != nil {
+			return values, err
+		}
+		values[i] = value
+	}
+	return values, nil
+}
+
+// Int32 converts the given string representation of an integer into int32.
+func Int32(val string) (int32, error) {
+	i, err := strconv.ParseInt(val, 0, 32)
+	if err != nil {
+		return 0, err
+	}
+	return int32(i), nil
+}
+
+// Int32Slice converts 'val' where individual integers are separated by
+// 'sep' into a int32 slice.
+func Int32Slice(val, sep string) ([]int32, error) {
+	s := strings.Split(val, sep)
+	values := make([]int32, len(s))
+	for i, v := range s {
+		value, err := Int32(v)
+		if err != nil {
+			return values, err
+		}
+		values[i] = value
+	}
+	return values, nil
+}
+
+// Uint64 converts the given string representation of an integer into uint64.
+func Uint64(val string) (uint64, error) {
+	return strconv.ParseUint(val, 0, 64)
+}
+
+// Uint64Slice converts 'val' where individual integers are separated by
+// 'sep' into a uint64 slice.
+func Uint64Slice(val, sep string) ([]uint64, error) {
+	s := strings.Split(val, sep)
+	values := make([]uint64, len(s))
+	for i, v := range s {
+		value, err := Uint64(v)
+		if err != nil {
+			return values, err
+		}
+		values[i] = value
+	}
+	return values, nil
+}
+
+// Uint32 converts the given string representation of an integer into uint32.
+func Uint32(val string) (uint32, error) {
+	i, err := strconv.ParseUint(val, 0, 32)
+	if err != nil {
+		return 0, err
+	}
+	return uint32(i), nil
+}
+
+// Uint32Slice converts 'val' where individual integers are separated by
+// 'sep' into a uint32 slice.
+func Uint32Slice(val, sep string) ([]uint32, error) {
+	s := strings.Split(val, sep)
+	values := make([]uint32, len(s))
+	for i, v := range s {
+		value, err := Uint32(v)
+		if err != nil {
+			return values, err
+		}
+		values[i] = value
+	}
+	return values, nil
+}
+
+// Bytes converts the given string representation of a byte sequence into a slice of bytes
+// A bytes sequence is encoded in URL-safe base64 without padding
+func Bytes(val string) ([]byte, error) {
+	b, err := base64.StdEncoding.DecodeString(val)
+	if err != nil {
+		b, err = base64.URLEncoding.DecodeString(val)
+		if err != nil {
+			return nil, err
+		}
+	}
+	return b, nil
+}
+
+// BytesSlice converts 'val' where individual bytes sequences, encoded in URL-safe
+// base64 without padding, are separated by 'sep' into a slice of bytes slices slice.
+func BytesSlice(val, sep string) ([][]byte, error) {
+	s := strings.Split(val, sep)
+	values := make([][]byte, len(s))
+	for i, v := range s {
+		value, err := Bytes(v)
+		if err != nil {
+			return values, err
+		}
+		values[i] = value
+	}
+	return values, nil
+}
+
+// Timestamp converts the given RFC3339 formatted string into a timestamp.Timestamp.
+func Timestamp(val string) (*timestamp.Timestamp, error) {
+	var r *timestamp.Timestamp
+	err := jsonpb.UnmarshalString(val, r)
+	return r, err
+}
+
+// Duration converts the given string into a timestamp.Duration.
+func Duration(val string) (*duration.Duration, error) {
+	var r *duration.Duration
+	err := jsonpb.UnmarshalString(val, r)
+	return r, err
+}
+
+// Enum converts the given string into an int32 that should be type casted into the
+// correct enum proto type.
+func Enum(val string, enumValMap map[string]int32) (int32, error) {
+	e, ok := enumValMap[val]
+	if ok {
+		return e, nil
+	}
+
+	i, err := Int32(val)
+	if err != nil {
+		return 0, fmt.Errorf("%s is not valid", val)
+	}
+	for _, v := range enumValMap {
+		if v == i {
+			return i, nil
+		}
+	}
+	return 0, fmt.Errorf("%s is not valid", val)
+}
+
+// EnumSlice converts 'val' where individual enums are separated by 'sep'
+// into a int32 slice. Each individual int32 should be type casted into the
+// correct enum proto type.
+func EnumSlice(val, sep string, enumValMap map[string]int32) ([]int32, error) {
+	s := strings.Split(val, sep)
+	values := make([]int32, len(s))
+	for i, v := range s {
+		value, err := Enum(v, enumValMap)
+		if err != nil {
+			return values, err
+		}
+		values[i] = value
+	}
+	return values, nil
+}
+
+/*
+	Support fot google.protobuf.wrappers on top of primitive types
+*/
+
+// StringValue well-known type support as wrapper around string type
+func StringValue(val string) (*wrappers.StringValue, error) {
+	return &wrappers.StringValue{Value: val}, nil
+}
+
+// FloatValue well-known type support as wrapper around float32 type
+func FloatValue(val string) (*wrappers.FloatValue, error) {
+	parsedVal, err := Float32(val)
+	return &wrappers.FloatValue{Value: parsedVal}, err
+}
+
+// DoubleValue well-known type support as wrapper around float64 type
+func DoubleValue(val string) (*wrappers.DoubleValue, error) {
+	parsedVal, err := Float64(val)
+	return &wrappers.DoubleValue{Value: parsedVal}, err
+}
+
+// BoolValue well-known type support as wrapper around bool type
+func BoolValue(val string) (*wrappers.BoolValue, error) {
+	parsedVal, err := Bool(val)
+	return &wrappers.BoolValue{Value: parsedVal}, err
+}
+
+// Int32Value well-known type support as wrapper around int32 type
+func Int32Value(val string) (*wrappers.Int32Value, error) {
+	parsedVal, err := Int32(val)
+	return &wrappers.Int32Value{Value: parsedVal}, err
+}
+
+// UInt32Value well-known type support as wrapper around uint32 type
+func UInt32Value(val string) (*wrappers.UInt32Value, error) {
+	parsedVal, err := Uint32(val)
+	return &wrappers.UInt32Value{Value: parsedVal}, err
+}
+
+// Int64Value well-known type support as wrapper around int64 type
+func Int64Value(val string) (*wrappers.Int64Value, error) {
+	parsedVal, err := Int64(val)
+	return &wrappers.Int64Value{Value: parsedVal}, err
+}
+
+// UInt64Value well-known type support as wrapper around uint64 type
+func UInt64Value(val string) (*wrappers.UInt64Value, error) {
+	parsedVal, err := Uint64(val)
+	return &wrappers.UInt64Value{Value: parsedVal}, err
+}
+
+// BytesValue well-known type support as wrapper around bytes[] type
+func BytesValue(val string) (*wrappers.BytesValue, error) {
+	parsedVal, err := Bytes(val)
+	return &wrappers.BytesValue{Value: parsedVal}, err
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/doc.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/doc.go
new file mode 100644
index 0000000..b6e5ddf
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/doc.go
@@ -0,0 +1,5 @@
+/*
+Package runtime contains runtime helper functions used by
+servers which protoc-gen-grpc-gateway generates.
+*/
+package runtime
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/errors.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/errors.go
new file mode 100644
index 0000000..ad94578
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/errors.go
@@ -0,0 +1,146 @@
+package runtime
+
+import (
+	"context"
+	"io"
+	"net/http"
+
+	"github.com/golang/protobuf/proto"
+	"github.com/golang/protobuf/ptypes/any"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/grpclog"
+	"google.golang.org/grpc/status"
+)
+
+// HTTPStatusFromCode converts a gRPC error code into the corresponding HTTP response status.
+// See: https://github.com/googleapis/googleapis/blob/master/google/rpc/code.proto
+func HTTPStatusFromCode(code codes.Code) int {
+	switch code {
+	case codes.OK:
+		return http.StatusOK
+	case codes.Canceled:
+		return http.StatusRequestTimeout
+	case codes.Unknown:
+		return http.StatusInternalServerError
+	case codes.InvalidArgument:
+		return http.StatusBadRequest
+	case codes.DeadlineExceeded:
+		return http.StatusGatewayTimeout
+	case codes.NotFound:
+		return http.StatusNotFound
+	case codes.AlreadyExists:
+		return http.StatusConflict
+	case codes.PermissionDenied:
+		return http.StatusForbidden
+	case codes.Unauthenticated:
+		return http.StatusUnauthorized
+	case codes.ResourceExhausted:
+		return http.StatusTooManyRequests
+	case codes.FailedPrecondition:
+		// Note, this deliberately doesn't translate to the similarly named '412 Precondition Failed' HTTP response status.
+		return http.StatusBadRequest
+	case codes.Aborted:
+		return http.StatusConflict
+	case codes.OutOfRange:
+		return http.StatusBadRequest
+	case codes.Unimplemented:
+		return http.StatusNotImplemented
+	case codes.Internal:
+		return http.StatusInternalServerError
+	case codes.Unavailable:
+		return http.StatusServiceUnavailable
+	case codes.DataLoss:
+		return http.StatusInternalServerError
+	}
+
+	grpclog.Infof("Unknown gRPC error code: %v", code)
+	return http.StatusInternalServerError
+}
+
+var (
+	// HTTPError replies to the request with the error.
+	// You can set a custom function to this variable to customize error format.
+	HTTPError = DefaultHTTPError
+	// OtherErrorHandler handles the following error used by the gateway: StatusMethodNotAllowed StatusNotFound and StatusBadRequest
+	OtherErrorHandler = DefaultOtherErrorHandler
+)
+
+type errorBody struct {
+	Error string `protobuf:"bytes,1,name=error" json:"error"`
+	// This is to make the error more compatible with users that expect errors to be Status objects:
+	// https://github.com/grpc/grpc/blob/master/src/proto/grpc/status/status.proto
+	// It should be the exact same message as the Error field.
+	Message string     `protobuf:"bytes,1,name=message" json:"message"`
+	Code    int32      `protobuf:"varint,2,name=code" json:"code"`
+	Details []*any.Any `protobuf:"bytes,3,rep,name=details" json:"details,omitempty"`
+}
+
+// Make this also conform to proto.Message for builtin JSONPb Marshaler
+func (e *errorBody) Reset()         { *e = errorBody{} }
+func (e *errorBody) String() string { return proto.CompactTextString(e) }
+func (*errorBody) ProtoMessage()    {}
+
+// DefaultHTTPError is the default implementation of HTTPError.
+// If "err" is an error from gRPC system, the function replies with the status code mapped by HTTPStatusFromCode.
+// If otherwise, it replies with http.StatusInternalServerError.
+//
+// The response body returned by this function is a JSON object,
+// which contains a member whose key is "error" and whose value is err.Error().
+func DefaultHTTPError(ctx context.Context, mux *ServeMux, marshaler Marshaler, w http.ResponseWriter, _ *http.Request, err error) {
+	const fallback = `{"error": "failed to marshal error message"}`
+
+	s, ok := status.FromError(err)
+	if !ok {
+		s = status.New(codes.Unknown, err.Error())
+	}
+
+	w.Header().Del("Trailer")
+
+	contentType := marshaler.ContentType()
+	// Check marshaler on run time in order to keep backwards compatability
+	// An interface param needs to be added to the ContentType() function on
+	// the Marshal interface to be able to remove this check
+	if httpBodyMarshaler, ok := marshaler.(*HTTPBodyMarshaler); ok {
+		pb := s.Proto()
+		contentType = httpBodyMarshaler.ContentTypeFromMessage(pb)
+	}
+	w.Header().Set("Content-Type", contentType)
+
+	body := &errorBody{
+		Error:   s.Message(),
+		Message: s.Message(),
+		Code:    int32(s.Code()),
+		Details: s.Proto().GetDetails(),
+	}
+
+	buf, merr := marshaler.Marshal(body)
+	if merr != nil {
+		grpclog.Infof("Failed to marshal error message %q: %v", body, merr)
+		w.WriteHeader(http.StatusInternalServerError)
+		if _, err := io.WriteString(w, fallback); err != nil {
+			grpclog.Infof("Failed to write response: %v", err)
+		}
+		return
+	}
+
+	md, ok := ServerMetadataFromContext(ctx)
+	if !ok {
+		grpclog.Infof("Failed to extract ServerMetadata from context")
+	}
+
+	handleForwardResponseServerMetadata(w, mux, md)
+	handleForwardResponseTrailerHeader(w, md)
+	st := HTTPStatusFromCode(s.Code())
+	w.WriteHeader(st)
+	if _, err := w.Write(buf); err != nil {
+		grpclog.Infof("Failed to write response: %v", err)
+	}
+
+	handleForwardResponseTrailer(w, md)
+}
+
+// DefaultOtherErrorHandler is the default implementation of OtherErrorHandler.
+// It simply writes a string representation of the given error into "w".
+func DefaultOtherErrorHandler(w http.ResponseWriter, _ *http.Request, msg string, code int) {
+	http.Error(w, msg, code)
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/fieldmask.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/fieldmask.go
new file mode 100644
index 0000000..e1cf7a9
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/fieldmask.go
@@ -0,0 +1,70 @@
+package runtime
+
+import (
+	"encoding/json"
+	"io"
+	"strings"
+
+	"github.com/golang/protobuf/protoc-gen-go/generator"
+	"google.golang.org/genproto/protobuf/field_mask"
+)
+
+// FieldMaskFromRequestBody creates a FieldMask printing all complete paths from the JSON body.
+func FieldMaskFromRequestBody(r io.Reader) (*field_mask.FieldMask, error) {
+	fm := &field_mask.FieldMask{}
+	var root interface{}
+	if err := json.NewDecoder(r).Decode(&root); err != nil {
+		if err == io.EOF {
+			return fm, nil
+		}
+		return nil, err
+	}
+
+	queue := []fieldMaskPathItem{{node: root}}
+	for len(queue) > 0 {
+		// dequeue an item
+		item := queue[0]
+		queue = queue[1:]
+
+		if m, ok := item.node.(map[string]interface{}); ok {
+			// if the item is an object, then enqueue all of its children
+			for k, v := range m {
+				queue = append(queue, fieldMaskPathItem{path: append(item.path, generator.CamelCase(k)), node: v})
+			}
+		} else if len(item.path) > 0 {
+			// otherwise, it's a leaf node so print its path
+			fm.Paths = append(fm.Paths, strings.Join(item.path, "."))
+		}
+	}
+
+	return fm, nil
+}
+
+// fieldMaskPathItem stores a in-progress deconstruction of a path for a fieldmask
+type fieldMaskPathItem struct {
+	// the list of prior fields leading up to node
+	path []string
+
+	// a generic decoded json object the current item to inspect for further path extraction
+	node interface{}
+}
+
+// CamelCaseFieldMask updates the given FieldMask by converting all of its paths to CamelCase, using the same heuristic
+// that's used for naming protobuf fields in Go.
+func CamelCaseFieldMask(mask *field_mask.FieldMask) {
+	if mask == nil || mask.Paths == nil {
+		return
+	}
+
+	var newPaths []string
+	for _, path := range mask.Paths {
+		lowerCasedParts := strings.Split(path, ".")
+		var camelCasedParts []string
+		for _, part := range lowerCasedParts {
+			camelCasedParts = append(camelCasedParts, generator.CamelCase(part))
+		}
+		newPaths = append(newPaths, strings.Join(camelCasedParts, "."))
+	}
+
+	mask.Paths = newPaths
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/handler.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/handler.go
new file mode 100644
index 0000000..2af9006
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/handler.go
@@ -0,0 +1,209 @@
+package runtime
+
+import (
+	"errors"
+	"fmt"
+	"io"
+	"net/http"
+	"net/textproto"
+
+	"context"
+	"github.com/golang/protobuf/proto"
+	"github.com/grpc-ecosystem/grpc-gateway/internal"
+	"google.golang.org/grpc/grpclog"
+)
+
+var errEmptyResponse = errors.New("empty response")
+
+// ForwardResponseStream forwards the stream from gRPC server to REST client.
+func ForwardResponseStream(ctx context.Context, mux *ServeMux, marshaler Marshaler, w http.ResponseWriter, req *http.Request, recv func() (proto.Message, error), opts ...func(context.Context, http.ResponseWriter, proto.Message) error) {
+	f, ok := w.(http.Flusher)
+	if !ok {
+		grpclog.Infof("Flush not supported in %T", w)
+		http.Error(w, "unexpected type of web server", http.StatusInternalServerError)
+		return
+	}
+
+	md, ok := ServerMetadataFromContext(ctx)
+	if !ok {
+		grpclog.Infof("Failed to extract ServerMetadata from context")
+		http.Error(w, "unexpected error", http.StatusInternalServerError)
+		return
+	}
+	handleForwardResponseServerMetadata(w, mux, md)
+
+	w.Header().Set("Transfer-Encoding", "chunked")
+	w.Header().Set("Content-Type", marshaler.ContentType())
+	if err := handleForwardResponseOptions(ctx, w, nil, opts); err != nil {
+		HTTPError(ctx, mux, marshaler, w, req, err)
+		return
+	}
+
+	var delimiter []byte
+	if d, ok := marshaler.(Delimited); ok {
+		delimiter = d.Delimiter()
+	} else {
+		delimiter = []byte("\n")
+	}
+
+	var wroteHeader bool
+	for {
+		resp, err := recv()
+		if err == io.EOF {
+			return
+		}
+		if err != nil {
+			handleForwardResponseStreamError(ctx, wroteHeader, marshaler, w, req, mux, err)
+			return
+		}
+		if err := handleForwardResponseOptions(ctx, w, resp, opts); err != nil {
+			handleForwardResponseStreamError(ctx, wroteHeader, marshaler, w, req, mux, err)
+			return
+		}
+
+		buf, err := marshaler.Marshal(streamChunk(ctx, resp, mux.streamErrorHandler))
+		if err != nil {
+			grpclog.Infof("Failed to marshal response chunk: %v", err)
+			handleForwardResponseStreamError(ctx, wroteHeader, marshaler, w, req, mux, err)
+			return
+		}
+		if _, err = w.Write(buf); err != nil {
+			grpclog.Infof("Failed to send response chunk: %v", err)
+			return
+		}
+		wroteHeader = true
+		if _, err = w.Write(delimiter); err != nil {
+			grpclog.Infof("Failed to send delimiter chunk: %v", err)
+			return
+		}
+		f.Flush()
+	}
+}
+
+func handleForwardResponseServerMetadata(w http.ResponseWriter, mux *ServeMux, md ServerMetadata) {
+	for k, vs := range md.HeaderMD {
+		if h, ok := mux.outgoingHeaderMatcher(k); ok {
+			for _, v := range vs {
+				w.Header().Add(h, v)
+			}
+		}
+	}
+}
+
+func handleForwardResponseTrailerHeader(w http.ResponseWriter, md ServerMetadata) {
+	for k := range md.TrailerMD {
+		tKey := textproto.CanonicalMIMEHeaderKey(fmt.Sprintf("%s%s", MetadataTrailerPrefix, k))
+		w.Header().Add("Trailer", tKey)
+	}
+}
+
+func handleForwardResponseTrailer(w http.ResponseWriter, md ServerMetadata) {
+	for k, vs := range md.TrailerMD {
+		tKey := fmt.Sprintf("%s%s", MetadataTrailerPrefix, k)
+		for _, v := range vs {
+			w.Header().Add(tKey, v)
+		}
+	}
+}
+
+// responseBody interface contains method for getting field for marshaling to the response body
+// this method is generated for response struct from the value of `response_body` in the `google.api.HttpRule`
+type responseBody interface {
+	XXX_ResponseBody() interface{}
+}
+
+// ForwardResponseMessage forwards the message "resp" from gRPC server to REST client.
+func ForwardResponseMessage(ctx context.Context, mux *ServeMux, marshaler Marshaler, w http.ResponseWriter, req *http.Request, resp proto.Message, opts ...func(context.Context, http.ResponseWriter, proto.Message) error) {
+	md, ok := ServerMetadataFromContext(ctx)
+	if !ok {
+		grpclog.Infof("Failed to extract ServerMetadata from context")
+	}
+
+	handleForwardResponseServerMetadata(w, mux, md)
+	handleForwardResponseTrailerHeader(w, md)
+
+	contentType := marshaler.ContentType()
+	// Check marshaler on run time in order to keep backwards compatability
+	// An interface param needs to be added to the ContentType() function on
+	// the Marshal interface to be able to remove this check
+	if httpBodyMarshaler, ok := marshaler.(*HTTPBodyMarshaler); ok {
+		contentType = httpBodyMarshaler.ContentTypeFromMessage(resp)
+	}
+	w.Header().Set("Content-Type", contentType)
+
+	if err := handleForwardResponseOptions(ctx, w, resp, opts); err != nil {
+		HTTPError(ctx, mux, marshaler, w, req, err)
+		return
+	}
+	var buf []byte
+	var err error
+	if rb, ok := resp.(responseBody); ok {
+		buf, err = marshaler.Marshal(rb.XXX_ResponseBody())
+	} else {
+		buf, err = marshaler.Marshal(resp)
+	}
+	if err != nil {
+		grpclog.Infof("Marshal error: %v", err)
+		HTTPError(ctx, mux, marshaler, w, req, err)
+		return
+	}
+
+	if _, err = w.Write(buf); err != nil {
+		grpclog.Infof("Failed to write response: %v", err)
+	}
+
+	handleForwardResponseTrailer(w, md)
+}
+
+func handleForwardResponseOptions(ctx context.Context, w http.ResponseWriter, resp proto.Message, opts []func(context.Context, http.ResponseWriter, proto.Message) error) error {
+	if len(opts) == 0 {
+		return nil
+	}
+	for _, opt := range opts {
+		if err := opt(ctx, w, resp); err != nil {
+			grpclog.Infof("Error handling ForwardResponseOptions: %v", err)
+			return err
+		}
+	}
+	return nil
+}
+
+func handleForwardResponseStreamError(ctx context.Context, wroteHeader bool, marshaler Marshaler, w http.ResponseWriter, req *http.Request, mux *ServeMux, err error) {
+	serr := streamError(ctx, mux.streamErrorHandler, err)
+	if !wroteHeader {
+		w.WriteHeader(int(serr.HttpCode))
+	}
+	buf, merr := marshaler.Marshal(errorChunk(serr))
+	if merr != nil {
+		grpclog.Infof("Failed to marshal an error: %v", merr)
+		return
+	}
+	if _, werr := w.Write(buf); werr != nil {
+		grpclog.Infof("Failed to notify error to client: %v", werr)
+		return
+	}
+}
+
+// streamChunk returns a chunk in a response stream for the given result. The
+// given errHandler is used to render an error chunk if result is nil.
+func streamChunk(ctx context.Context, result proto.Message, errHandler StreamErrorHandlerFunc) map[string]proto.Message {
+	if result == nil {
+		return errorChunk(streamError(ctx, errHandler, errEmptyResponse))
+	}
+	return map[string]proto.Message{"result": result}
+}
+
+// streamError returns the payload for the final message in a response stream
+// that represents the given err.
+func streamError(ctx context.Context, errHandler StreamErrorHandlerFunc, err error) *StreamError {
+	serr := errHandler(ctx, err)
+	if serr != nil {
+		return serr
+	}
+	// TODO: log about misbehaving stream error handler?
+	return DefaultHTTPStreamErrorHandler(ctx, err)
+}
+
+func errorChunk(err *StreamError) map[string]proto.Message {
+	return map[string]proto.Message{"error": (*internal.StreamError)(err)}
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/marshal_httpbodyproto.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/marshal_httpbodyproto.go
new file mode 100644
index 0000000..f55285b
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/marshal_httpbodyproto.go
@@ -0,0 +1,43 @@
+package runtime
+
+import (
+	"google.golang.org/genproto/googleapis/api/httpbody"
+)
+
+// SetHTTPBodyMarshaler overwrite the default marshaler with the HTTPBodyMarshaler
+func SetHTTPBodyMarshaler(serveMux *ServeMux) {
+	serveMux.marshalers.mimeMap[MIMEWildcard] = &HTTPBodyMarshaler{
+		Marshaler: &JSONPb{OrigName: true},
+	}
+}
+
+// HTTPBodyMarshaler is a Marshaler which supports marshaling of a
+// google.api.HttpBody message as the full response body if it is
+// the actual message used as the response. If not, then this will
+// simply fallback to the Marshaler specified as its default Marshaler.
+type HTTPBodyMarshaler struct {
+	Marshaler
+}
+
+// ContentType implementation to keep backwards compatability with marshal interface
+func (h *HTTPBodyMarshaler) ContentType() string {
+	return h.ContentTypeFromMessage(nil)
+}
+
+// ContentTypeFromMessage in case v is a google.api.HttpBody message it returns
+// its specified content type otherwise fall back to the default Marshaler.
+func (h *HTTPBodyMarshaler) ContentTypeFromMessage(v interface{}) string {
+	if httpBody, ok := v.(*httpbody.HttpBody); ok {
+		return httpBody.GetContentType()
+	}
+	return h.Marshaler.ContentType()
+}
+
+// Marshal marshals "v" by returning the body bytes if v is a
+// google.api.HttpBody message, otherwise it falls back to the default Marshaler.
+func (h *HTTPBodyMarshaler) Marshal(v interface{}) ([]byte, error) {
+	if httpBody, ok := v.(*httpbody.HttpBody); ok {
+		return httpBody.Data, nil
+	}
+	return h.Marshaler.Marshal(v)
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/marshal_json.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/marshal_json.go
new file mode 100644
index 0000000..f9d3a58
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/marshal_json.go
@@ -0,0 +1,45 @@
+package runtime
+
+import (
+	"encoding/json"
+	"io"
+)
+
+// JSONBuiltin is a Marshaler which marshals/unmarshals into/from JSON
+// with the standard "encoding/json" package of Golang.
+// Although it is generally faster for simple proto messages than JSONPb,
+// it does not support advanced features of protobuf, e.g. map, oneof, ....
+//
+// The NewEncoder and NewDecoder types return *json.Encoder and
+// *json.Decoder respectively.
+type JSONBuiltin struct{}
+
+// ContentType always Returns "application/json".
+func (*JSONBuiltin) ContentType() string {
+	return "application/json"
+}
+
+// Marshal marshals "v" into JSON
+func (j *JSONBuiltin) Marshal(v interface{}) ([]byte, error) {
+	return json.Marshal(v)
+}
+
+// Unmarshal unmarshals JSON data into "v".
+func (j *JSONBuiltin) Unmarshal(data []byte, v interface{}) error {
+	return json.Unmarshal(data, v)
+}
+
+// NewDecoder returns a Decoder which reads JSON stream from "r".
+func (j *JSONBuiltin) NewDecoder(r io.Reader) Decoder {
+	return json.NewDecoder(r)
+}
+
+// NewEncoder returns an Encoder which writes JSON stream into "w".
+func (j *JSONBuiltin) NewEncoder(w io.Writer) Encoder {
+	return json.NewEncoder(w)
+}
+
+// Delimiter for newline encoded JSON streams.
+func (j *JSONBuiltin) Delimiter() []byte {
+	return []byte("\n")
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/marshal_jsonpb.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/marshal_jsonpb.go
new file mode 100644
index 0000000..f0de351
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/marshal_jsonpb.go
@@ -0,0 +1,262 @@
+package runtime
+
+import (
+	"bytes"
+	"encoding/json"
+	"fmt"
+	"io"
+	"reflect"
+
+	"github.com/golang/protobuf/jsonpb"
+	"github.com/golang/protobuf/proto"
+)
+
+// JSONPb is a Marshaler which marshals/unmarshals into/from JSON
+// with the "github.com/golang/protobuf/jsonpb".
+// It supports fully functionality of protobuf unlike JSONBuiltin.
+//
+// The NewDecoder method returns a DecoderWrapper, so the underlying
+// *json.Decoder methods can be used.
+type JSONPb jsonpb.Marshaler
+
+// ContentType always returns "application/json".
+func (*JSONPb) ContentType() string {
+	return "application/json"
+}
+
+// Marshal marshals "v" into JSON.
+func (j *JSONPb) Marshal(v interface{}) ([]byte, error) {
+	if _, ok := v.(proto.Message); !ok {
+		return j.marshalNonProtoField(v)
+	}
+
+	var buf bytes.Buffer
+	if err := j.marshalTo(&buf, v); err != nil {
+		return nil, err
+	}
+	return buf.Bytes(), nil
+}
+
+func (j *JSONPb) marshalTo(w io.Writer, v interface{}) error {
+	p, ok := v.(proto.Message)
+	if !ok {
+		buf, err := j.marshalNonProtoField(v)
+		if err != nil {
+			return err
+		}
+		_, err = w.Write(buf)
+		return err
+	}
+	return (*jsonpb.Marshaler)(j).Marshal(w, p)
+}
+
+var (
+	// protoMessageType is stored to prevent constant lookup of the same type at runtime.
+	protoMessageType = reflect.TypeOf((*proto.Message)(nil)).Elem()
+)
+
+// marshalNonProto marshals a non-message field of a protobuf message.
+// This function does not correctly marshals arbitrary data structure into JSON,
+// but it is only capable of marshaling non-message field values of protobuf,
+// i.e. primitive types, enums; pointers to primitives or enums; maps from
+// integer/string types to primitives/enums/pointers to messages.
+func (j *JSONPb) marshalNonProtoField(v interface{}) ([]byte, error) {
+	if v == nil {
+		return []byte("null"), nil
+	}
+	rv := reflect.ValueOf(v)
+	for rv.Kind() == reflect.Ptr {
+		if rv.IsNil() {
+			return []byte("null"), nil
+		}
+		rv = rv.Elem()
+	}
+
+	if rv.Kind() == reflect.Slice {
+		if rv.IsNil() {
+			if j.EmitDefaults {
+				return []byte("[]"), nil
+			}
+			return []byte("null"), nil
+		}
+
+		if rv.Type().Elem().Implements(protoMessageType) {
+			var buf bytes.Buffer
+			err := buf.WriteByte('[')
+			if err != nil {
+				return nil, err
+			}
+			for i := 0; i < rv.Len(); i++ {
+				if i != 0 {
+					err = buf.WriteByte(',')
+					if err != nil {
+						return nil, err
+					}
+				}
+				if err = (*jsonpb.Marshaler)(j).Marshal(&buf, rv.Index(i).Interface().(proto.Message)); err != nil {
+					return nil, err
+				}
+			}
+			err = buf.WriteByte(']')
+			if err != nil {
+				return nil, err
+			}
+
+			return buf.Bytes(), nil
+		}
+	}
+
+	if rv.Kind() == reflect.Map {
+		m := make(map[string]*json.RawMessage)
+		for _, k := range rv.MapKeys() {
+			buf, err := j.Marshal(rv.MapIndex(k).Interface())
+			if err != nil {
+				return nil, err
+			}
+			m[fmt.Sprintf("%v", k.Interface())] = (*json.RawMessage)(&buf)
+		}
+		if j.Indent != "" {
+			return json.MarshalIndent(m, "", j.Indent)
+		}
+		return json.Marshal(m)
+	}
+	if enum, ok := rv.Interface().(protoEnum); ok && !j.EnumsAsInts {
+		return json.Marshal(enum.String())
+	}
+	return json.Marshal(rv.Interface())
+}
+
+// Unmarshal unmarshals JSON "data" into "v"
+func (j *JSONPb) Unmarshal(data []byte, v interface{}) error {
+	return unmarshalJSONPb(data, v)
+}
+
+// NewDecoder returns a Decoder which reads JSON stream from "r".
+func (j *JSONPb) NewDecoder(r io.Reader) Decoder {
+	d := json.NewDecoder(r)
+	return DecoderWrapper{Decoder: d}
+}
+
+// DecoderWrapper is a wrapper around a *json.Decoder that adds
+// support for protos to the Decode method.
+type DecoderWrapper struct {
+	*json.Decoder
+}
+
+// Decode wraps the embedded decoder's Decode method to support
+// protos using a jsonpb.Unmarshaler.
+func (d DecoderWrapper) Decode(v interface{}) error {
+	return decodeJSONPb(d.Decoder, v)
+}
+
+// NewEncoder returns an Encoder which writes JSON stream into "w".
+func (j *JSONPb) NewEncoder(w io.Writer) Encoder {
+	return EncoderFunc(func(v interface{}) error {
+		if err := j.marshalTo(w, v); err != nil {
+			return err
+		}
+		// mimic json.Encoder by adding a newline (makes output
+		// easier to read when it contains multiple encoded items)
+		_, err := w.Write(j.Delimiter())
+		return err
+	})
+}
+
+func unmarshalJSONPb(data []byte, v interface{}) error {
+	d := json.NewDecoder(bytes.NewReader(data))
+	return decodeJSONPb(d, v)
+}
+
+func decodeJSONPb(d *json.Decoder, v interface{}) error {
+	p, ok := v.(proto.Message)
+	if !ok {
+		return decodeNonProtoField(d, v)
+	}
+	unmarshaler := &jsonpb.Unmarshaler{AllowUnknownFields: allowUnknownFields}
+	return unmarshaler.UnmarshalNext(d, p)
+}
+
+func decodeNonProtoField(d *json.Decoder, v interface{}) error {
+	rv := reflect.ValueOf(v)
+	if rv.Kind() != reflect.Ptr {
+		return fmt.Errorf("%T is not a pointer", v)
+	}
+	for rv.Kind() == reflect.Ptr {
+		if rv.IsNil() {
+			rv.Set(reflect.New(rv.Type().Elem()))
+		}
+		if rv.Type().ConvertibleTo(typeProtoMessage) {
+			unmarshaler := &jsonpb.Unmarshaler{AllowUnknownFields: allowUnknownFields}
+			return unmarshaler.UnmarshalNext(d, rv.Interface().(proto.Message))
+		}
+		rv = rv.Elem()
+	}
+	if rv.Kind() == reflect.Map {
+		if rv.IsNil() {
+			rv.Set(reflect.MakeMap(rv.Type()))
+		}
+		conv, ok := convFromType[rv.Type().Key().Kind()]
+		if !ok {
+			return fmt.Errorf("unsupported type of map field key: %v", rv.Type().Key())
+		}
+
+		m := make(map[string]*json.RawMessage)
+		if err := d.Decode(&m); err != nil {
+			return err
+		}
+		for k, v := range m {
+			result := conv.Call([]reflect.Value{reflect.ValueOf(k)})
+			if err := result[1].Interface(); err != nil {
+				return err.(error)
+			}
+			bk := result[0]
+			bv := reflect.New(rv.Type().Elem())
+			if err := unmarshalJSONPb([]byte(*v), bv.Interface()); err != nil {
+				return err
+			}
+			rv.SetMapIndex(bk, bv.Elem())
+		}
+		return nil
+	}
+	if _, ok := rv.Interface().(protoEnum); ok {
+		var repr interface{}
+		if err := d.Decode(&repr); err != nil {
+			return err
+		}
+		switch repr.(type) {
+		case string:
+			// TODO(yugui) Should use proto.StructProperties?
+			return fmt.Errorf("unmarshaling of symbolic enum %q not supported: %T", repr, rv.Interface())
+		case float64:
+			rv.Set(reflect.ValueOf(int32(repr.(float64))).Convert(rv.Type()))
+			return nil
+		default:
+			return fmt.Errorf("cannot assign %#v into Go type %T", repr, rv.Interface())
+		}
+	}
+	return d.Decode(v)
+}
+
+type protoEnum interface {
+	fmt.Stringer
+	EnumDescriptor() ([]byte, []int)
+}
+
+var typeProtoMessage = reflect.TypeOf((*proto.Message)(nil)).Elem()
+
+// Delimiter for newline encoded JSON streams.
+func (j *JSONPb) Delimiter() []byte {
+	return []byte("\n")
+}
+
+// allowUnknownFields helps not to return an error when the destination
+// is a struct and the input contains object keys which do not match any
+// non-ignored, exported fields in the destination.
+var allowUnknownFields = true
+
+// DisallowUnknownFields enables option in decoder (unmarshaller) to
+// return an error when it finds an unknown field. This function must be
+// called before using the JSON marshaller.
+func DisallowUnknownFields() {
+	allowUnknownFields = false
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/marshal_proto.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/marshal_proto.go
new file mode 100644
index 0000000..f65d1a2
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/marshal_proto.go
@@ -0,0 +1,62 @@
+package runtime
+
+import (
+	"io"
+
+	"errors"
+	"github.com/golang/protobuf/proto"
+	"io/ioutil"
+)
+
+// ProtoMarshaller is a Marshaller which marshals/unmarshals into/from serialize proto bytes
+type ProtoMarshaller struct{}
+
+// ContentType always returns "application/octet-stream".
+func (*ProtoMarshaller) ContentType() string {
+	return "application/octet-stream"
+}
+
+// Marshal marshals "value" into Proto
+func (*ProtoMarshaller) Marshal(value interface{}) ([]byte, error) {
+	message, ok := value.(proto.Message)
+	if !ok {
+		return nil, errors.New("unable to marshal non proto field")
+	}
+	return proto.Marshal(message)
+}
+
+// Unmarshal unmarshals proto "data" into "value"
+func (*ProtoMarshaller) Unmarshal(data []byte, value interface{}) error {
+	message, ok := value.(proto.Message)
+	if !ok {
+		return errors.New("unable to unmarshal non proto field")
+	}
+	return proto.Unmarshal(data, message)
+}
+
+// NewDecoder returns a Decoder which reads proto stream from "reader".
+func (marshaller *ProtoMarshaller) NewDecoder(reader io.Reader) Decoder {
+	return DecoderFunc(func(value interface{}) error {
+		buffer, err := ioutil.ReadAll(reader)
+		if err != nil {
+			return err
+		}
+		return marshaller.Unmarshal(buffer, value)
+	})
+}
+
+// NewEncoder returns an Encoder which writes proto stream into "writer".
+func (marshaller *ProtoMarshaller) NewEncoder(writer io.Writer) Encoder {
+	return EncoderFunc(func(value interface{}) error {
+		buffer, err := marshaller.Marshal(value)
+		if err != nil {
+			return err
+		}
+		_, err = writer.Write(buffer)
+		if err != nil {
+			return err
+		}
+
+		return nil
+	})
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/marshaler.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/marshaler.go
new file mode 100644
index 0000000..98fe6e8
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/marshaler.go
@@ -0,0 +1,48 @@
+package runtime
+
+import (
+	"io"
+)
+
+// Marshaler defines a conversion between byte sequence and gRPC payloads / fields.
+type Marshaler interface {
+	// Marshal marshals "v" into byte sequence.
+	Marshal(v interface{}) ([]byte, error)
+	// Unmarshal unmarshals "data" into "v".
+	// "v" must be a pointer value.
+	Unmarshal(data []byte, v interface{}) error
+	// NewDecoder returns a Decoder which reads byte sequence from "r".
+	NewDecoder(r io.Reader) Decoder
+	// NewEncoder returns an Encoder which writes bytes sequence into "w".
+	NewEncoder(w io.Writer) Encoder
+	// ContentType returns the Content-Type which this marshaler is responsible for.
+	ContentType() string
+}
+
+// Decoder decodes a byte sequence
+type Decoder interface {
+	Decode(v interface{}) error
+}
+
+// Encoder encodes gRPC payloads / fields into byte sequence.
+type Encoder interface {
+	Encode(v interface{}) error
+}
+
+// DecoderFunc adapts an decoder function into Decoder.
+type DecoderFunc func(v interface{}) error
+
+// Decode delegates invocations to the underlying function itself.
+func (f DecoderFunc) Decode(v interface{}) error { return f(v) }
+
+// EncoderFunc adapts an encoder function into Encoder
+type EncoderFunc func(v interface{}) error
+
+// Encode delegates invocations to the underlying function itself.
+func (f EncoderFunc) Encode(v interface{}) error { return f(v) }
+
+// Delimited defines the streaming delimiter.
+type Delimited interface {
+	// Delimiter returns the record seperator for the stream.
+	Delimiter() []byte
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/marshaler_registry.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/marshaler_registry.go
new file mode 100644
index 0000000..5cc53ae
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/marshaler_registry.go
@@ -0,0 +1,91 @@
+package runtime
+
+import (
+	"errors"
+	"net/http"
+)
+
+// MIMEWildcard is the fallback MIME type used for requests which do not match
+// a registered MIME type.
+const MIMEWildcard = "*"
+
+var (
+	acceptHeader      = http.CanonicalHeaderKey("Accept")
+	contentTypeHeader = http.CanonicalHeaderKey("Content-Type")
+
+	defaultMarshaler = &JSONPb{OrigName: true}
+)
+
+// MarshalerForRequest returns the inbound/outbound marshalers for this request.
+// It checks the registry on the ServeMux for the MIME type set by the Content-Type header.
+// If it isn't set (or the request Content-Type is empty), checks for "*".
+// If there are multiple Content-Type headers set, choose the first one that it can
+// exactly match in the registry.
+// Otherwise, it follows the above logic for "*"/InboundMarshaler/OutboundMarshaler.
+func MarshalerForRequest(mux *ServeMux, r *http.Request) (inbound Marshaler, outbound Marshaler) {
+	for _, acceptVal := range r.Header[acceptHeader] {
+		if m, ok := mux.marshalers.mimeMap[acceptVal]; ok {
+			outbound = m
+			break
+		}
+	}
+
+	for _, contentTypeVal := range r.Header[contentTypeHeader] {
+		if m, ok := mux.marshalers.mimeMap[contentTypeVal]; ok {
+			inbound = m
+			break
+		}
+	}
+
+	if inbound == nil {
+		inbound = mux.marshalers.mimeMap[MIMEWildcard]
+	}
+	if outbound == nil {
+		outbound = inbound
+	}
+
+	return inbound, outbound
+}
+
+// marshalerRegistry is a mapping from MIME types to Marshalers.
+type marshalerRegistry struct {
+	mimeMap map[string]Marshaler
+}
+
+// add adds a marshaler for a case-sensitive MIME type string ("*" to match any
+// MIME type).
+func (m marshalerRegistry) add(mime string, marshaler Marshaler) error {
+	if len(mime) == 0 {
+		return errors.New("empty MIME type")
+	}
+
+	m.mimeMap[mime] = marshaler
+
+	return nil
+}
+
+// makeMarshalerMIMERegistry returns a new registry of marshalers.
+// It allows for a mapping of case-sensitive Content-Type MIME type string to runtime.Marshaler interfaces.
+//
+// For example, you could allow the client to specify the use of the runtime.JSONPb marshaler
+// with a "application/jsonpb" Content-Type and the use of the runtime.JSONBuiltin marshaler
+// with a "application/json" Content-Type.
+// "*" can be used to match any Content-Type.
+// This can be attached to a ServerMux with the marshaler option.
+func makeMarshalerMIMERegistry() marshalerRegistry {
+	return marshalerRegistry{
+		mimeMap: map[string]Marshaler{
+			MIMEWildcard: defaultMarshaler,
+		},
+	}
+}
+
+// WithMarshalerOption returns a ServeMuxOption which associates inbound and outbound
+// Marshalers to a MIME type in mux.
+func WithMarshalerOption(mime string, marshaler Marshaler) ServeMuxOption {
+	return func(mux *ServeMux) {
+		if err := mux.marshalers.add(mime, marshaler); err != nil {
+			panic(err)
+		}
+	}
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/mux.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/mux.go
new file mode 100644
index 0000000..1da3a58
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/mux.go
@@ -0,0 +1,303 @@
+package runtime
+
+import (
+	"context"
+	"fmt"
+	"net/http"
+	"net/textproto"
+	"strings"
+
+	"github.com/golang/protobuf/proto"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/metadata"
+	"google.golang.org/grpc/status"
+)
+
+// A HandlerFunc handles a specific pair of path pattern and HTTP method.
+type HandlerFunc func(w http.ResponseWriter, r *http.Request, pathParams map[string]string)
+
+// ErrUnknownURI is the error supplied to a custom ProtoErrorHandlerFunc when
+// a request is received with a URI path that does not match any registered
+// service method.
+//
+// Since gRPC servers return an "Unimplemented" code for requests with an
+// unrecognized URI path, this error also has a gRPC "Unimplemented" code.
+var ErrUnknownURI = status.Error(codes.Unimplemented, http.StatusText(http.StatusNotImplemented))
+
+// ServeMux is a request multiplexer for grpc-gateway.
+// It matches http requests to patterns and invokes the corresponding handler.
+type ServeMux struct {
+	// handlers maps HTTP method to a list of handlers.
+	handlers                  map[string][]handler
+	forwardResponseOptions    []func(context.Context, http.ResponseWriter, proto.Message) error
+	marshalers                marshalerRegistry
+	incomingHeaderMatcher     HeaderMatcherFunc
+	outgoingHeaderMatcher     HeaderMatcherFunc
+	metadataAnnotators        []func(context.Context, *http.Request) metadata.MD
+	streamErrorHandler        StreamErrorHandlerFunc
+	protoErrorHandler         ProtoErrorHandlerFunc
+	disablePathLengthFallback bool
+	lastMatchWins             bool
+}
+
+// ServeMuxOption is an option that can be given to a ServeMux on construction.
+type ServeMuxOption func(*ServeMux)
+
+// WithForwardResponseOption returns a ServeMuxOption representing the forwardResponseOption.
+//
+// forwardResponseOption is an option that will be called on the relevant context.Context,
+// http.ResponseWriter, and proto.Message before every forwarded response.
+//
+// The message may be nil in the case where just a header is being sent.
+func WithForwardResponseOption(forwardResponseOption func(context.Context, http.ResponseWriter, proto.Message) error) ServeMuxOption {
+	return func(serveMux *ServeMux) {
+		serveMux.forwardResponseOptions = append(serveMux.forwardResponseOptions, forwardResponseOption)
+	}
+}
+
+// HeaderMatcherFunc checks whether a header key should be forwarded to/from gRPC context.
+type HeaderMatcherFunc func(string) (string, bool)
+
+// DefaultHeaderMatcher is used to pass http request headers to/from gRPC context. This adds permanent HTTP header
+// keys (as specified by the IANA) to gRPC context with grpcgateway- prefix. HTTP headers that start with
+// 'Grpc-Metadata-' are mapped to gRPC metadata after removing prefix 'Grpc-Metadata-'.
+func DefaultHeaderMatcher(key string) (string, bool) {
+	key = textproto.CanonicalMIMEHeaderKey(key)
+	if isPermanentHTTPHeader(key) {
+		return MetadataPrefix + key, true
+	} else if strings.HasPrefix(key, MetadataHeaderPrefix) {
+		return key[len(MetadataHeaderPrefix):], true
+	}
+	return "", false
+}
+
+// WithIncomingHeaderMatcher returns a ServeMuxOption representing a headerMatcher for incoming request to gateway.
+//
+// This matcher will be called with each header in http.Request. If matcher returns true, that header will be
+// passed to gRPC context. To transform the header before passing to gRPC context, matcher should return modified header.
+func WithIncomingHeaderMatcher(fn HeaderMatcherFunc) ServeMuxOption {
+	return func(mux *ServeMux) {
+		mux.incomingHeaderMatcher = fn
+	}
+}
+
+// WithOutgoingHeaderMatcher returns a ServeMuxOption representing a headerMatcher for outgoing response from gateway.
+//
+// This matcher will be called with each header in response header metadata. If matcher returns true, that header will be
+// passed to http response returned from gateway. To transform the header before passing to response,
+// matcher should return modified header.
+func WithOutgoingHeaderMatcher(fn HeaderMatcherFunc) ServeMuxOption {
+	return func(mux *ServeMux) {
+		mux.outgoingHeaderMatcher = fn
+	}
+}
+
+// WithMetadata returns a ServeMuxOption for passing metadata to a gRPC context.
+//
+// This can be used by services that need to read from http.Request and modify gRPC context. A common use case
+// is reading token from cookie and adding it in gRPC context.
+func WithMetadata(annotator func(context.Context, *http.Request) metadata.MD) ServeMuxOption {
+	return func(serveMux *ServeMux) {
+		serveMux.metadataAnnotators = append(serveMux.metadataAnnotators, annotator)
+	}
+}
+
+// WithProtoErrorHandler returns a ServeMuxOption for passing metadata to a gRPC context.
+//
+// This can be used to handle an error as general proto message defined by gRPC.
+// The response including body and status is not backward compatible with the default error handler.
+// When this option is used, HTTPError and OtherErrorHandler are overwritten on initialization.
+func WithProtoErrorHandler(fn ProtoErrorHandlerFunc) ServeMuxOption {
+	return func(serveMux *ServeMux) {
+		serveMux.protoErrorHandler = fn
+	}
+}
+
+// WithDisablePathLengthFallback returns a ServeMuxOption for disable path length fallback.
+func WithDisablePathLengthFallback() ServeMuxOption {
+	return func(serveMux *ServeMux) {
+		serveMux.disablePathLengthFallback = true
+	}
+}
+
+// WithStreamErrorHandler returns a ServeMuxOption that will use the given custom stream
+// error handler, which allows for customizing the error trailer for server-streaming
+// calls.
+//
+// For stream errors that occur before any response has been written, the mux's
+// ProtoErrorHandler will be invoked. However, once data has been written, the errors must
+// be handled differently: they must be included in the response body. The response body's
+// final message will include the error details returned by the stream error handler.
+func WithStreamErrorHandler(fn StreamErrorHandlerFunc) ServeMuxOption {
+	return func(serveMux *ServeMux) {
+		serveMux.streamErrorHandler = fn
+	}
+}
+
+// WithLastMatchWins returns a ServeMuxOption that will enable "last
+// match wins" behavior, where if multiple path patterns match a
+// request path, the last one defined in the .proto file will be used.
+func WithLastMatchWins() ServeMuxOption {
+	return func(serveMux *ServeMux) {
+		serveMux.lastMatchWins = true
+	}
+}
+
+// NewServeMux returns a new ServeMux whose internal mapping is empty.
+func NewServeMux(opts ...ServeMuxOption) *ServeMux {
+	serveMux := &ServeMux{
+		handlers:               make(map[string][]handler),
+		forwardResponseOptions: make([]func(context.Context, http.ResponseWriter, proto.Message) error, 0),
+		marshalers:             makeMarshalerMIMERegistry(),
+		streamErrorHandler:     DefaultHTTPStreamErrorHandler,
+	}
+
+	for _, opt := range opts {
+		opt(serveMux)
+	}
+
+	if serveMux.protoErrorHandler != nil {
+		HTTPError = serveMux.protoErrorHandler
+		// OtherErrorHandler is no longer used when protoErrorHandler is set.
+		// Overwritten by a special error handler to return Unknown.
+		OtherErrorHandler = func(w http.ResponseWriter, r *http.Request, _ string, _ int) {
+			ctx := context.Background()
+			_, outboundMarshaler := MarshalerForRequest(serveMux, r)
+			sterr := status.Error(codes.Unknown, "unexpected use of OtherErrorHandler")
+			serveMux.protoErrorHandler(ctx, serveMux, outboundMarshaler, w, r, sterr)
+		}
+	}
+
+	if serveMux.incomingHeaderMatcher == nil {
+		serveMux.incomingHeaderMatcher = DefaultHeaderMatcher
+	}
+
+	if serveMux.outgoingHeaderMatcher == nil {
+		serveMux.outgoingHeaderMatcher = func(key string) (string, bool) {
+			return fmt.Sprintf("%s%s", MetadataHeaderPrefix, key), true
+		}
+	}
+
+	return serveMux
+}
+
+// Handle associates "h" to the pair of HTTP method and path pattern.
+func (s *ServeMux) Handle(meth string, pat Pattern, h HandlerFunc) {
+	if s.lastMatchWins {
+		s.handlers[meth] = append([]handler{handler{pat: pat, h: h}}, s.handlers[meth]...)
+	} else {
+		s.handlers[meth] = append(s.handlers[meth], handler{pat: pat, h: h})
+	}
+}
+
+// ServeHTTP dispatches the request to the first handler whose pattern matches to r.Method and r.Path.
+func (s *ServeMux) ServeHTTP(w http.ResponseWriter, r *http.Request) {
+	ctx := r.Context()
+
+	path := r.URL.Path
+	if !strings.HasPrefix(path, "/") {
+		if s.protoErrorHandler != nil {
+			_, outboundMarshaler := MarshalerForRequest(s, r)
+			sterr := status.Error(codes.InvalidArgument, http.StatusText(http.StatusBadRequest))
+			s.protoErrorHandler(ctx, s, outboundMarshaler, w, r, sterr)
+		} else {
+			OtherErrorHandler(w, r, http.StatusText(http.StatusBadRequest), http.StatusBadRequest)
+		}
+		return
+	}
+
+	components := strings.Split(path[1:], "/")
+	l := len(components)
+	var verb string
+	if idx := strings.LastIndex(components[l-1], ":"); idx == 0 {
+		if s.protoErrorHandler != nil {
+			_, outboundMarshaler := MarshalerForRequest(s, r)
+			s.protoErrorHandler(ctx, s, outboundMarshaler, w, r, ErrUnknownURI)
+		} else {
+			OtherErrorHandler(w, r, http.StatusText(http.StatusNotFound), http.StatusNotFound)
+		}
+		return
+	} else if idx > 0 {
+		c := components[l-1]
+		components[l-1], verb = c[:idx], c[idx+1:]
+	}
+
+	if override := r.Header.Get("X-HTTP-Method-Override"); override != "" && s.isPathLengthFallback(r) {
+		r.Method = strings.ToUpper(override)
+		if err := r.ParseForm(); err != nil {
+			if s.protoErrorHandler != nil {
+				_, outboundMarshaler := MarshalerForRequest(s, r)
+				sterr := status.Error(codes.InvalidArgument, err.Error())
+				s.protoErrorHandler(ctx, s, outboundMarshaler, w, r, sterr)
+			} else {
+				OtherErrorHandler(w, r, err.Error(), http.StatusBadRequest)
+			}
+			return
+		}
+	}
+	for _, h := range s.handlers[r.Method] {
+		pathParams, err := h.pat.Match(components, verb)
+		if err != nil {
+			continue
+		}
+		h.h(w, r, pathParams)
+		return
+	}
+
+	// lookup other methods to handle fallback from GET to POST and
+	// to determine if it is MethodNotAllowed or NotFound.
+	for m, handlers := range s.handlers {
+		if m == r.Method {
+			continue
+		}
+		for _, h := range handlers {
+			pathParams, err := h.pat.Match(components, verb)
+			if err != nil {
+				continue
+			}
+			// X-HTTP-Method-Override is optional. Always allow fallback to POST.
+			if s.isPathLengthFallback(r) {
+				if err := r.ParseForm(); err != nil {
+					if s.protoErrorHandler != nil {
+						_, outboundMarshaler := MarshalerForRequest(s, r)
+						sterr := status.Error(codes.InvalidArgument, err.Error())
+						s.protoErrorHandler(ctx, s, outboundMarshaler, w, r, sterr)
+					} else {
+						OtherErrorHandler(w, r, err.Error(), http.StatusBadRequest)
+					}
+					return
+				}
+				h.h(w, r, pathParams)
+				return
+			}
+			if s.protoErrorHandler != nil {
+				_, outboundMarshaler := MarshalerForRequest(s, r)
+				s.protoErrorHandler(ctx, s, outboundMarshaler, w, r, ErrUnknownURI)
+			} else {
+				OtherErrorHandler(w, r, http.StatusText(http.StatusMethodNotAllowed), http.StatusMethodNotAllowed)
+			}
+			return
+		}
+	}
+
+	if s.protoErrorHandler != nil {
+		_, outboundMarshaler := MarshalerForRequest(s, r)
+		s.protoErrorHandler(ctx, s, outboundMarshaler, w, r, ErrUnknownURI)
+	} else {
+		OtherErrorHandler(w, r, http.StatusText(http.StatusNotFound), http.StatusNotFound)
+	}
+}
+
+// GetForwardResponseOptions returns the ForwardResponseOptions associated with this ServeMux.
+func (s *ServeMux) GetForwardResponseOptions() []func(context.Context, http.ResponseWriter, proto.Message) error {
+	return s.forwardResponseOptions
+}
+
+func (s *ServeMux) isPathLengthFallback(r *http.Request) bool {
+	return !s.disablePathLengthFallback && r.Method == "POST" && r.Header.Get("Content-Type") == "application/x-www-form-urlencoded"
+}
+
+type handler struct {
+	pat Pattern
+	h   HandlerFunc
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/pattern.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/pattern.go
new file mode 100644
index 0000000..0905369
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/pattern.go
@@ -0,0 +1,262 @@
+package runtime
+
+import (
+	"errors"
+	"fmt"
+	"strings"
+
+	"github.com/grpc-ecosystem/grpc-gateway/utilities"
+	"google.golang.org/grpc/grpclog"
+)
+
+var (
+	// ErrNotMatch indicates that the given HTTP request path does not match to the pattern.
+	ErrNotMatch = errors.New("not match to the path pattern")
+	// ErrInvalidPattern indicates that the given definition of Pattern is not valid.
+	ErrInvalidPattern = errors.New("invalid pattern")
+)
+
+type op struct {
+	code    utilities.OpCode
+	operand int
+}
+
+// Pattern is a template pattern of http request paths defined in github.com/googleapis/googleapis/google/api/http.proto.
+type Pattern struct {
+	// ops is a list of operations
+	ops []op
+	// pool is a constant pool indexed by the operands or vars.
+	pool []string
+	// vars is a list of variables names to be bound by this pattern
+	vars []string
+	// stacksize is the max depth of the stack
+	stacksize int
+	// tailLen is the length of the fixed-size segments after a deep wildcard
+	tailLen int
+	// verb is the VERB part of the path pattern. It is empty if the pattern does not have VERB part.
+	verb string
+	// assumeColonVerb indicates whether a path suffix after a final
+	// colon may only be interpreted as a verb.
+	assumeColonVerb bool
+}
+
+type patternOptions struct {
+	assumeColonVerb bool
+}
+
+// PatternOpt is an option for creating Patterns.
+type PatternOpt func(*patternOptions)
+
+// NewPattern returns a new Pattern from the given definition values.
+// "ops" is a sequence of op codes. "pool" is a constant pool.
+// "verb" is the verb part of the pattern. It is empty if the pattern does not have the part.
+// "version" must be 1 for now.
+// It returns an error if the given definition is invalid.
+func NewPattern(version int, ops []int, pool []string, verb string, opts ...PatternOpt) (Pattern, error) {
+	options := patternOptions{
+		assumeColonVerb: true,
+	}
+	for _, o := range opts {
+		o(&options)
+	}
+
+	if version != 1 {
+		grpclog.Infof("unsupported version: %d", version)
+		return Pattern{}, ErrInvalidPattern
+	}
+
+	l := len(ops)
+	if l%2 != 0 {
+		grpclog.Infof("odd number of ops codes: %d", l)
+		return Pattern{}, ErrInvalidPattern
+	}
+
+	var (
+		typedOps        []op
+		stack, maxstack int
+		tailLen         int
+		pushMSeen       bool
+		vars            []string
+	)
+	for i := 0; i < l; i += 2 {
+		op := op{code: utilities.OpCode(ops[i]), operand: ops[i+1]}
+		switch op.code {
+		case utilities.OpNop:
+			continue
+		case utilities.OpPush:
+			if pushMSeen {
+				tailLen++
+			}
+			stack++
+		case utilities.OpPushM:
+			if pushMSeen {
+				grpclog.Infof("pushM appears twice")
+				return Pattern{}, ErrInvalidPattern
+			}
+			pushMSeen = true
+			stack++
+		case utilities.OpLitPush:
+			if op.operand < 0 || len(pool) <= op.operand {
+				grpclog.Infof("negative literal index: %d", op.operand)
+				return Pattern{}, ErrInvalidPattern
+			}
+			if pushMSeen {
+				tailLen++
+			}
+			stack++
+		case utilities.OpConcatN:
+			if op.operand <= 0 {
+				grpclog.Infof("negative concat size: %d", op.operand)
+				return Pattern{}, ErrInvalidPattern
+			}
+			stack -= op.operand
+			if stack < 0 {
+				grpclog.Print("stack underflow")
+				return Pattern{}, ErrInvalidPattern
+			}
+			stack++
+		case utilities.OpCapture:
+			if op.operand < 0 || len(pool) <= op.operand {
+				grpclog.Infof("variable name index out of bound: %d", op.operand)
+				return Pattern{}, ErrInvalidPattern
+			}
+			v := pool[op.operand]
+			op.operand = len(vars)
+			vars = append(vars, v)
+			stack--
+			if stack < 0 {
+				grpclog.Infof("stack underflow")
+				return Pattern{}, ErrInvalidPattern
+			}
+		default:
+			grpclog.Infof("invalid opcode: %d", op.code)
+			return Pattern{}, ErrInvalidPattern
+		}
+
+		if maxstack < stack {
+			maxstack = stack
+		}
+		typedOps = append(typedOps, op)
+	}
+	return Pattern{
+		ops:             typedOps,
+		pool:            pool,
+		vars:            vars,
+		stacksize:       maxstack,
+		tailLen:         tailLen,
+		verb:            verb,
+		assumeColonVerb: options.assumeColonVerb,
+	}, nil
+}
+
+// MustPattern is a helper function which makes it easier to call NewPattern in variable initialization.
+func MustPattern(p Pattern, err error) Pattern {
+	if err != nil {
+		grpclog.Fatalf("Pattern initialization failed: %v", err)
+	}
+	return p
+}
+
+// Match examines components if it matches to the Pattern.
+// If it matches, the function returns a mapping from field paths to their captured values.
+// If otherwise, the function returns an error.
+func (p Pattern) Match(components []string, verb string) (map[string]string, error) {
+	if p.verb != verb {
+		if p.assumeColonVerb || p.verb != "" {
+			return nil, ErrNotMatch
+		}
+		if len(components) == 0 {
+			components = []string{":" + verb}
+		} else {
+			components = append([]string{}, components...)
+			components[len(components)-1] += ":" + verb
+		}
+		verb = ""
+	}
+
+	var pos int
+	stack := make([]string, 0, p.stacksize)
+	captured := make([]string, len(p.vars))
+	l := len(components)
+	for _, op := range p.ops {
+		switch op.code {
+		case utilities.OpNop:
+			continue
+		case utilities.OpPush, utilities.OpLitPush:
+			if pos >= l {
+				return nil, ErrNotMatch
+			}
+			c := components[pos]
+			if op.code == utilities.OpLitPush {
+				if lit := p.pool[op.operand]; c != lit {
+					return nil, ErrNotMatch
+				}
+			}
+			stack = append(stack, c)
+			pos++
+		case utilities.OpPushM:
+			end := len(components)
+			if end < pos+p.tailLen {
+				return nil, ErrNotMatch
+			}
+			end -= p.tailLen
+			stack = append(stack, strings.Join(components[pos:end], "/"))
+			pos = end
+		case utilities.OpConcatN:
+			n := op.operand
+			l := len(stack) - n
+			stack = append(stack[:l], strings.Join(stack[l:], "/"))
+		case utilities.OpCapture:
+			n := len(stack) - 1
+			captured[op.operand] = stack[n]
+			stack = stack[:n]
+		}
+	}
+	if pos < l {
+		return nil, ErrNotMatch
+	}
+	bindings := make(map[string]string)
+	for i, val := range captured {
+		bindings[p.vars[i]] = val
+	}
+	return bindings, nil
+}
+
+// Verb returns the verb part of the Pattern.
+func (p Pattern) Verb() string { return p.verb }
+
+func (p Pattern) String() string {
+	var stack []string
+	for _, op := range p.ops {
+		switch op.code {
+		case utilities.OpNop:
+			continue
+		case utilities.OpPush:
+			stack = append(stack, "*")
+		case utilities.OpLitPush:
+			stack = append(stack, p.pool[op.operand])
+		case utilities.OpPushM:
+			stack = append(stack, "**")
+		case utilities.OpConcatN:
+			n := op.operand
+			l := len(stack) - n
+			stack = append(stack[:l], strings.Join(stack[l:], "/"))
+		case utilities.OpCapture:
+			n := len(stack) - 1
+			stack[n] = fmt.Sprintf("{%s=%s}", p.vars[op.operand], stack[n])
+		}
+	}
+	segs := strings.Join(stack, "/")
+	if p.verb != "" {
+		return fmt.Sprintf("/%s:%s", segs, p.verb)
+	}
+	return "/" + segs
+}
+
+// AssumeColonVerbOpt indicates whether a path suffix after a final
+// colon may only be interpreted as a verb.
+func AssumeColonVerbOpt(val bool) PatternOpt {
+	return PatternOpt(func(o *patternOptions) {
+		o.assumeColonVerb = val
+	})
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/proto2_convert.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/proto2_convert.go
new file mode 100644
index 0000000..a3151e2
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/proto2_convert.go
@@ -0,0 +1,80 @@
+package runtime
+
+import (
+	"github.com/golang/protobuf/proto"
+)
+
+// StringP returns a pointer to a string whose pointee is same as the given string value.
+func StringP(val string) (*string, error) {
+	return proto.String(val), nil
+}
+
+// BoolP parses the given string representation of a boolean value,
+// and returns a pointer to a bool whose value is same as the parsed value.
+func BoolP(val string) (*bool, error) {
+	b, err := Bool(val)
+	if err != nil {
+		return nil, err
+	}
+	return proto.Bool(b), nil
+}
+
+// Float64P parses the given string representation of a floating point number,
+// and returns a pointer to a float64 whose value is same as the parsed number.
+func Float64P(val string) (*float64, error) {
+	f, err := Float64(val)
+	if err != nil {
+		return nil, err
+	}
+	return proto.Float64(f), nil
+}
+
+// Float32P parses the given string representation of a floating point number,
+// and returns a pointer to a float32 whose value is same as the parsed number.
+func Float32P(val string) (*float32, error) {
+	f, err := Float32(val)
+	if err != nil {
+		return nil, err
+	}
+	return proto.Float32(f), nil
+}
+
+// Int64P parses the given string representation of an integer
+// and returns a pointer to a int64 whose value is same as the parsed integer.
+func Int64P(val string) (*int64, error) {
+	i, err := Int64(val)
+	if err != nil {
+		return nil, err
+	}
+	return proto.Int64(i), nil
+}
+
+// Int32P parses the given string representation of an integer
+// and returns a pointer to a int32 whose value is same as the parsed integer.
+func Int32P(val string) (*int32, error) {
+	i, err := Int32(val)
+	if err != nil {
+		return nil, err
+	}
+	return proto.Int32(i), err
+}
+
+// Uint64P parses the given string representation of an integer
+// and returns a pointer to a uint64 whose value is same as the parsed integer.
+func Uint64P(val string) (*uint64, error) {
+	i, err := Uint64(val)
+	if err != nil {
+		return nil, err
+	}
+	return proto.Uint64(i), err
+}
+
+// Uint32P parses the given string representation of an integer
+// and returns a pointer to a uint32 whose value is same as the parsed integer.
+func Uint32P(val string) (*uint32, error) {
+	i, err := Uint32(val)
+	if err != nil {
+		return nil, err
+	}
+	return proto.Uint32(i), err
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/proto_errors.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/proto_errors.go
new file mode 100644
index 0000000..ca76324
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/proto_errors.go
@@ -0,0 +1,106 @@
+package runtime
+
+import (
+	"context"
+	"io"
+	"net/http"
+
+	"github.com/golang/protobuf/ptypes/any"
+	"github.com/grpc-ecosystem/grpc-gateway/internal"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/grpclog"
+	"google.golang.org/grpc/status"
+)
+
+// StreamErrorHandlerFunc accepts an error as a gRPC error generated via status package and translates it into a
+// a proto struct used to represent error at the end of a stream.
+type StreamErrorHandlerFunc func(context.Context, error) *StreamError
+
+// StreamError is the payload for the final message in a server stream in the event that the server returns an
+// error after a response message has already been sent.
+type StreamError internal.StreamError
+
+// ProtoErrorHandlerFunc handles the error as a gRPC error generated via status package and replies to the request.
+type ProtoErrorHandlerFunc func(context.Context, *ServeMux, Marshaler, http.ResponseWriter, *http.Request, error)
+
+var _ ProtoErrorHandlerFunc = DefaultHTTPProtoErrorHandler
+
+// DefaultHTTPProtoErrorHandler is an implementation of HTTPError.
+// If "err" is an error from gRPC system, the function replies with the status code mapped by HTTPStatusFromCode.
+// If otherwise, it replies with http.StatusInternalServerError.
+//
+// The response body returned by this function is a Status message marshaled by a Marshaler.
+//
+// Do not set this function to HTTPError variable directly, use WithProtoErrorHandler option instead.
+func DefaultHTTPProtoErrorHandler(ctx context.Context, mux *ServeMux, marshaler Marshaler, w http.ResponseWriter, _ *http.Request, err error) {
+	// return Internal when Marshal failed
+	const fallback = `{"code": 13, "message": "failed to marshal error message"}`
+
+	s, ok := status.FromError(err)
+	if !ok {
+		s = status.New(codes.Unknown, err.Error())
+	}
+
+	w.Header().Del("Trailer")
+
+	contentType := marshaler.ContentType()
+	// Check marshaler on run time in order to keep backwards compatability
+	// An interface param needs to be added to the ContentType() function on
+	// the Marshal interface to be able to remove this check
+	if httpBodyMarshaler, ok := marshaler.(*HTTPBodyMarshaler); ok {
+		pb := s.Proto()
+		contentType = httpBodyMarshaler.ContentTypeFromMessage(pb)
+	}
+	w.Header().Set("Content-Type", contentType)
+
+	buf, merr := marshaler.Marshal(s.Proto())
+	if merr != nil {
+		grpclog.Infof("Failed to marshal error message %q: %v", s.Proto(), merr)
+		w.WriteHeader(http.StatusInternalServerError)
+		if _, err := io.WriteString(w, fallback); err != nil {
+			grpclog.Infof("Failed to write response: %v", err)
+		}
+		return
+	}
+
+	md, ok := ServerMetadataFromContext(ctx)
+	if !ok {
+		grpclog.Infof("Failed to extract ServerMetadata from context")
+	}
+
+	handleForwardResponseServerMetadata(w, mux, md)
+	handleForwardResponseTrailerHeader(w, md)
+	st := HTTPStatusFromCode(s.Code())
+	w.WriteHeader(st)
+	if _, err := w.Write(buf); err != nil {
+		grpclog.Infof("Failed to write response: %v", err)
+	}
+
+	handleForwardResponseTrailer(w, md)
+}
+
+// DefaultHTTPStreamErrorHandler converts the given err into a *StreamError via
+// default logic.
+//
+// It extracts the gRPC status from err if possible. The fields of the status are
+// used to populate the returned StreamError, and the HTTP status code is derived
+// from the gRPC code via HTTPStatusFromCode. If the given err does not contain a
+// gRPC status, an "Unknown" gRPC code is used and "Internal Server Error" HTTP code.
+func DefaultHTTPStreamErrorHandler(_ context.Context, err error) *StreamError {
+	grpcCode := codes.Unknown
+	grpcMessage := err.Error()
+	var grpcDetails []*any.Any
+	if s, ok := status.FromError(err); ok {
+		grpcCode = s.Code()
+		grpcMessage = s.Message()
+		grpcDetails = s.Proto().GetDetails()
+	}
+	httpCode := HTTPStatusFromCode(grpcCode)
+	return &StreamError{
+		GrpcCode:   int32(grpcCode),
+		HttpCode:   int32(httpCode),
+		Message:    grpcMessage,
+		HttpStatus: http.StatusText(httpCode),
+		Details:    grpcDetails,
+	}
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/query.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/query.go
new file mode 100644
index 0000000..5fbba5e
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/runtime/query.go
@@ -0,0 +1,391 @@
+package runtime
+
+import (
+	"encoding/base64"
+	"fmt"
+	"net/url"
+	"reflect"
+	"regexp"
+	"strconv"
+	"strings"
+	"time"
+
+	"github.com/golang/protobuf/proto"
+	"github.com/grpc-ecosystem/grpc-gateway/utilities"
+	"google.golang.org/grpc/grpclog"
+)
+
+// PopulateQueryParameters populates "values" into "msg".
+// A value is ignored if its key starts with one of the elements in "filter".
+func PopulateQueryParameters(msg proto.Message, values url.Values, filter *utilities.DoubleArray) error {
+	for key, values := range values {
+		re, err := regexp.Compile("^(.*)\\[(.*)\\]$")
+		if err != nil {
+			return err
+		}
+		match := re.FindStringSubmatch(key)
+		if len(match) == 3 {
+			key = match[1]
+			values = append([]string{match[2]}, values...)
+		}
+		fieldPath := strings.Split(key, ".")
+		if filter.HasCommonPrefix(fieldPath) {
+			continue
+		}
+		if err := populateFieldValueFromPath(msg, fieldPath, values); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+// PopulateFieldFromPath sets a value in a nested Protobuf structure.
+// It instantiates missing protobuf fields as it goes.
+func PopulateFieldFromPath(msg proto.Message, fieldPathString string, value string) error {
+	fieldPath := strings.Split(fieldPathString, ".")
+	return populateFieldValueFromPath(msg, fieldPath, []string{value})
+}
+
+func populateFieldValueFromPath(msg proto.Message, fieldPath []string, values []string) error {
+	m := reflect.ValueOf(msg)
+	if m.Kind() != reflect.Ptr {
+		return fmt.Errorf("unexpected type %T: %v", msg, msg)
+	}
+	var props *proto.Properties
+	m = m.Elem()
+	for i, fieldName := range fieldPath {
+		isLast := i == len(fieldPath)-1
+		if !isLast && m.Kind() != reflect.Struct {
+			return fmt.Errorf("non-aggregate type in the mid of path: %s", strings.Join(fieldPath, "."))
+		}
+		var f reflect.Value
+		var err error
+		f, props, err = fieldByProtoName(m, fieldName)
+		if err != nil {
+			return err
+		} else if !f.IsValid() {
+			grpclog.Infof("field not found in %T: %s", msg, strings.Join(fieldPath, "."))
+			return nil
+		}
+
+		switch f.Kind() {
+		case reflect.Bool, reflect.Float32, reflect.Float64, reflect.Int32, reflect.Int64, reflect.String, reflect.Uint32, reflect.Uint64:
+			if !isLast {
+				return fmt.Errorf("unexpected nested field %s in %s", fieldPath[i+1], strings.Join(fieldPath[:i+1], "."))
+			}
+			m = f
+		case reflect.Slice:
+			if !isLast {
+				return fmt.Errorf("unexpected repeated field in %s", strings.Join(fieldPath, "."))
+			}
+			// Handle []byte
+			if f.Type().Elem().Kind() == reflect.Uint8 {
+				m = f
+				break
+			}
+			return populateRepeatedField(f, values, props)
+		case reflect.Ptr:
+			if f.IsNil() {
+				m = reflect.New(f.Type().Elem())
+				f.Set(m.Convert(f.Type()))
+			}
+			m = f.Elem()
+			continue
+		case reflect.Struct:
+			m = f
+			continue
+		case reflect.Map:
+			if !isLast {
+				return fmt.Errorf("unexpected nested field %s in %s", fieldPath[i+1], strings.Join(fieldPath[:i+1], "."))
+			}
+			return populateMapField(f, values, props)
+		default:
+			return fmt.Errorf("unexpected type %s in %T", f.Type(), msg)
+		}
+	}
+	switch len(values) {
+	case 0:
+		return fmt.Errorf("no value of field: %s", strings.Join(fieldPath, "."))
+	case 1:
+	default:
+		grpclog.Infof("too many field values: %s", strings.Join(fieldPath, "."))
+	}
+	return populateField(m, values[0], props)
+}
+
+// fieldByProtoName looks up a field whose corresponding protobuf field name is "name".
+// "m" must be a struct value. It returns zero reflect.Value if no such field found.
+func fieldByProtoName(m reflect.Value, name string) (reflect.Value, *proto.Properties, error) {
+	props := proto.GetProperties(m.Type())
+
+	// look up field name in oneof map
+	if op, ok := props.OneofTypes[name]; ok {
+		v := reflect.New(op.Type.Elem())
+		field := m.Field(op.Field)
+		if !field.IsNil() {
+			return reflect.Value{}, nil, fmt.Errorf("field already set for %s oneof", props.Prop[op.Field].OrigName)
+		}
+		field.Set(v)
+		return v.Elem().Field(0), op.Prop, nil
+	}
+
+	for _, p := range props.Prop {
+		if p.OrigName == name {
+			return m.FieldByName(p.Name), p, nil
+		}
+		if p.JSONName == name {
+			return m.FieldByName(p.Name), p, nil
+		}
+	}
+	return reflect.Value{}, nil, nil
+}
+
+func populateMapField(f reflect.Value, values []string, props *proto.Properties) error {
+	if len(values) != 2 {
+		return fmt.Errorf("more than one value provided for key %s in map %s", values[0], props.Name)
+	}
+
+	key, value := values[0], values[1]
+	keyType := f.Type().Key()
+	valueType := f.Type().Elem()
+	if f.IsNil() {
+		f.Set(reflect.MakeMap(f.Type()))
+	}
+
+	keyConv, ok := convFromType[keyType.Kind()]
+	if !ok {
+		return fmt.Errorf("unsupported key type %s in map %s", keyType, props.Name)
+	}
+	valueConv, ok := convFromType[valueType.Kind()]
+	if !ok {
+		return fmt.Errorf("unsupported value type %s in map %s", valueType, props.Name)
+	}
+
+	keyV := keyConv.Call([]reflect.Value{reflect.ValueOf(key)})
+	if err := keyV[1].Interface(); err != nil {
+		return err.(error)
+	}
+	valueV := valueConv.Call([]reflect.Value{reflect.ValueOf(value)})
+	if err := valueV[1].Interface(); err != nil {
+		return err.(error)
+	}
+
+	f.SetMapIndex(keyV[0].Convert(keyType), valueV[0].Convert(valueType))
+
+	return nil
+}
+
+func populateRepeatedField(f reflect.Value, values []string, props *proto.Properties) error {
+	elemType := f.Type().Elem()
+
+	// is the destination field a slice of an enumeration type?
+	if enumValMap := proto.EnumValueMap(props.Enum); enumValMap != nil {
+		return populateFieldEnumRepeated(f, values, enumValMap)
+	}
+
+	conv, ok := convFromType[elemType.Kind()]
+	if !ok {
+		return fmt.Errorf("unsupported field type %s", elemType)
+	}
+	f.Set(reflect.MakeSlice(f.Type(), len(values), len(values)).Convert(f.Type()))
+	for i, v := range values {
+		result := conv.Call([]reflect.Value{reflect.ValueOf(v)})
+		if err := result[1].Interface(); err != nil {
+			return err.(error)
+		}
+		f.Index(i).Set(result[0].Convert(f.Index(i).Type()))
+	}
+	return nil
+}
+
+func populateField(f reflect.Value, value string, props *proto.Properties) error {
+	i := f.Addr().Interface()
+
+	// Handle protobuf well known types
+	var name string
+	switch m := i.(type) {
+	case interface{ XXX_WellKnownType() string }:
+		name = m.XXX_WellKnownType()
+	case proto.Message:
+		const wktPrefix = "google.protobuf."
+		if fullName := proto.MessageName(m); strings.HasPrefix(fullName, wktPrefix) {
+			name = fullName[len(wktPrefix):]
+		}
+	}
+	switch name {
+	case "Timestamp":
+		if value == "null" {
+			f.FieldByName("Seconds").SetInt(0)
+			f.FieldByName("Nanos").SetInt(0)
+			return nil
+		}
+
+		t, err := time.Parse(time.RFC3339Nano, value)
+		if err != nil {
+			return fmt.Errorf("bad Timestamp: %v", err)
+		}
+		f.FieldByName("Seconds").SetInt(int64(t.Unix()))
+		f.FieldByName("Nanos").SetInt(int64(t.Nanosecond()))
+		return nil
+	case "Duration":
+		if value == "null" {
+			f.FieldByName("Seconds").SetInt(0)
+			f.FieldByName("Nanos").SetInt(0)
+			return nil
+		}
+		d, err := time.ParseDuration(value)
+		if err != nil {
+			return fmt.Errorf("bad Duration: %v", err)
+		}
+
+		ns := d.Nanoseconds()
+		s := ns / 1e9
+		ns %= 1e9
+		f.FieldByName("Seconds").SetInt(s)
+		f.FieldByName("Nanos").SetInt(ns)
+		return nil
+	case "DoubleValue":
+		fallthrough
+	case "FloatValue":
+		float64Val, err := strconv.ParseFloat(value, 64)
+		if err != nil {
+			return fmt.Errorf("bad DoubleValue: %s", value)
+		}
+		f.FieldByName("Value").SetFloat(float64Val)
+		return nil
+	case "Int64Value":
+		fallthrough
+	case "Int32Value":
+		int64Val, err := strconv.ParseInt(value, 10, 64)
+		if err != nil {
+			return fmt.Errorf("bad DoubleValue: %s", value)
+		}
+		f.FieldByName("Value").SetInt(int64Val)
+		return nil
+	case "UInt64Value":
+		fallthrough
+	case "UInt32Value":
+		uint64Val, err := strconv.ParseUint(value, 10, 64)
+		if err != nil {
+			return fmt.Errorf("bad DoubleValue: %s", value)
+		}
+		f.FieldByName("Value").SetUint(uint64Val)
+		return nil
+	case "BoolValue":
+		if value == "true" {
+			f.FieldByName("Value").SetBool(true)
+		} else if value == "false" {
+			f.FieldByName("Value").SetBool(false)
+		} else {
+			return fmt.Errorf("bad BoolValue: %s", value)
+		}
+		return nil
+	case "StringValue":
+		f.FieldByName("Value").SetString(value)
+		return nil
+	case "BytesValue":
+		bytesVal, err := base64.StdEncoding.DecodeString(value)
+		if err != nil {
+			return fmt.Errorf("bad BytesValue: %s", value)
+		}
+		f.FieldByName("Value").SetBytes(bytesVal)
+		return nil
+	case "FieldMask":
+		p := f.FieldByName("Paths")
+		for _, v := range strings.Split(value, ",") {
+			if v != "" {
+				p.Set(reflect.Append(p, reflect.ValueOf(v)))
+			}
+		}
+		return nil
+	}
+
+	// Handle Time and Duration stdlib types
+	switch t := i.(type) {
+	case *time.Time:
+		pt, err := time.Parse(time.RFC3339Nano, value)
+		if err != nil {
+			return fmt.Errorf("bad Timestamp: %v", err)
+		}
+		*t = pt
+		return nil
+	case *time.Duration:
+		d, err := time.ParseDuration(value)
+		if err != nil {
+			return fmt.Errorf("bad Duration: %v", err)
+		}
+		*t = d
+		return nil
+	}
+
+	// is the destination field an enumeration type?
+	if enumValMap := proto.EnumValueMap(props.Enum); enumValMap != nil {
+		return populateFieldEnum(f, value, enumValMap)
+	}
+
+	conv, ok := convFromType[f.Kind()]
+	if !ok {
+		return fmt.Errorf("field type %T is not supported in query parameters", i)
+	}
+	result := conv.Call([]reflect.Value{reflect.ValueOf(value)})
+	if err := result[1].Interface(); err != nil {
+		return err.(error)
+	}
+	f.Set(result[0].Convert(f.Type()))
+	return nil
+}
+
+func convertEnum(value string, t reflect.Type, enumValMap map[string]int32) (reflect.Value, error) {
+	// see if it's an enumeration string
+	if enumVal, ok := enumValMap[value]; ok {
+		return reflect.ValueOf(enumVal).Convert(t), nil
+	}
+
+	// check for an integer that matches an enumeration value
+	eVal, err := strconv.Atoi(value)
+	if err != nil {
+		return reflect.Value{}, fmt.Errorf("%s is not a valid %s", value, t)
+	}
+	for _, v := range enumValMap {
+		if v == int32(eVal) {
+			return reflect.ValueOf(eVal).Convert(t), nil
+		}
+	}
+	return reflect.Value{}, fmt.Errorf("%s is not a valid %s", value, t)
+}
+
+func populateFieldEnum(f reflect.Value, value string, enumValMap map[string]int32) error {
+	cval, err := convertEnum(value, f.Type(), enumValMap)
+	if err != nil {
+		return err
+	}
+	f.Set(cval)
+	return nil
+}
+
+func populateFieldEnumRepeated(f reflect.Value, values []string, enumValMap map[string]int32) error {
+	elemType := f.Type().Elem()
+	f.Set(reflect.MakeSlice(f.Type(), len(values), len(values)).Convert(f.Type()))
+	for i, v := range values {
+		result, err := convertEnum(v, elemType, enumValMap)
+		if err != nil {
+			return err
+		}
+		f.Index(i).Set(result)
+	}
+	return nil
+}
+
+var (
+	convFromType = map[reflect.Kind]reflect.Value{
+		reflect.String:  reflect.ValueOf(String),
+		reflect.Bool:    reflect.ValueOf(Bool),
+		reflect.Float64: reflect.ValueOf(Float64),
+		reflect.Float32: reflect.ValueOf(Float32),
+		reflect.Int64:   reflect.ValueOf(Int64),
+		reflect.Int32:   reflect.ValueOf(Int32),
+		reflect.Uint64:  reflect.ValueOf(Uint64),
+		reflect.Uint32:  reflect.ValueOf(Uint32),
+		reflect.Slice:   reflect.ValueOf(Bytes),
+	}
+)
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/utilities/BUILD.bazel b/vendor/github.com/grpc-ecosystem/grpc-gateway/utilities/BUILD.bazel
new file mode 100644
index 0000000..7109d79
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/utilities/BUILD.bazel
@@ -0,0 +1,21 @@
+load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
+
+package(default_visibility = ["//visibility:public"])
+
+go_library(
+    name = "go_default_library",
+    srcs = [
+        "doc.go",
+        "pattern.go",
+        "readerfactory.go",
+        "trie.go",
+    ],
+    importpath = "github.com/grpc-ecosystem/grpc-gateway/utilities",
+)
+
+go_test(
+    name = "go_default_test",
+    size = "small",
+    srcs = ["trie_test.go"],
+    embed = [":go_default_library"],
+)
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/utilities/doc.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/utilities/doc.go
new file mode 100644
index 0000000..cf79a4d
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/utilities/doc.go
@@ -0,0 +1,2 @@
+// Package utilities provides members for internal use in grpc-gateway.
+package utilities
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/utilities/pattern.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/utilities/pattern.go
new file mode 100644
index 0000000..dfe7de4
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/utilities/pattern.go
@@ -0,0 +1,22 @@
+package utilities
+
+// An OpCode is a opcode of compiled path patterns.
+type OpCode int
+
+// These constants are the valid values of OpCode.
+const (
+	// OpNop does nothing
+	OpNop = OpCode(iota)
+	// OpPush pushes a component to stack
+	OpPush
+	// OpLitPush pushes a component to stack if it matches to the literal
+	OpLitPush
+	// OpPushM concatenates the remaining components and pushes it to stack
+	OpPushM
+	// OpConcatN pops N items from stack, concatenates them and pushes it back to stack
+	OpConcatN
+	// OpCapture pops an item and binds it to the variable
+	OpCapture
+	// OpEnd is the least positive invalid opcode.
+	OpEnd
+)
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/utilities/readerfactory.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/utilities/readerfactory.go
new file mode 100644
index 0000000..6dd3854
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/utilities/readerfactory.go
@@ -0,0 +1,20 @@
+package utilities
+
+import (
+	"bytes"
+	"io"
+	"io/ioutil"
+)
+
+// IOReaderFactory takes in an io.Reader and returns a function that will allow you to create a new reader that begins
+// at the start of the stream
+func IOReaderFactory(r io.Reader) (func() io.Reader, error) {
+	b, err := ioutil.ReadAll(r)
+	if err != nil {
+		return nil, err
+	}
+
+	return func() io.Reader {
+		return bytes.NewReader(b)
+	}, nil
+}
diff --git a/vendor/github.com/grpc-ecosystem/grpc-gateway/utilities/trie.go b/vendor/github.com/grpc-ecosystem/grpc-gateway/utilities/trie.go
new file mode 100644
index 0000000..c2b7b30
--- /dev/null
+++ b/vendor/github.com/grpc-ecosystem/grpc-gateway/utilities/trie.go
@@ -0,0 +1,177 @@
+package utilities
+
+import (
+	"sort"
+)
+
+// DoubleArray is a Double Array implementation of trie on sequences of strings.
+type DoubleArray struct {
+	// Encoding keeps an encoding from string to int
+	Encoding map[string]int
+	// Base is the base array of Double Array
+	Base []int
+	// Check is the check array of Double Array
+	Check []int
+}
+
+// NewDoubleArray builds a DoubleArray from a set of sequences of strings.
+func NewDoubleArray(seqs [][]string) *DoubleArray {
+	da := &DoubleArray{Encoding: make(map[string]int)}
+	if len(seqs) == 0 {
+		return da
+	}
+
+	encoded := registerTokens(da, seqs)
+	sort.Sort(byLex(encoded))
+
+	root := node{row: -1, col: -1, left: 0, right: len(encoded)}
+	addSeqs(da, encoded, 0, root)
+
+	for i := len(da.Base); i > 0; i-- {
+		if da.Check[i-1] != 0 {
+			da.Base = da.Base[:i]
+			da.Check = da.Check[:i]
+			break
+		}
+	}
+	return da
+}
+
+func registerTokens(da *DoubleArray, seqs [][]string) [][]int {
+	var result [][]int
+	for _, seq := range seqs {
+		var encoded []int
+		for _, token := range seq {
+			if _, ok := da.Encoding[token]; !ok {
+				da.Encoding[token] = len(da.Encoding)
+			}
+			encoded = append(encoded, da.Encoding[token])
+		}
+		result = append(result, encoded)
+	}
+	for i := range result {
+		result[i] = append(result[i], len(da.Encoding))
+	}
+	return result
+}
+
+type node struct {
+	row, col    int
+	left, right int
+}
+
+func (n node) value(seqs [][]int) int {
+	return seqs[n.row][n.col]
+}
+
+func (n node) children(seqs [][]int) []*node {
+	var result []*node
+	lastVal := int(-1)
+	last := new(node)
+	for i := n.left; i < n.right; i++ {
+		if lastVal == seqs[i][n.col+1] {
+			continue
+		}
+		last.right = i
+		last = &node{
+			row:  i,
+			col:  n.col + 1,
+			left: i,
+		}
+		result = append(result, last)
+	}
+	last.right = n.right
+	return result
+}
+
+func addSeqs(da *DoubleArray, seqs [][]int, pos int, n node) {
+	ensureSize(da, pos)
+
+	children := n.children(seqs)
+	var i int
+	for i = 1; ; i++ {
+		ok := func() bool {
+			for _, child := range children {
+				code := child.value(seqs)
+				j := i + code
+				ensureSize(da, j)
+				if da.Check[j] != 0 {
+					return false
+				}
+			}
+			return true
+		}()
+		if ok {
+			break
+		}
+	}
+	da.Base[pos] = i
+	for _, child := range children {
+		code := child.value(seqs)
+		j := i + code
+		da.Check[j] = pos + 1
+	}
+	terminator := len(da.Encoding)
+	for _, child := range children {
+		code := child.value(seqs)
+		if code == terminator {
+			continue
+		}
+		j := i + code
+		addSeqs(da, seqs, j, *child)
+	}
+}
+
+func ensureSize(da *DoubleArray, i int) {
+	for i >= len(da.Base) {
+		da.Base = append(da.Base, make([]int, len(da.Base)+1)...)
+		da.Check = append(da.Check, make([]int, len(da.Check)+1)...)
+	}
+}
+
+type byLex [][]int
+
+func (l byLex) Len() int      { return len(l) }
+func (l byLex) Swap(i, j int) { l[i], l[j] = l[j], l[i] }
+func (l byLex) Less(i, j int) bool {
+	si := l[i]
+	sj := l[j]
+	var k int
+	for k = 0; k < len(si) && k < len(sj); k++ {
+		if si[k] < sj[k] {
+			return true
+		}
+		if si[k] > sj[k] {
+			return false
+		}
+	}
+	if k < len(sj) {
+		return true
+	}
+	return false
+}
+
+// HasCommonPrefix determines if any sequence in the DoubleArray is a prefix of the given sequence.
+func (da *DoubleArray) HasCommonPrefix(seq []string) bool {
+	if len(da.Base) == 0 {
+		return false
+	}
+
+	var i int
+	for _, t := range seq {
+		code, ok := da.Encoding[t]
+		if !ok {
+			break
+		}
+		j := da.Base[i] + code
+		if len(da.Check) <= j || da.Check[j] != i+1 {
+			break
+		}
+		i = j
+	}
+	j := da.Base[i] + len(da.Encoding)
+	if len(da.Check) <= j || da.Check[j] != i+1 {
+		return false
+	}
+	return true
+}
diff --git a/vendor/github.com/jonboulle/clockwork/.gitignore b/vendor/github.com/jonboulle/clockwork/.gitignore
new file mode 100644
index 0000000..010c242
--- /dev/null
+++ b/vendor/github.com/jonboulle/clockwork/.gitignore
@@ -0,0 +1,25 @@
+# Compiled Object files, Static and Dynamic libs (Shared Objects)
+*.o
+*.a
+*.so
+
+# Folders
+_obj
+_test
+
+# Architecture specific extensions/prefixes
+*.[568vq]
+[568vq].out
+
+*.cgo1.go
+*.cgo2.c
+_cgo_defun.c
+_cgo_gotypes.go
+_cgo_export.*
+
+_testmain.go
+
+*.exe
+*.test
+
+*.swp
diff --git a/vendor/github.com/jonboulle/clockwork/.travis.yml b/vendor/github.com/jonboulle/clockwork/.travis.yml
new file mode 100644
index 0000000..aefda90
--- /dev/null
+++ b/vendor/github.com/jonboulle/clockwork/.travis.yml
@@ -0,0 +1,5 @@
+language: go
+go:
+  - 1.3
+
+sudo: false
diff --git a/vendor/github.com/jonboulle/clockwork/LICENSE b/vendor/github.com/jonboulle/clockwork/LICENSE
new file mode 100644
index 0000000..5c304d1
--- /dev/null
+++ b/vendor/github.com/jonboulle/clockwork/LICENSE
@@ -0,0 +1,201 @@
+Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "{}"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright {yyyy} {name of copyright owner}
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
diff --git a/vendor/github.com/jonboulle/clockwork/README.md b/vendor/github.com/jonboulle/clockwork/README.md
new file mode 100644
index 0000000..d43a6c7
--- /dev/null
+++ b/vendor/github.com/jonboulle/clockwork/README.md
@@ -0,0 +1,61 @@
+clockwork
+=========
+
+[![Build Status](https://travis-ci.org/jonboulle/clockwork.png?branch=master)](https://travis-ci.org/jonboulle/clockwork)
+[![godoc](https://godoc.org/github.com/jonboulle/clockwork?status.svg)](http://godoc.org/github.com/jonboulle/clockwork) 
+
+a simple fake clock for golang
+
+# Usage
+
+Replace uses of the `time` package with the `clockwork.Clock` interface instead.
+
+For example, instead of using `time.Sleep` directly:
+
+```
+func my_func() {
+	time.Sleep(3 * time.Second)
+	do_something()
+}
+```
+
+inject a clock and use its `Sleep` method instead:
+
+```
+func my_func(clock clockwork.Clock) {
+	clock.Sleep(3 * time.Second)
+	do_something()
+}
+```
+
+Now you can easily test `my_func` with a `FakeClock`:
+
+```
+func TestMyFunc(t *testing.T) {
+	c := clockwork.NewFakeClock()
+
+	// Start our sleepy function
+	my_func(c)
+
+	// Ensure we wait until my_func is sleeping
+	c.BlockUntil(1)
+
+	assert_state()
+
+	// Advance the FakeClock forward in time
+	c.Advance(3)
+
+	assert_state()
+}
+```
+
+and in production builds, simply inject the real clock instead:
+```
+my_func(clockwork.NewRealClock())
+```
+
+See [example_test.go](example_test.go) for a full example.
+
+# Credits
+
+clockwork is inspired by @wickman's [threaded fake clock](https://gist.github.com/wickman/3840816), and the [Golang playground](http://blog.golang.org/playground#Faking time)
diff --git a/vendor/github.com/jonboulle/clockwork/clockwork.go b/vendor/github.com/jonboulle/clockwork/clockwork.go
new file mode 100644
index 0000000..9ec96ed
--- /dev/null
+++ b/vendor/github.com/jonboulle/clockwork/clockwork.go
@@ -0,0 +1,169 @@
+package clockwork
+
+import (
+	"sync"
+	"time"
+)
+
+// Clock provides an interface that packages can use instead of directly
+// using the time module, so that chronology-related behavior can be tested
+type Clock interface {
+	After(d time.Duration) <-chan time.Time
+	Sleep(d time.Duration)
+	Now() time.Time
+}
+
+// FakeClock provides an interface for a clock which can be
+// manually advanced through time
+type FakeClock interface {
+	Clock
+	// Advance advances the FakeClock to a new point in time, ensuring any existing
+	// sleepers are notified appropriately before returning
+	Advance(d time.Duration)
+	// BlockUntil will block until the FakeClock has the given number of
+	// sleepers (callers of Sleep or After)
+	BlockUntil(n int)
+}
+
+// NewRealClock returns a Clock which simply delegates calls to the actual time
+// package; it should be used by packages in production.
+func NewRealClock() Clock {
+	return &realClock{}
+}
+
+// NewFakeClock returns a FakeClock implementation which can be
+// manually advanced through time for testing. The initial time of the
+// FakeClock will be an arbitrary non-zero time.
+func NewFakeClock() FakeClock {
+	// use a fixture that does not fulfill Time.IsZero()
+	return NewFakeClockAt(time.Date(1984, time.April, 4, 0, 0, 0, 0, time.UTC))
+}
+
+// NewFakeClockAt returns a FakeClock initialised at the given time.Time.
+func NewFakeClockAt(t time.Time) FakeClock {
+	return &fakeClock{
+		time: t,
+	}
+}
+
+type realClock struct{}
+
+func (rc *realClock) After(d time.Duration) <-chan time.Time {
+	return time.After(d)
+}
+
+func (rc *realClock) Sleep(d time.Duration) {
+	time.Sleep(d)
+}
+
+func (rc *realClock) Now() time.Time {
+	return time.Now()
+}
+
+type fakeClock struct {
+	sleepers []*sleeper
+	blockers []*blocker
+	time     time.Time
+
+	l sync.RWMutex
+}
+
+// sleeper represents a caller of After or Sleep
+type sleeper struct {
+	until time.Time
+	done  chan time.Time
+}
+
+// blocker represents a caller of BlockUntil
+type blocker struct {
+	count int
+	ch    chan struct{}
+}
+
+// After mimics time.After; it waits for the given duration to elapse on the
+// fakeClock, then sends the current time on the returned channel.
+func (fc *fakeClock) After(d time.Duration) <-chan time.Time {
+	fc.l.Lock()
+	defer fc.l.Unlock()
+	now := fc.time
+	done := make(chan time.Time, 1)
+	if d.Nanoseconds() == 0 {
+		// special case - trigger immediately
+		done <- now
+	} else {
+		// otherwise, add to the set of sleepers
+		s := &sleeper{
+			until: now.Add(d),
+			done:  done,
+		}
+		fc.sleepers = append(fc.sleepers, s)
+		// and notify any blockers
+		fc.blockers = notifyBlockers(fc.blockers, len(fc.sleepers))
+	}
+	return done
+}
+
+// notifyBlockers notifies all the blockers waiting until the
+// given number of sleepers are waiting on the fakeClock. It
+// returns an updated slice of blockers (i.e. those still waiting)
+func notifyBlockers(blockers []*blocker, count int) (newBlockers []*blocker) {
+	for _, b := range blockers {
+		if b.count == count {
+			close(b.ch)
+		} else {
+			newBlockers = append(newBlockers, b)
+		}
+	}
+	return
+}
+
+// Sleep blocks until the given duration has passed on the fakeClock
+func (fc *fakeClock) Sleep(d time.Duration) {
+	<-fc.After(d)
+}
+
+// Time returns the current time of the fakeClock
+func (fc *fakeClock) Now() time.Time {
+	fc.l.RLock()
+	t := fc.time
+	fc.l.RUnlock()
+	return t
+}
+
+// Advance advances fakeClock to a new point in time, ensuring channels from any
+// previous invocations of After are notified appropriately before returning
+func (fc *fakeClock) Advance(d time.Duration) {
+	fc.l.Lock()
+	defer fc.l.Unlock()
+	end := fc.time.Add(d)
+	var newSleepers []*sleeper
+	for _, s := range fc.sleepers {
+		if end.Sub(s.until) >= 0 {
+			s.done <- end
+		} else {
+			newSleepers = append(newSleepers, s)
+		}
+	}
+	fc.sleepers = newSleepers
+	fc.blockers = notifyBlockers(fc.blockers, len(fc.sleepers))
+	fc.time = end
+}
+
+// BlockUntil will block until the fakeClock has the given number of sleepers
+// (callers of Sleep or After)
+func (fc *fakeClock) BlockUntil(n int) {
+	fc.l.Lock()
+	// Fast path: current number of sleepers is what we're looking for
+	if len(fc.sleepers) == n {
+		fc.l.Unlock()
+		return
+	}
+	// Otherwise, set up a new blocker
+	b := &blocker{
+		count: n,
+		ch:    make(chan struct{}),
+	}
+	fc.blockers = append(fc.blockers, b)
+	fc.l.Unlock()
+	<-b.ch
+}
diff --git a/vendor/github.com/json-iterator/go/.codecov.yml b/vendor/github.com/json-iterator/go/.codecov.yml
new file mode 100644
index 0000000..955dc0b
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/.codecov.yml
@@ -0,0 +1,3 @@
+ignore:
+    - "output_tests/.*"
+
diff --git a/vendor/github.com/json-iterator/go/.gitignore b/vendor/github.com/json-iterator/go/.gitignore
new file mode 100644
index 0000000..1555653
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/.gitignore
@@ -0,0 +1,4 @@
+/vendor
+/bug_test.go
+/coverage.txt
+/.idea
diff --git a/vendor/github.com/json-iterator/go/.travis.yml b/vendor/github.com/json-iterator/go/.travis.yml
new file mode 100644
index 0000000..449e67c
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/.travis.yml
@@ -0,0 +1,14 @@
+language: go
+
+go:
+  - 1.8.x
+  - 1.x
+
+before_install:
+  - go get -t -v ./...
+
+script:
+  - ./test.sh
+
+after_success:
+  - bash <(curl -s https://codecov.io/bash)
diff --git a/vendor/github.com/json-iterator/go/Gopkg.lock b/vendor/github.com/json-iterator/go/Gopkg.lock
new file mode 100644
index 0000000..c8a9fbb
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/Gopkg.lock
@@ -0,0 +1,21 @@
+# This file is autogenerated, do not edit; changes may be undone by the next 'dep ensure'.
+
+
+[[projects]]
+  name = "github.com/modern-go/concurrent"
+  packages = ["."]
+  revision = "e0a39a4cb4216ea8db28e22a69f4ec25610d513a"
+  version = "1.0.0"
+
+[[projects]]
+  name = "github.com/modern-go/reflect2"
+  packages = ["."]
+  revision = "4b7aa43c6742a2c18fdef89dd197aaae7dac7ccd"
+  version = "1.0.1"
+
+[solve-meta]
+  analyzer-name = "dep"
+  analyzer-version = 1
+  inputs-digest = "ea54a775e5a354cb015502d2e7aa4b74230fc77e894f34a838b268c25ec8eeb8"
+  solver-name = "gps-cdcl"
+  solver-version = 1
diff --git a/vendor/github.com/json-iterator/go/Gopkg.toml b/vendor/github.com/json-iterator/go/Gopkg.toml
new file mode 100644
index 0000000..313a0f8
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/Gopkg.toml
@@ -0,0 +1,26 @@
+# Gopkg.toml example
+#
+# Refer to https://github.com/golang/dep/blob/master/docs/Gopkg.toml.md
+# for detailed Gopkg.toml documentation.
+#
+# required = ["github.com/user/thing/cmd/thing"]
+# ignored = ["github.com/user/project/pkgX", "bitbucket.org/user/project/pkgA/pkgY"]
+#
+# [[constraint]]
+#   name = "github.com/user/project"
+#   version = "1.0.0"
+#
+# [[constraint]]
+#   name = "github.com/user/project2"
+#   branch = "dev"
+#   source = "github.com/myfork/project2"
+#
+# [[override]]
+#  name = "github.com/x/y"
+#  version = "2.4.0"
+
+ignored = ["github.com/davecgh/go-spew*","github.com/google/gofuzz*","github.com/stretchr/testify*"]
+
+[[constraint]]
+  name = "github.com/modern-go/reflect2"
+  version = "1.0.1"
diff --git a/vendor/github.com/json-iterator/go/LICENSE b/vendor/github.com/json-iterator/go/LICENSE
new file mode 100644
index 0000000..2cf4f5a
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/LICENSE
@@ -0,0 +1,21 @@
+MIT License
+
+Copyright (c) 2016 json-iterator
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
diff --git a/vendor/github.com/json-iterator/go/README.md b/vendor/github.com/json-iterator/go/README.md
new file mode 100644
index 0000000..50d56ff
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/README.md
@@ -0,0 +1,87 @@
+[![Sourcegraph](https://sourcegraph.com/github.com/json-iterator/go/-/badge.svg)](https://sourcegraph.com/github.com/json-iterator/go?badge)
+[![GoDoc](http://img.shields.io/badge/go-documentation-blue.svg?style=flat-square)](http://godoc.org/github.com/json-iterator/go)
+[![Build Status](https://travis-ci.org/json-iterator/go.svg?branch=master)](https://travis-ci.org/json-iterator/go)
+[![codecov](https://codecov.io/gh/json-iterator/go/branch/master/graph/badge.svg)](https://codecov.io/gh/json-iterator/go)
+[![rcard](https://goreportcard.com/badge/github.com/json-iterator/go)](https://goreportcard.com/report/github.com/json-iterator/go)
+[![License](http://img.shields.io/badge/license-mit-blue.svg?style=flat-square)](https://raw.githubusercontent.com/json-iterator/go/master/LICENSE)
+[![Gitter chat](https://badges.gitter.im/gitterHQ/gitter.png)](https://gitter.im/json-iterator/Lobby)
+
+A high-performance 100% compatible drop-in replacement of "encoding/json"
+
+You can also use thrift like JSON using [thrift-iterator](https://github.com/thrift-iterator/go)
+
+# Benchmark
+
+![benchmark](http://jsoniter.com/benchmarks/go-benchmark.png)
+
+Source code: https://github.com/json-iterator/go-benchmark/blob/master/src/github.com/json-iterator/go-benchmark/benchmark_medium_payload_test.go
+
+Raw Result (easyjson requires static code generation)
+
+| | ns/op | allocation bytes | allocation times |
+| --- | --- | --- | --- |
+| std decode | 35510 ns/op | 1960 B/op | 99 allocs/op |
+| easyjson decode | 8499 ns/op | 160 B/op | 4 allocs/op |
+| jsoniter decode | 5623 ns/op | 160 B/op | 3 allocs/op |
+| std encode | 2213 ns/op | 712 B/op | 5 allocs/op |
+| easyjson encode | 883 ns/op | 576 B/op | 3 allocs/op |
+| jsoniter encode | 837 ns/op | 384 B/op | 4 allocs/op |
+
+Always benchmark with your own workload. 
+The result depends heavily on the data input.
+
+# Usage
+
+100% compatibility with standard lib
+
+Replace
+
+```go
+import "encoding/json"
+json.Marshal(&data)
+```
+
+with 
+
+```go
+import "github.com/json-iterator/go"
+
+var json = jsoniter.ConfigCompatibleWithStandardLibrary
+json.Marshal(&data)
+```
+
+Replace
+
+```go
+import "encoding/json"
+json.Unmarshal(input, &data)
+```
+
+with
+
+```go
+import "github.com/json-iterator/go"
+
+var json = jsoniter.ConfigCompatibleWithStandardLibrary
+json.Unmarshal(input, &data)
+```
+
+[More documentation](http://jsoniter.com/migrate-from-go-std.html)
+
+# How to get
+
+```
+go get github.com/json-iterator/go
+```
+
+# Contribution Welcomed !
+
+Contributors
+
+* [thockin](https://github.com/thockin) 
+* [mattn](https://github.com/mattn)
+* [cch123](https://github.com/cch123)
+* [Oleg Shaldybin](https://github.com/olegshaldybin)
+* [Jason Toffaletti](https://github.com/toffaletti)
+
+Report issue or pull request, or email taowen@gmail.com, or [![Gitter chat](https://badges.gitter.im/gitterHQ/gitter.png)](https://gitter.im/json-iterator/Lobby)
diff --git a/vendor/github.com/json-iterator/go/adapter.go b/vendor/github.com/json-iterator/go/adapter.go
new file mode 100644
index 0000000..92d2cc4
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/adapter.go
@@ -0,0 +1,150 @@
+package jsoniter
+
+import (
+	"bytes"
+	"io"
+)
+
+// RawMessage to make replace json with jsoniter
+type RawMessage []byte
+
+// Unmarshal adapts to json/encoding Unmarshal API
+//
+// Unmarshal parses the JSON-encoded data and stores the result in the value pointed to by v.
+// Refer to https://godoc.org/encoding/json#Unmarshal for more information
+func Unmarshal(data []byte, v interface{}) error {
+	return ConfigDefault.Unmarshal(data, v)
+}
+
+// UnmarshalFromString is a convenient method to read from string instead of []byte
+func UnmarshalFromString(str string, v interface{}) error {
+	return ConfigDefault.UnmarshalFromString(str, v)
+}
+
+// Get quick method to get value from deeply nested JSON structure
+func Get(data []byte, path ...interface{}) Any {
+	return ConfigDefault.Get(data, path...)
+}
+
+// Marshal adapts to json/encoding Marshal API
+//
+// Marshal returns the JSON encoding of v, adapts to json/encoding Marshal API
+// Refer to https://godoc.org/encoding/json#Marshal for more information
+func Marshal(v interface{}) ([]byte, error) {
+	return ConfigDefault.Marshal(v)
+}
+
+// MarshalIndent same as json.MarshalIndent. Prefix is not supported.
+func MarshalIndent(v interface{}, prefix, indent string) ([]byte, error) {
+	return ConfigDefault.MarshalIndent(v, prefix, indent)
+}
+
+// MarshalToString convenient method to write as string instead of []byte
+func MarshalToString(v interface{}) (string, error) {
+	return ConfigDefault.MarshalToString(v)
+}
+
+// NewDecoder adapts to json/stream NewDecoder API.
+//
+// NewDecoder returns a new decoder that reads from r.
+//
+// Instead of a json/encoding Decoder, an Decoder is returned
+// Refer to https://godoc.org/encoding/json#NewDecoder for more information
+func NewDecoder(reader io.Reader) *Decoder {
+	return ConfigDefault.NewDecoder(reader)
+}
+
+// Decoder reads and decodes JSON values from an input stream.
+// Decoder provides identical APIs with json/stream Decoder (Token() and UseNumber() are in progress)
+type Decoder struct {
+	iter *Iterator
+}
+
+// Decode decode JSON into interface{}
+func (adapter *Decoder) Decode(obj interface{}) error {
+	if adapter.iter.head == adapter.iter.tail && adapter.iter.reader != nil {
+		if !adapter.iter.loadMore() {
+			return io.EOF
+		}
+	}
+	adapter.iter.ReadVal(obj)
+	err := adapter.iter.Error
+	if err == io.EOF {
+		return nil
+	}
+	return adapter.iter.Error
+}
+
+// More is there more?
+func (adapter *Decoder) More() bool {
+	iter := adapter.iter
+	if iter.Error != nil {
+		return false
+	}
+	c := iter.nextToken()
+	if c == 0 {
+		return false
+	}
+	iter.unreadByte()
+	return c != ']' && c != '}'
+}
+
+// Buffered remaining buffer
+func (adapter *Decoder) Buffered() io.Reader {
+	remaining := adapter.iter.buf[adapter.iter.head:adapter.iter.tail]
+	return bytes.NewReader(remaining)
+}
+
+// UseNumber causes the Decoder to unmarshal a number into an interface{} as a
+// Number instead of as a float64.
+func (adapter *Decoder) UseNumber() {
+	cfg := adapter.iter.cfg.configBeforeFrozen
+	cfg.UseNumber = true
+	adapter.iter.cfg = cfg.frozeWithCacheReuse(adapter.iter.cfg.extraExtensions)
+}
+
+// DisallowUnknownFields causes the Decoder to return an error when the destination
+// is a struct and the input contains object keys which do not match any
+// non-ignored, exported fields in the destination.
+func (adapter *Decoder) DisallowUnknownFields() {
+	cfg := adapter.iter.cfg.configBeforeFrozen
+	cfg.DisallowUnknownFields = true
+	adapter.iter.cfg = cfg.frozeWithCacheReuse(adapter.iter.cfg.extraExtensions)
+}
+
+// NewEncoder same as json.NewEncoder
+func NewEncoder(writer io.Writer) *Encoder {
+	return ConfigDefault.NewEncoder(writer)
+}
+
+// Encoder same as json.Encoder
+type Encoder struct {
+	stream *Stream
+}
+
+// Encode encode interface{} as JSON to io.Writer
+func (adapter *Encoder) Encode(val interface{}) error {
+	adapter.stream.WriteVal(val)
+	adapter.stream.WriteRaw("\n")
+	adapter.stream.Flush()
+	return adapter.stream.Error
+}
+
+// SetIndent set the indention. Prefix is not supported
+func (adapter *Encoder) SetIndent(prefix, indent string) {
+	config := adapter.stream.cfg.configBeforeFrozen
+	config.IndentionStep = len(indent)
+	adapter.stream.cfg = config.frozeWithCacheReuse(adapter.stream.cfg.extraExtensions)
+}
+
+// SetEscapeHTML escape html by default, set to false to disable
+func (adapter *Encoder) SetEscapeHTML(escapeHTML bool) {
+	config := adapter.stream.cfg.configBeforeFrozen
+	config.EscapeHTML = escapeHTML
+	adapter.stream.cfg = config.frozeWithCacheReuse(adapter.stream.cfg.extraExtensions)
+}
+
+// Valid reports whether data is a valid JSON encoding.
+func Valid(data []byte) bool {
+	return ConfigDefault.Valid(data)
+}
diff --git a/vendor/github.com/json-iterator/go/any.go b/vendor/github.com/json-iterator/go/any.go
new file mode 100644
index 0000000..f6b8aea
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/any.go
@@ -0,0 +1,325 @@
+package jsoniter
+
+import (
+	"errors"
+	"fmt"
+	"github.com/modern-go/reflect2"
+	"io"
+	"reflect"
+	"strconv"
+	"unsafe"
+)
+
+// Any generic object representation.
+// The lazy json implementation holds []byte and parse lazily.
+type Any interface {
+	LastError() error
+	ValueType() ValueType
+	MustBeValid() Any
+	ToBool() bool
+	ToInt() int
+	ToInt32() int32
+	ToInt64() int64
+	ToUint() uint
+	ToUint32() uint32
+	ToUint64() uint64
+	ToFloat32() float32
+	ToFloat64() float64
+	ToString() string
+	ToVal(val interface{})
+	Get(path ...interface{}) Any
+	Size() int
+	Keys() []string
+	GetInterface() interface{}
+	WriteTo(stream *Stream)
+}
+
+type baseAny struct{}
+
+func (any *baseAny) Get(path ...interface{}) Any {
+	return &invalidAny{baseAny{}, fmt.Errorf("GetIndex %v from simple value", path)}
+}
+
+func (any *baseAny) Size() int {
+	return 0
+}
+
+func (any *baseAny) Keys() []string {
+	return []string{}
+}
+
+func (any *baseAny) ToVal(obj interface{}) {
+	panic("not implemented")
+}
+
+// WrapInt32 turn int32 into Any interface
+func WrapInt32(val int32) Any {
+	return &int32Any{baseAny{}, val}
+}
+
+// WrapInt64 turn int64 into Any interface
+func WrapInt64(val int64) Any {
+	return &int64Any{baseAny{}, val}
+}
+
+// WrapUint32 turn uint32 into Any interface
+func WrapUint32(val uint32) Any {
+	return &uint32Any{baseAny{}, val}
+}
+
+// WrapUint64 turn uint64 into Any interface
+func WrapUint64(val uint64) Any {
+	return &uint64Any{baseAny{}, val}
+}
+
+// WrapFloat64 turn float64 into Any interface
+func WrapFloat64(val float64) Any {
+	return &floatAny{baseAny{}, val}
+}
+
+// WrapString turn string into Any interface
+func WrapString(val string) Any {
+	return &stringAny{baseAny{}, val}
+}
+
+// Wrap turn a go object into Any interface
+func Wrap(val interface{}) Any {
+	if val == nil {
+		return &nilAny{}
+	}
+	asAny, isAny := val.(Any)
+	if isAny {
+		return asAny
+	}
+	typ := reflect2.TypeOf(val)
+	switch typ.Kind() {
+	case reflect.Slice:
+		return wrapArray(val)
+	case reflect.Struct:
+		return wrapStruct(val)
+	case reflect.Map:
+		return wrapMap(val)
+	case reflect.String:
+		return WrapString(val.(string))
+	case reflect.Int:
+		if strconv.IntSize == 32 {
+			return WrapInt32(int32(val.(int)))
+		}
+		return WrapInt64(int64(val.(int)))
+	case reflect.Int8:
+		return WrapInt32(int32(val.(int8)))
+	case reflect.Int16:
+		return WrapInt32(int32(val.(int16)))
+	case reflect.Int32:
+		return WrapInt32(val.(int32))
+	case reflect.Int64:
+		return WrapInt64(val.(int64))
+	case reflect.Uint:
+		if strconv.IntSize == 32 {
+			return WrapUint32(uint32(val.(uint)))
+		}
+		return WrapUint64(uint64(val.(uint)))
+	case reflect.Uintptr:
+		if ptrSize == 32 {
+			return WrapUint32(uint32(val.(uintptr)))
+		}
+		return WrapUint64(uint64(val.(uintptr)))
+	case reflect.Uint8:
+		return WrapUint32(uint32(val.(uint8)))
+	case reflect.Uint16:
+		return WrapUint32(uint32(val.(uint16)))
+	case reflect.Uint32:
+		return WrapUint32(uint32(val.(uint32)))
+	case reflect.Uint64:
+		return WrapUint64(val.(uint64))
+	case reflect.Float32:
+		return WrapFloat64(float64(val.(float32)))
+	case reflect.Float64:
+		return WrapFloat64(val.(float64))
+	case reflect.Bool:
+		if val.(bool) == true {
+			return &trueAny{}
+		}
+		return &falseAny{}
+	}
+	return &invalidAny{baseAny{}, fmt.Errorf("unsupported type: %v", typ)}
+}
+
+// ReadAny read next JSON element as an Any object. It is a better json.RawMessage.
+func (iter *Iterator) ReadAny() Any {
+	return iter.readAny()
+}
+
+func (iter *Iterator) readAny() Any {
+	c := iter.nextToken()
+	switch c {
+	case '"':
+		iter.unreadByte()
+		return &stringAny{baseAny{}, iter.ReadString()}
+	case 'n':
+		iter.skipThreeBytes('u', 'l', 'l') // null
+		return &nilAny{}
+	case 't':
+		iter.skipThreeBytes('r', 'u', 'e') // true
+		return &trueAny{}
+	case 'f':
+		iter.skipFourBytes('a', 'l', 's', 'e') // false
+		return &falseAny{}
+	case '{':
+		return iter.readObjectAny()
+	case '[':
+		return iter.readArrayAny()
+	case '-':
+		return iter.readNumberAny(false)
+	case 0:
+		return &invalidAny{baseAny{}, errors.New("input is empty")}
+	default:
+		return iter.readNumberAny(true)
+	}
+}
+
+func (iter *Iterator) readNumberAny(positive bool) Any {
+	iter.startCapture(iter.head - 1)
+	iter.skipNumber()
+	lazyBuf := iter.stopCapture()
+	return &numberLazyAny{baseAny{}, iter.cfg, lazyBuf, nil}
+}
+
+func (iter *Iterator) readObjectAny() Any {
+	iter.startCapture(iter.head - 1)
+	iter.skipObject()
+	lazyBuf := iter.stopCapture()
+	return &objectLazyAny{baseAny{}, iter.cfg, lazyBuf, nil}
+}
+
+func (iter *Iterator) readArrayAny() Any {
+	iter.startCapture(iter.head - 1)
+	iter.skipArray()
+	lazyBuf := iter.stopCapture()
+	return &arrayLazyAny{baseAny{}, iter.cfg, lazyBuf, nil}
+}
+
+func locateObjectField(iter *Iterator, target string) []byte {
+	var found []byte
+	iter.ReadObjectCB(func(iter *Iterator, field string) bool {
+		if field == target {
+			found = iter.SkipAndReturnBytes()
+			return false
+		}
+		iter.Skip()
+		return true
+	})
+	return found
+}
+
+func locateArrayElement(iter *Iterator, target int) []byte {
+	var found []byte
+	n := 0
+	iter.ReadArrayCB(func(iter *Iterator) bool {
+		if n == target {
+			found = iter.SkipAndReturnBytes()
+			return false
+		}
+		iter.Skip()
+		n++
+		return true
+	})
+	return found
+}
+
+func locatePath(iter *Iterator, path []interface{}) Any {
+	for i, pathKeyObj := range path {
+		switch pathKey := pathKeyObj.(type) {
+		case string:
+			valueBytes := locateObjectField(iter, pathKey)
+			if valueBytes == nil {
+				return newInvalidAny(path[i:])
+			}
+			iter.ResetBytes(valueBytes)
+		case int:
+			valueBytes := locateArrayElement(iter, pathKey)
+			if valueBytes == nil {
+				return newInvalidAny(path[i:])
+			}
+			iter.ResetBytes(valueBytes)
+		case int32:
+			if '*' == pathKey {
+				return iter.readAny().Get(path[i:]...)
+			}
+			return newInvalidAny(path[i:])
+		default:
+			return newInvalidAny(path[i:])
+		}
+	}
+	if iter.Error != nil && iter.Error != io.EOF {
+		return &invalidAny{baseAny{}, iter.Error}
+	}
+	return iter.readAny()
+}
+
+var anyType = reflect2.TypeOfPtr((*Any)(nil)).Elem()
+
+func createDecoderOfAny(ctx *ctx, typ reflect2.Type) ValDecoder {
+	if typ == anyType {
+		return &directAnyCodec{}
+	}
+	if typ.Implements(anyType) {
+		return &anyCodec{
+			valType: typ,
+		}
+	}
+	return nil
+}
+
+func createEncoderOfAny(ctx *ctx, typ reflect2.Type) ValEncoder {
+	if typ == anyType {
+		return &directAnyCodec{}
+	}
+	if typ.Implements(anyType) {
+		return &anyCodec{
+			valType: typ,
+		}
+	}
+	return nil
+}
+
+type anyCodec struct {
+	valType reflect2.Type
+}
+
+func (codec *anyCodec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	panic("not implemented")
+}
+
+func (codec *anyCodec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	obj := codec.valType.UnsafeIndirect(ptr)
+	any := obj.(Any)
+	any.WriteTo(stream)
+}
+
+func (codec *anyCodec) IsEmpty(ptr unsafe.Pointer) bool {
+	obj := codec.valType.UnsafeIndirect(ptr)
+	any := obj.(Any)
+	return any.Size() == 0
+}
+
+type directAnyCodec struct {
+}
+
+func (codec *directAnyCodec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	*(*Any)(ptr) = iter.readAny()
+}
+
+func (codec *directAnyCodec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	any := *(*Any)(ptr)
+	if any == nil {
+		stream.WriteNil()
+		return
+	}
+	any.WriteTo(stream)
+}
+
+func (codec *directAnyCodec) IsEmpty(ptr unsafe.Pointer) bool {
+	any := *(*Any)(ptr)
+	return any.Size() == 0
+}
diff --git a/vendor/github.com/json-iterator/go/any_array.go b/vendor/github.com/json-iterator/go/any_array.go
new file mode 100644
index 0000000..0449e9a
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/any_array.go
@@ -0,0 +1,278 @@
+package jsoniter
+
+import (
+	"reflect"
+	"unsafe"
+)
+
+type arrayLazyAny struct {
+	baseAny
+	cfg *frozenConfig
+	buf []byte
+	err error
+}
+
+func (any *arrayLazyAny) ValueType() ValueType {
+	return ArrayValue
+}
+
+func (any *arrayLazyAny) MustBeValid() Any {
+	return any
+}
+
+func (any *arrayLazyAny) LastError() error {
+	return any.err
+}
+
+func (any *arrayLazyAny) ToBool() bool {
+	iter := any.cfg.BorrowIterator(any.buf)
+	defer any.cfg.ReturnIterator(iter)
+	return iter.ReadArray()
+}
+
+func (any *arrayLazyAny) ToInt() int {
+	if any.ToBool() {
+		return 1
+	}
+	return 0
+}
+
+func (any *arrayLazyAny) ToInt32() int32 {
+	if any.ToBool() {
+		return 1
+	}
+	return 0
+}
+
+func (any *arrayLazyAny) ToInt64() int64 {
+	if any.ToBool() {
+		return 1
+	}
+	return 0
+}
+
+func (any *arrayLazyAny) ToUint() uint {
+	if any.ToBool() {
+		return 1
+	}
+	return 0
+}
+
+func (any *arrayLazyAny) ToUint32() uint32 {
+	if any.ToBool() {
+		return 1
+	}
+	return 0
+}
+
+func (any *arrayLazyAny) ToUint64() uint64 {
+	if any.ToBool() {
+		return 1
+	}
+	return 0
+}
+
+func (any *arrayLazyAny) ToFloat32() float32 {
+	if any.ToBool() {
+		return 1
+	}
+	return 0
+}
+
+func (any *arrayLazyAny) ToFloat64() float64 {
+	if any.ToBool() {
+		return 1
+	}
+	return 0
+}
+
+func (any *arrayLazyAny) ToString() string {
+	return *(*string)(unsafe.Pointer(&any.buf))
+}
+
+func (any *arrayLazyAny) ToVal(val interface{}) {
+	iter := any.cfg.BorrowIterator(any.buf)
+	defer any.cfg.ReturnIterator(iter)
+	iter.ReadVal(val)
+}
+
+func (any *arrayLazyAny) Get(path ...interface{}) Any {
+	if len(path) == 0 {
+		return any
+	}
+	switch firstPath := path[0].(type) {
+	case int:
+		iter := any.cfg.BorrowIterator(any.buf)
+		defer any.cfg.ReturnIterator(iter)
+		valueBytes := locateArrayElement(iter, firstPath)
+		if valueBytes == nil {
+			return newInvalidAny(path)
+		}
+		iter.ResetBytes(valueBytes)
+		return locatePath(iter, path[1:])
+	case int32:
+		if '*' == firstPath {
+			iter := any.cfg.BorrowIterator(any.buf)
+			defer any.cfg.ReturnIterator(iter)
+			arr := make([]Any, 0)
+			iter.ReadArrayCB(func(iter *Iterator) bool {
+				found := iter.readAny().Get(path[1:]...)
+				if found.ValueType() != InvalidValue {
+					arr = append(arr, found)
+				}
+				return true
+			})
+			return wrapArray(arr)
+		}
+		return newInvalidAny(path)
+	default:
+		return newInvalidAny(path)
+	}
+}
+
+func (any *arrayLazyAny) Size() int {
+	size := 0
+	iter := any.cfg.BorrowIterator(any.buf)
+	defer any.cfg.ReturnIterator(iter)
+	iter.ReadArrayCB(func(iter *Iterator) bool {
+		size++
+		iter.Skip()
+		return true
+	})
+	return size
+}
+
+func (any *arrayLazyAny) WriteTo(stream *Stream) {
+	stream.Write(any.buf)
+}
+
+func (any *arrayLazyAny) GetInterface() interface{} {
+	iter := any.cfg.BorrowIterator(any.buf)
+	defer any.cfg.ReturnIterator(iter)
+	return iter.Read()
+}
+
+type arrayAny struct {
+	baseAny
+	val reflect.Value
+}
+
+func wrapArray(val interface{}) *arrayAny {
+	return &arrayAny{baseAny{}, reflect.ValueOf(val)}
+}
+
+func (any *arrayAny) ValueType() ValueType {
+	return ArrayValue
+}
+
+func (any *arrayAny) MustBeValid() Any {
+	return any
+}
+
+func (any *arrayAny) LastError() error {
+	return nil
+}
+
+func (any *arrayAny) ToBool() bool {
+	return any.val.Len() != 0
+}
+
+func (any *arrayAny) ToInt() int {
+	if any.val.Len() == 0 {
+		return 0
+	}
+	return 1
+}
+
+func (any *arrayAny) ToInt32() int32 {
+	if any.val.Len() == 0 {
+		return 0
+	}
+	return 1
+}
+
+func (any *arrayAny) ToInt64() int64 {
+	if any.val.Len() == 0 {
+		return 0
+	}
+	return 1
+}
+
+func (any *arrayAny) ToUint() uint {
+	if any.val.Len() == 0 {
+		return 0
+	}
+	return 1
+}
+
+func (any *arrayAny) ToUint32() uint32 {
+	if any.val.Len() == 0 {
+		return 0
+	}
+	return 1
+}
+
+func (any *arrayAny) ToUint64() uint64 {
+	if any.val.Len() == 0 {
+		return 0
+	}
+	return 1
+}
+
+func (any *arrayAny) ToFloat32() float32 {
+	if any.val.Len() == 0 {
+		return 0
+	}
+	return 1
+}
+
+func (any *arrayAny) ToFloat64() float64 {
+	if any.val.Len() == 0 {
+		return 0
+	}
+	return 1
+}
+
+func (any *arrayAny) ToString() string {
+	str, _ := MarshalToString(any.val.Interface())
+	return str
+}
+
+func (any *arrayAny) Get(path ...interface{}) Any {
+	if len(path) == 0 {
+		return any
+	}
+	switch firstPath := path[0].(type) {
+	case int:
+		if firstPath < 0 || firstPath >= any.val.Len() {
+			return newInvalidAny(path)
+		}
+		return Wrap(any.val.Index(firstPath).Interface())
+	case int32:
+		if '*' == firstPath {
+			mappedAll := make([]Any, 0)
+			for i := 0; i < any.val.Len(); i++ {
+				mapped := Wrap(any.val.Index(i).Interface()).Get(path[1:]...)
+				if mapped.ValueType() != InvalidValue {
+					mappedAll = append(mappedAll, mapped)
+				}
+			}
+			return wrapArray(mappedAll)
+		}
+		return newInvalidAny(path)
+	default:
+		return newInvalidAny(path)
+	}
+}
+
+func (any *arrayAny) Size() int {
+	return any.val.Len()
+}
+
+func (any *arrayAny) WriteTo(stream *Stream) {
+	stream.WriteVal(any.val)
+}
+
+func (any *arrayAny) GetInterface() interface{} {
+	return any.val.Interface()
+}
diff --git a/vendor/github.com/json-iterator/go/any_bool.go b/vendor/github.com/json-iterator/go/any_bool.go
new file mode 100644
index 0000000..9452324
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/any_bool.go
@@ -0,0 +1,137 @@
+package jsoniter
+
+type trueAny struct {
+	baseAny
+}
+
+func (any *trueAny) LastError() error {
+	return nil
+}
+
+func (any *trueAny) ToBool() bool {
+	return true
+}
+
+func (any *trueAny) ToInt() int {
+	return 1
+}
+
+func (any *trueAny) ToInt32() int32 {
+	return 1
+}
+
+func (any *trueAny) ToInt64() int64 {
+	return 1
+}
+
+func (any *trueAny) ToUint() uint {
+	return 1
+}
+
+func (any *trueAny) ToUint32() uint32 {
+	return 1
+}
+
+func (any *trueAny) ToUint64() uint64 {
+	return 1
+}
+
+func (any *trueAny) ToFloat32() float32 {
+	return 1
+}
+
+func (any *trueAny) ToFloat64() float64 {
+	return 1
+}
+
+func (any *trueAny) ToString() string {
+	return "true"
+}
+
+func (any *trueAny) WriteTo(stream *Stream) {
+	stream.WriteTrue()
+}
+
+func (any *trueAny) Parse() *Iterator {
+	return nil
+}
+
+func (any *trueAny) GetInterface() interface{} {
+	return true
+}
+
+func (any *trueAny) ValueType() ValueType {
+	return BoolValue
+}
+
+func (any *trueAny) MustBeValid() Any {
+	return any
+}
+
+type falseAny struct {
+	baseAny
+}
+
+func (any *falseAny) LastError() error {
+	return nil
+}
+
+func (any *falseAny) ToBool() bool {
+	return false
+}
+
+func (any *falseAny) ToInt() int {
+	return 0
+}
+
+func (any *falseAny) ToInt32() int32 {
+	return 0
+}
+
+func (any *falseAny) ToInt64() int64 {
+	return 0
+}
+
+func (any *falseAny) ToUint() uint {
+	return 0
+}
+
+func (any *falseAny) ToUint32() uint32 {
+	return 0
+}
+
+func (any *falseAny) ToUint64() uint64 {
+	return 0
+}
+
+func (any *falseAny) ToFloat32() float32 {
+	return 0
+}
+
+func (any *falseAny) ToFloat64() float64 {
+	return 0
+}
+
+func (any *falseAny) ToString() string {
+	return "false"
+}
+
+func (any *falseAny) WriteTo(stream *Stream) {
+	stream.WriteFalse()
+}
+
+func (any *falseAny) Parse() *Iterator {
+	return nil
+}
+
+func (any *falseAny) GetInterface() interface{} {
+	return false
+}
+
+func (any *falseAny) ValueType() ValueType {
+	return BoolValue
+}
+
+func (any *falseAny) MustBeValid() Any {
+	return any
+}
diff --git a/vendor/github.com/json-iterator/go/any_float.go b/vendor/github.com/json-iterator/go/any_float.go
new file mode 100644
index 0000000..35fdb09
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/any_float.go
@@ -0,0 +1,83 @@
+package jsoniter
+
+import (
+	"strconv"
+)
+
+type floatAny struct {
+	baseAny
+	val float64
+}
+
+func (any *floatAny) Parse() *Iterator {
+	return nil
+}
+
+func (any *floatAny) ValueType() ValueType {
+	return NumberValue
+}
+
+func (any *floatAny) MustBeValid() Any {
+	return any
+}
+
+func (any *floatAny) LastError() error {
+	return nil
+}
+
+func (any *floatAny) ToBool() bool {
+	return any.ToFloat64() != 0
+}
+
+func (any *floatAny) ToInt() int {
+	return int(any.val)
+}
+
+func (any *floatAny) ToInt32() int32 {
+	return int32(any.val)
+}
+
+func (any *floatAny) ToInt64() int64 {
+	return int64(any.val)
+}
+
+func (any *floatAny) ToUint() uint {
+	if any.val > 0 {
+		return uint(any.val)
+	}
+	return 0
+}
+
+func (any *floatAny) ToUint32() uint32 {
+	if any.val > 0 {
+		return uint32(any.val)
+	}
+	return 0
+}
+
+func (any *floatAny) ToUint64() uint64 {
+	if any.val > 0 {
+		return uint64(any.val)
+	}
+	return 0
+}
+
+func (any *floatAny) ToFloat32() float32 {
+	return float32(any.val)
+}
+
+func (any *floatAny) ToFloat64() float64 {
+	return any.val
+}
+
+func (any *floatAny) ToString() string {
+	return strconv.FormatFloat(any.val, 'E', -1, 64)
+}
+
+func (any *floatAny) WriteTo(stream *Stream) {
+	stream.WriteFloat64(any.val)
+}
+
+func (any *floatAny) GetInterface() interface{} {
+	return any.val
+}
diff --git a/vendor/github.com/json-iterator/go/any_int32.go b/vendor/github.com/json-iterator/go/any_int32.go
new file mode 100644
index 0000000..1b56f39
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/any_int32.go
@@ -0,0 +1,74 @@
+package jsoniter
+
+import (
+	"strconv"
+)
+
+type int32Any struct {
+	baseAny
+	val int32
+}
+
+func (any *int32Any) LastError() error {
+	return nil
+}
+
+func (any *int32Any) ValueType() ValueType {
+	return NumberValue
+}
+
+func (any *int32Any) MustBeValid() Any {
+	return any
+}
+
+func (any *int32Any) ToBool() bool {
+	return any.val != 0
+}
+
+func (any *int32Any) ToInt() int {
+	return int(any.val)
+}
+
+func (any *int32Any) ToInt32() int32 {
+	return any.val
+}
+
+func (any *int32Any) ToInt64() int64 {
+	return int64(any.val)
+}
+
+func (any *int32Any) ToUint() uint {
+	return uint(any.val)
+}
+
+func (any *int32Any) ToUint32() uint32 {
+	return uint32(any.val)
+}
+
+func (any *int32Any) ToUint64() uint64 {
+	return uint64(any.val)
+}
+
+func (any *int32Any) ToFloat32() float32 {
+	return float32(any.val)
+}
+
+func (any *int32Any) ToFloat64() float64 {
+	return float64(any.val)
+}
+
+func (any *int32Any) ToString() string {
+	return strconv.FormatInt(int64(any.val), 10)
+}
+
+func (any *int32Any) WriteTo(stream *Stream) {
+	stream.WriteInt32(any.val)
+}
+
+func (any *int32Any) Parse() *Iterator {
+	return nil
+}
+
+func (any *int32Any) GetInterface() interface{} {
+	return any.val
+}
diff --git a/vendor/github.com/json-iterator/go/any_int64.go b/vendor/github.com/json-iterator/go/any_int64.go
new file mode 100644
index 0000000..c440d72
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/any_int64.go
@@ -0,0 +1,74 @@
+package jsoniter
+
+import (
+	"strconv"
+)
+
+type int64Any struct {
+	baseAny
+	val int64
+}
+
+func (any *int64Any) LastError() error {
+	return nil
+}
+
+func (any *int64Any) ValueType() ValueType {
+	return NumberValue
+}
+
+func (any *int64Any) MustBeValid() Any {
+	return any
+}
+
+func (any *int64Any) ToBool() bool {
+	return any.val != 0
+}
+
+func (any *int64Any) ToInt() int {
+	return int(any.val)
+}
+
+func (any *int64Any) ToInt32() int32 {
+	return int32(any.val)
+}
+
+func (any *int64Any) ToInt64() int64 {
+	return any.val
+}
+
+func (any *int64Any) ToUint() uint {
+	return uint(any.val)
+}
+
+func (any *int64Any) ToUint32() uint32 {
+	return uint32(any.val)
+}
+
+func (any *int64Any) ToUint64() uint64 {
+	return uint64(any.val)
+}
+
+func (any *int64Any) ToFloat32() float32 {
+	return float32(any.val)
+}
+
+func (any *int64Any) ToFloat64() float64 {
+	return float64(any.val)
+}
+
+func (any *int64Any) ToString() string {
+	return strconv.FormatInt(any.val, 10)
+}
+
+func (any *int64Any) WriteTo(stream *Stream) {
+	stream.WriteInt64(any.val)
+}
+
+func (any *int64Any) Parse() *Iterator {
+	return nil
+}
+
+func (any *int64Any) GetInterface() interface{} {
+	return any.val
+}
diff --git a/vendor/github.com/json-iterator/go/any_invalid.go b/vendor/github.com/json-iterator/go/any_invalid.go
new file mode 100644
index 0000000..1d859ea
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/any_invalid.go
@@ -0,0 +1,82 @@
+package jsoniter
+
+import "fmt"
+
+type invalidAny struct {
+	baseAny
+	err error
+}
+
+func newInvalidAny(path []interface{}) *invalidAny {
+	return &invalidAny{baseAny{}, fmt.Errorf("%v not found", path)}
+}
+
+func (any *invalidAny) LastError() error {
+	return any.err
+}
+
+func (any *invalidAny) ValueType() ValueType {
+	return InvalidValue
+}
+
+func (any *invalidAny) MustBeValid() Any {
+	panic(any.err)
+}
+
+func (any *invalidAny) ToBool() bool {
+	return false
+}
+
+func (any *invalidAny) ToInt() int {
+	return 0
+}
+
+func (any *invalidAny) ToInt32() int32 {
+	return 0
+}
+
+func (any *invalidAny) ToInt64() int64 {
+	return 0
+}
+
+func (any *invalidAny) ToUint() uint {
+	return 0
+}
+
+func (any *invalidAny) ToUint32() uint32 {
+	return 0
+}
+
+func (any *invalidAny) ToUint64() uint64 {
+	return 0
+}
+
+func (any *invalidAny) ToFloat32() float32 {
+	return 0
+}
+
+func (any *invalidAny) ToFloat64() float64 {
+	return 0
+}
+
+func (any *invalidAny) ToString() string {
+	return ""
+}
+
+func (any *invalidAny) WriteTo(stream *Stream) {
+}
+
+func (any *invalidAny) Get(path ...interface{}) Any {
+	if any.err == nil {
+		return &invalidAny{baseAny{}, fmt.Errorf("get %v from invalid", path)}
+	}
+	return &invalidAny{baseAny{}, fmt.Errorf("%v, get %v from invalid", any.err, path)}
+}
+
+func (any *invalidAny) Parse() *Iterator {
+	return nil
+}
+
+func (any *invalidAny) GetInterface() interface{} {
+	return nil
+}
diff --git a/vendor/github.com/json-iterator/go/any_nil.go b/vendor/github.com/json-iterator/go/any_nil.go
new file mode 100644
index 0000000..d04cb54
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/any_nil.go
@@ -0,0 +1,69 @@
+package jsoniter
+
+type nilAny struct {
+	baseAny
+}
+
+func (any *nilAny) LastError() error {
+	return nil
+}
+
+func (any *nilAny) ValueType() ValueType {
+	return NilValue
+}
+
+func (any *nilAny) MustBeValid() Any {
+	return any
+}
+
+func (any *nilAny) ToBool() bool {
+	return false
+}
+
+func (any *nilAny) ToInt() int {
+	return 0
+}
+
+func (any *nilAny) ToInt32() int32 {
+	return 0
+}
+
+func (any *nilAny) ToInt64() int64 {
+	return 0
+}
+
+func (any *nilAny) ToUint() uint {
+	return 0
+}
+
+func (any *nilAny) ToUint32() uint32 {
+	return 0
+}
+
+func (any *nilAny) ToUint64() uint64 {
+	return 0
+}
+
+func (any *nilAny) ToFloat32() float32 {
+	return 0
+}
+
+func (any *nilAny) ToFloat64() float64 {
+	return 0
+}
+
+func (any *nilAny) ToString() string {
+	return ""
+}
+
+func (any *nilAny) WriteTo(stream *Stream) {
+	stream.WriteNil()
+}
+
+func (any *nilAny) Parse() *Iterator {
+	return nil
+}
+
+func (any *nilAny) GetInterface() interface{} {
+	return nil
+}
diff --git a/vendor/github.com/json-iterator/go/any_number.go b/vendor/github.com/json-iterator/go/any_number.go
new file mode 100644
index 0000000..9d1e901
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/any_number.go
@@ -0,0 +1,123 @@
+package jsoniter
+
+import (
+	"io"
+	"unsafe"
+)
+
+type numberLazyAny struct {
+	baseAny
+	cfg *frozenConfig
+	buf []byte
+	err error
+}
+
+func (any *numberLazyAny) ValueType() ValueType {
+	return NumberValue
+}
+
+func (any *numberLazyAny) MustBeValid() Any {
+	return any
+}
+
+func (any *numberLazyAny) LastError() error {
+	return any.err
+}
+
+func (any *numberLazyAny) ToBool() bool {
+	return any.ToFloat64() != 0
+}
+
+func (any *numberLazyAny) ToInt() int {
+	iter := any.cfg.BorrowIterator(any.buf)
+	defer any.cfg.ReturnIterator(iter)
+	val := iter.ReadInt()
+	if iter.Error != nil && iter.Error != io.EOF {
+		any.err = iter.Error
+	}
+	return val
+}
+
+func (any *numberLazyAny) ToInt32() int32 {
+	iter := any.cfg.BorrowIterator(any.buf)
+	defer any.cfg.ReturnIterator(iter)
+	val := iter.ReadInt32()
+	if iter.Error != nil && iter.Error != io.EOF {
+		any.err = iter.Error
+	}
+	return val
+}
+
+func (any *numberLazyAny) ToInt64() int64 {
+	iter := any.cfg.BorrowIterator(any.buf)
+	defer any.cfg.ReturnIterator(iter)
+	val := iter.ReadInt64()
+	if iter.Error != nil && iter.Error != io.EOF {
+		any.err = iter.Error
+	}
+	return val
+}
+
+func (any *numberLazyAny) ToUint() uint {
+	iter := any.cfg.BorrowIterator(any.buf)
+	defer any.cfg.ReturnIterator(iter)
+	val := iter.ReadUint()
+	if iter.Error != nil && iter.Error != io.EOF {
+		any.err = iter.Error
+	}
+	return val
+}
+
+func (any *numberLazyAny) ToUint32() uint32 {
+	iter := any.cfg.BorrowIterator(any.buf)
+	defer any.cfg.ReturnIterator(iter)
+	val := iter.ReadUint32()
+	if iter.Error != nil && iter.Error != io.EOF {
+		any.err = iter.Error
+	}
+	return val
+}
+
+func (any *numberLazyAny) ToUint64() uint64 {
+	iter := any.cfg.BorrowIterator(any.buf)
+	defer any.cfg.ReturnIterator(iter)
+	val := iter.ReadUint64()
+	if iter.Error != nil && iter.Error != io.EOF {
+		any.err = iter.Error
+	}
+	return val
+}
+
+func (any *numberLazyAny) ToFloat32() float32 {
+	iter := any.cfg.BorrowIterator(any.buf)
+	defer any.cfg.ReturnIterator(iter)
+	val := iter.ReadFloat32()
+	if iter.Error != nil && iter.Error != io.EOF {
+		any.err = iter.Error
+	}
+	return val
+}
+
+func (any *numberLazyAny) ToFloat64() float64 {
+	iter := any.cfg.BorrowIterator(any.buf)
+	defer any.cfg.ReturnIterator(iter)
+	val := iter.ReadFloat64()
+	if iter.Error != nil && iter.Error != io.EOF {
+		any.err = iter.Error
+	}
+	return val
+}
+
+func (any *numberLazyAny) ToString() string {
+	return *(*string)(unsafe.Pointer(&any.buf))
+}
+
+func (any *numberLazyAny) WriteTo(stream *Stream) {
+	stream.Write(any.buf)
+}
+
+func (any *numberLazyAny) GetInterface() interface{} {
+	iter := any.cfg.BorrowIterator(any.buf)
+	defer any.cfg.ReturnIterator(iter)
+	return iter.Read()
+}
diff --git a/vendor/github.com/json-iterator/go/any_object.go b/vendor/github.com/json-iterator/go/any_object.go
new file mode 100644
index 0000000..c44ef5c
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/any_object.go
@@ -0,0 +1,374 @@
+package jsoniter
+
+import (
+	"reflect"
+	"unsafe"
+)
+
+type objectLazyAny struct {
+	baseAny
+	cfg *frozenConfig
+	buf []byte
+	err error
+}
+
+func (any *objectLazyAny) ValueType() ValueType {
+	return ObjectValue
+}
+
+func (any *objectLazyAny) MustBeValid() Any {
+	return any
+}
+
+func (any *objectLazyAny) LastError() error {
+	return any.err
+}
+
+func (any *objectLazyAny) ToBool() bool {
+	return true
+}
+
+func (any *objectLazyAny) ToInt() int {
+	return 0
+}
+
+func (any *objectLazyAny) ToInt32() int32 {
+	return 0
+}
+
+func (any *objectLazyAny) ToInt64() int64 {
+	return 0
+}
+
+func (any *objectLazyAny) ToUint() uint {
+	return 0
+}
+
+func (any *objectLazyAny) ToUint32() uint32 {
+	return 0
+}
+
+func (any *objectLazyAny) ToUint64() uint64 {
+	return 0
+}
+
+func (any *objectLazyAny) ToFloat32() float32 {
+	return 0
+}
+
+func (any *objectLazyAny) ToFloat64() float64 {
+	return 0
+}
+
+func (any *objectLazyAny) ToString() string {
+	return *(*string)(unsafe.Pointer(&any.buf))
+}
+
+func (any *objectLazyAny) ToVal(obj interface{}) {
+	iter := any.cfg.BorrowIterator(any.buf)
+	defer any.cfg.ReturnIterator(iter)
+	iter.ReadVal(obj)
+}
+
+func (any *objectLazyAny) Get(path ...interface{}) Any {
+	if len(path) == 0 {
+		return any
+	}
+	switch firstPath := path[0].(type) {
+	case string:
+		iter := any.cfg.BorrowIterator(any.buf)
+		defer any.cfg.ReturnIterator(iter)
+		valueBytes := locateObjectField(iter, firstPath)
+		if valueBytes == nil {
+			return newInvalidAny(path)
+		}
+		iter.ResetBytes(valueBytes)
+		return locatePath(iter, path[1:])
+	case int32:
+		if '*' == firstPath {
+			mappedAll := map[string]Any{}
+			iter := any.cfg.BorrowIterator(any.buf)
+			defer any.cfg.ReturnIterator(iter)
+			iter.ReadMapCB(func(iter *Iterator, field string) bool {
+				mapped := locatePath(iter, path[1:])
+				if mapped.ValueType() != InvalidValue {
+					mappedAll[field] = mapped
+				}
+				return true
+			})
+			return wrapMap(mappedAll)
+		}
+		return newInvalidAny(path)
+	default:
+		return newInvalidAny(path)
+	}
+}
+
+func (any *objectLazyAny) Keys() []string {
+	keys := []string{}
+	iter := any.cfg.BorrowIterator(any.buf)
+	defer any.cfg.ReturnIterator(iter)
+	iter.ReadMapCB(func(iter *Iterator, field string) bool {
+		iter.Skip()
+		keys = append(keys, field)
+		return true
+	})
+	return keys
+}
+
+func (any *objectLazyAny) Size() int {
+	size := 0
+	iter := any.cfg.BorrowIterator(any.buf)
+	defer any.cfg.ReturnIterator(iter)
+	iter.ReadObjectCB(func(iter *Iterator, field string) bool {
+		iter.Skip()
+		size++
+		return true
+	})
+	return size
+}
+
+func (any *objectLazyAny) WriteTo(stream *Stream) {
+	stream.Write(any.buf)
+}
+
+func (any *objectLazyAny) GetInterface() interface{} {
+	iter := any.cfg.BorrowIterator(any.buf)
+	defer any.cfg.ReturnIterator(iter)
+	return iter.Read()
+}
+
+type objectAny struct {
+	baseAny
+	err error
+	val reflect.Value
+}
+
+func wrapStruct(val interface{}) *objectAny {
+	return &objectAny{baseAny{}, nil, reflect.ValueOf(val)}
+}
+
+func (any *objectAny) ValueType() ValueType {
+	return ObjectValue
+}
+
+func (any *objectAny) MustBeValid() Any {
+	return any
+}
+
+func (any *objectAny) Parse() *Iterator {
+	return nil
+}
+
+func (any *objectAny) LastError() error {
+	return any.err
+}
+
+func (any *objectAny) ToBool() bool {
+	return any.val.NumField() != 0
+}
+
+func (any *objectAny) ToInt() int {
+	return 0
+}
+
+func (any *objectAny) ToInt32() int32 {
+	return 0
+}
+
+func (any *objectAny) ToInt64() int64 {
+	return 0
+}
+
+func (any *objectAny) ToUint() uint {
+	return 0
+}
+
+func (any *objectAny) ToUint32() uint32 {
+	return 0
+}
+
+func (any *objectAny) ToUint64() uint64 {
+	return 0
+}
+
+func (any *objectAny) ToFloat32() float32 {
+	return 0
+}
+
+func (any *objectAny) ToFloat64() float64 {
+	return 0
+}
+
+func (any *objectAny) ToString() string {
+	str, err := MarshalToString(any.val.Interface())
+	any.err = err
+	return str
+}
+
+func (any *objectAny) Get(path ...interface{}) Any {
+	if len(path) == 0 {
+		return any
+	}
+	switch firstPath := path[0].(type) {
+	case string:
+		field := any.val.FieldByName(firstPath)
+		if !field.IsValid() {
+			return newInvalidAny(path)
+		}
+		return Wrap(field.Interface())
+	case int32:
+		if '*' == firstPath {
+			mappedAll := map[string]Any{}
+			for i := 0; i < any.val.NumField(); i++ {
+				field := any.val.Field(i)
+				if field.CanInterface() {
+					mapped := Wrap(field.Interface()).Get(path[1:]...)
+					if mapped.ValueType() != InvalidValue {
+						mappedAll[any.val.Type().Field(i).Name] = mapped
+					}
+				}
+			}
+			return wrapMap(mappedAll)
+		}
+		return newInvalidAny(path)
+	default:
+		return newInvalidAny(path)
+	}
+}
+
+func (any *objectAny) Keys() []string {
+	keys := make([]string, 0, any.val.NumField())
+	for i := 0; i < any.val.NumField(); i++ {
+		keys = append(keys, any.val.Type().Field(i).Name)
+	}
+	return keys
+}
+
+func (any *objectAny) Size() int {
+	return any.val.NumField()
+}
+
+func (any *objectAny) WriteTo(stream *Stream) {
+	stream.WriteVal(any.val)
+}
+
+func (any *objectAny) GetInterface() interface{} {
+	return any.val.Interface()
+}
+
+type mapAny struct {
+	baseAny
+	err error
+	val reflect.Value
+}
+
+func wrapMap(val interface{}) *mapAny {
+	return &mapAny{baseAny{}, nil, reflect.ValueOf(val)}
+}
+
+func (any *mapAny) ValueType() ValueType {
+	return ObjectValue
+}
+
+func (any *mapAny) MustBeValid() Any {
+	return any
+}
+
+func (any *mapAny) Parse() *Iterator {
+	return nil
+}
+
+func (any *mapAny) LastError() error {
+	return any.err
+}
+
+func (any *mapAny) ToBool() bool {
+	return true
+}
+
+func (any *mapAny) ToInt() int {
+	return 0
+}
+
+func (any *mapAny) ToInt32() int32 {
+	return 0
+}
+
+func (any *mapAny) ToInt64() int64 {
+	return 0
+}
+
+func (any *mapAny) ToUint() uint {
+	return 0
+}
+
+func (any *mapAny) ToUint32() uint32 {
+	return 0
+}
+
+func (any *mapAny) ToUint64() uint64 {
+	return 0
+}
+
+func (any *mapAny) ToFloat32() float32 {
+	return 0
+}
+
+func (any *mapAny) ToFloat64() float64 {
+	return 0
+}
+
+func (any *mapAny) ToString() string {
+	str, err := MarshalToString(any.val.Interface())
+	any.err = err
+	return str
+}
+
+func (any *mapAny) Get(path ...interface{}) Any {
+	if len(path) == 0 {
+		return any
+	}
+	switch firstPath := path[0].(type) {
+	case int32:
+		if '*' == firstPath {
+			mappedAll := map[string]Any{}
+			for _, key := range any.val.MapKeys() {
+				keyAsStr := key.String()
+				element := Wrap(any.val.MapIndex(key).Interface())
+				mapped := element.Get(path[1:]...)
+				if mapped.ValueType() != InvalidValue {
+					mappedAll[keyAsStr] = mapped
+				}
+			}
+			return wrapMap(mappedAll)
+		}
+		return newInvalidAny(path)
+	default:
+		value := any.val.MapIndex(reflect.ValueOf(firstPath))
+		if !value.IsValid() {
+			return newInvalidAny(path)
+		}
+		return Wrap(value.Interface())
+	}
+}
+
+func (any *mapAny) Keys() []string {
+	keys := make([]string, 0, any.val.Len())
+	for _, key := range any.val.MapKeys() {
+		keys = append(keys, key.String())
+	}
+	return keys
+}
+
+func (any *mapAny) Size() int {
+	return any.val.Len()
+}
+
+func (any *mapAny) WriteTo(stream *Stream) {
+	stream.WriteVal(any.val)
+}
+
+func (any *mapAny) GetInterface() interface{} {
+	return any.val.Interface()
+}
diff --git a/vendor/github.com/json-iterator/go/any_str.go b/vendor/github.com/json-iterator/go/any_str.go
new file mode 100644
index 0000000..a4b93c7
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/any_str.go
@@ -0,0 +1,166 @@
+package jsoniter
+
+import (
+	"fmt"
+	"strconv"
+)
+
+type stringAny struct {
+	baseAny
+	val string
+}
+
+func (any *stringAny) Get(path ...interface{}) Any {
+	if len(path) == 0 {
+		return any
+	}
+	return &invalidAny{baseAny{}, fmt.Errorf("GetIndex %v from simple value", path)}
+}
+
+func (any *stringAny) Parse() *Iterator {
+	return nil
+}
+
+func (any *stringAny) ValueType() ValueType {
+	return StringValue
+}
+
+func (any *stringAny) MustBeValid() Any {
+	return any
+}
+
+func (any *stringAny) LastError() error {
+	return nil
+}
+
+func (any *stringAny) ToBool() bool {
+	str := any.ToString()
+	if str == "0" {
+		return false
+	}
+	for _, c := range str {
+		switch c {
+		case ' ', '\n', '\r', '\t':
+		default:
+			return true
+		}
+	}
+	return false
+}
+
+func (any *stringAny) ToInt() int {
+	return int(any.ToInt64())
+
+}
+
+func (any *stringAny) ToInt32() int32 {
+	return int32(any.ToInt64())
+}
+
+func (any *stringAny) ToInt64() int64 {
+	if any.val == "" {
+		return 0
+	}
+
+	flag := 1
+	startPos := 0
+	endPos := 0
+	if any.val[0] == '+' || any.val[0] == '-' {
+		startPos = 1
+	}
+
+	if any.val[0] == '-' {
+		flag = -1
+	}
+
+	for i := startPos; i < len(any.val); i++ {
+		if any.val[i] >= '0' && any.val[i] <= '9' {
+			endPos = i + 1
+		} else {
+			break
+		}
+	}
+	parsed, _ := strconv.ParseInt(any.val[startPos:endPos], 10, 64)
+	return int64(flag) * parsed
+}
+
+func (any *stringAny) ToUint() uint {
+	return uint(any.ToUint64())
+}
+
+func (any *stringAny) ToUint32() uint32 {
+	return uint32(any.ToUint64())
+}
+
+func (any *stringAny) ToUint64() uint64 {
+	if any.val == "" {
+		return 0
+	}
+
+	startPos := 0
+	endPos := 0
+
+	if any.val[0] == '-' {
+		return 0
+	}
+	if any.val[0] == '+' {
+		startPos = 1
+	}
+
+	for i := startPos; i < len(any.val); i++ {
+		if any.val[i] >= '0' && any.val[i] <= '9' {
+			endPos = i + 1
+		} else {
+			break
+		}
+	}
+	parsed, _ := strconv.ParseUint(any.val[startPos:endPos], 10, 64)
+	return parsed
+}
+
+func (any *stringAny) ToFloat32() float32 {
+	return float32(any.ToFloat64())
+}
+
+func (any *stringAny) ToFloat64() float64 {
+	if len(any.val) == 0 {
+		return 0
+	}
+
+	// first char invalid
+	if any.val[0] != '+' && any.val[0] != '-' && (any.val[0] > '9' || any.val[0] < '0') {
+		return 0
+	}
+
+	// extract valid num expression from string
+	// eg 123true => 123, -12.12xxa => -12.12
+	endPos := 1
+	for i := 1; i < len(any.val); i++ {
+		if any.val[i] == '.' || any.val[i] == 'e' || any.val[i] == 'E' || any.val[i] == '+' || any.val[i] == '-' {
+			endPos = i + 1
+			continue
+		}
+
+		// end position is the first char which is not digit
+		if any.val[i] >= '0' && any.val[i] <= '9' {
+			endPos = i + 1
+		} else {
+			endPos = i
+			break
+		}
+	}
+	parsed, _ := strconv.ParseFloat(any.val[:endPos], 64)
+	return parsed
+}
+
+func (any *stringAny) ToString() string {
+	return any.val
+}
+
+func (any *stringAny) WriteTo(stream *Stream) {
+	stream.WriteString(any.val)
+}
+
+func (any *stringAny) GetInterface() interface{} {
+	return any.val
+}
diff --git a/vendor/github.com/json-iterator/go/any_uint32.go b/vendor/github.com/json-iterator/go/any_uint32.go
new file mode 100644
index 0000000..656bbd3
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/any_uint32.go
@@ -0,0 +1,74 @@
+package jsoniter
+
+import (
+	"strconv"
+)
+
+type uint32Any struct {
+	baseAny
+	val uint32
+}
+
+func (any *uint32Any) LastError() error {
+	return nil
+}
+
+func (any *uint32Any) ValueType() ValueType {
+	return NumberValue
+}
+
+func (any *uint32Any) MustBeValid() Any {
+	return any
+}
+
+func (any *uint32Any) ToBool() bool {
+	return any.val != 0
+}
+
+func (any *uint32Any) ToInt() int {
+	return int(any.val)
+}
+
+func (any *uint32Any) ToInt32() int32 {
+	return int32(any.val)
+}
+
+func (any *uint32Any) ToInt64() int64 {
+	return int64(any.val)
+}
+
+func (any *uint32Any) ToUint() uint {
+	return uint(any.val)
+}
+
+func (any *uint32Any) ToUint32() uint32 {
+	return any.val
+}
+
+func (any *uint32Any) ToUint64() uint64 {
+	return uint64(any.val)
+}
+
+func (any *uint32Any) ToFloat32() float32 {
+	return float32(any.val)
+}
+
+func (any *uint32Any) ToFloat64() float64 {
+	return float64(any.val)
+}
+
+func (any *uint32Any) ToString() string {
+	return strconv.FormatInt(int64(any.val), 10)
+}
+
+func (any *uint32Any) WriteTo(stream *Stream) {
+	stream.WriteUint32(any.val)
+}
+
+func (any *uint32Any) Parse() *Iterator {
+	return nil
+}
+
+func (any *uint32Any) GetInterface() interface{} {
+	return any.val
+}
diff --git a/vendor/github.com/json-iterator/go/any_uint64.go b/vendor/github.com/json-iterator/go/any_uint64.go
new file mode 100644
index 0000000..7df2fce
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/any_uint64.go
@@ -0,0 +1,74 @@
+package jsoniter
+
+import (
+	"strconv"
+)
+
+type uint64Any struct {
+	baseAny
+	val uint64
+}
+
+func (any *uint64Any) LastError() error {
+	return nil
+}
+
+func (any *uint64Any) ValueType() ValueType {
+	return NumberValue
+}
+
+func (any *uint64Any) MustBeValid() Any {
+	return any
+}
+
+func (any *uint64Any) ToBool() bool {
+	return any.val != 0
+}
+
+func (any *uint64Any) ToInt() int {
+	return int(any.val)
+}
+
+func (any *uint64Any) ToInt32() int32 {
+	return int32(any.val)
+}
+
+func (any *uint64Any) ToInt64() int64 {
+	return int64(any.val)
+}
+
+func (any *uint64Any) ToUint() uint {
+	return uint(any.val)
+}
+
+func (any *uint64Any) ToUint32() uint32 {
+	return uint32(any.val)
+}
+
+func (any *uint64Any) ToUint64() uint64 {
+	return any.val
+}
+
+func (any *uint64Any) ToFloat32() float32 {
+	return float32(any.val)
+}
+
+func (any *uint64Any) ToFloat64() float64 {
+	return float64(any.val)
+}
+
+func (any *uint64Any) ToString() string {
+	return strconv.FormatUint(any.val, 10)
+}
+
+func (any *uint64Any) WriteTo(stream *Stream) {
+	stream.WriteUint64(any.val)
+}
+
+func (any *uint64Any) Parse() *Iterator {
+	return nil
+}
+
+func (any *uint64Any) GetInterface() interface{} {
+	return any.val
+}
diff --git a/vendor/github.com/json-iterator/go/build.sh b/vendor/github.com/json-iterator/go/build.sh
new file mode 100644
index 0000000..b45ef68
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/build.sh
@@ -0,0 +1,12 @@
+#!/bin/bash
+set -e
+set -x
+
+if [ ! -d /tmp/build-golang/src/github.com/json-iterator ]; then
+    mkdir -p /tmp/build-golang/src/github.com/json-iterator
+    ln -s $PWD /tmp/build-golang/src/github.com/json-iterator/go
+fi
+export GOPATH=/tmp/build-golang
+go get -u github.com/golang/dep/cmd/dep
+cd /tmp/build-golang/src/github.com/json-iterator/go
+exec $GOPATH/bin/dep ensure -update
diff --git a/vendor/github.com/json-iterator/go/config.go b/vendor/github.com/json-iterator/go/config.go
new file mode 100644
index 0000000..8c58fcb
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/config.go
@@ -0,0 +1,375 @@
+package jsoniter
+
+import (
+	"encoding/json"
+	"io"
+	"reflect"
+	"sync"
+	"unsafe"
+
+	"github.com/modern-go/concurrent"
+	"github.com/modern-go/reflect2"
+)
+
+// Config customize how the API should behave.
+// The API is created from Config by Froze.
+type Config struct {
+	IndentionStep                 int
+	MarshalFloatWith6Digits       bool
+	EscapeHTML                    bool
+	SortMapKeys                   bool
+	UseNumber                     bool
+	DisallowUnknownFields         bool
+	TagKey                        string
+	OnlyTaggedField               bool
+	ValidateJsonRawMessage        bool
+	ObjectFieldMustBeSimpleString bool
+	CaseSensitive                 bool
+}
+
+// API the public interface of this package.
+// Primary Marshal and Unmarshal.
+type API interface {
+	IteratorPool
+	StreamPool
+	MarshalToString(v interface{}) (string, error)
+	Marshal(v interface{}) ([]byte, error)
+	MarshalIndent(v interface{}, prefix, indent string) ([]byte, error)
+	UnmarshalFromString(str string, v interface{}) error
+	Unmarshal(data []byte, v interface{}) error
+	Get(data []byte, path ...interface{}) Any
+	NewEncoder(writer io.Writer) *Encoder
+	NewDecoder(reader io.Reader) *Decoder
+	Valid(data []byte) bool
+	RegisterExtension(extension Extension)
+	DecoderOf(typ reflect2.Type) ValDecoder
+	EncoderOf(typ reflect2.Type) ValEncoder
+}
+
+// ConfigDefault the default API
+var ConfigDefault = Config{
+	EscapeHTML: true,
+}.Froze()
+
+// ConfigCompatibleWithStandardLibrary tries to be 100% compatible with standard library behavior
+var ConfigCompatibleWithStandardLibrary = Config{
+	EscapeHTML:             true,
+	SortMapKeys:            true,
+	ValidateJsonRawMessage: true,
+}.Froze()
+
+// ConfigFastest marshals float with only 6 digits precision
+var ConfigFastest = Config{
+	EscapeHTML:                    false,
+	MarshalFloatWith6Digits:       true, // will lose precession
+	ObjectFieldMustBeSimpleString: true, // do not unescape object field
+}.Froze()
+
+type frozenConfig struct {
+	configBeforeFrozen            Config
+	sortMapKeys                   bool
+	indentionStep                 int
+	objectFieldMustBeSimpleString bool
+	onlyTaggedField               bool
+	disallowUnknownFields         bool
+	decoderCache                  *concurrent.Map
+	encoderCache                  *concurrent.Map
+	encoderExtension              Extension
+	decoderExtension              Extension
+	extraExtensions               []Extension
+	streamPool                    *sync.Pool
+	iteratorPool                  *sync.Pool
+	caseSensitive                 bool
+}
+
+func (cfg *frozenConfig) initCache() {
+	cfg.decoderCache = concurrent.NewMap()
+	cfg.encoderCache = concurrent.NewMap()
+}
+
+func (cfg *frozenConfig) addDecoderToCache(cacheKey uintptr, decoder ValDecoder) {
+	cfg.decoderCache.Store(cacheKey, decoder)
+}
+
+func (cfg *frozenConfig) addEncoderToCache(cacheKey uintptr, encoder ValEncoder) {
+	cfg.encoderCache.Store(cacheKey, encoder)
+}
+
+func (cfg *frozenConfig) getDecoderFromCache(cacheKey uintptr) ValDecoder {
+	decoder, found := cfg.decoderCache.Load(cacheKey)
+	if found {
+		return decoder.(ValDecoder)
+	}
+	return nil
+}
+
+func (cfg *frozenConfig) getEncoderFromCache(cacheKey uintptr) ValEncoder {
+	encoder, found := cfg.encoderCache.Load(cacheKey)
+	if found {
+		return encoder.(ValEncoder)
+	}
+	return nil
+}
+
+var cfgCache = concurrent.NewMap()
+
+func getFrozenConfigFromCache(cfg Config) *frozenConfig {
+	obj, found := cfgCache.Load(cfg)
+	if found {
+		return obj.(*frozenConfig)
+	}
+	return nil
+}
+
+func addFrozenConfigToCache(cfg Config, frozenConfig *frozenConfig) {
+	cfgCache.Store(cfg, frozenConfig)
+}
+
+// Froze forge API from config
+func (cfg Config) Froze() API {
+	api := &frozenConfig{
+		sortMapKeys:                   cfg.SortMapKeys,
+		indentionStep:                 cfg.IndentionStep,
+		objectFieldMustBeSimpleString: cfg.ObjectFieldMustBeSimpleString,
+		onlyTaggedField:               cfg.OnlyTaggedField,
+		disallowUnknownFields:         cfg.DisallowUnknownFields,
+		caseSensitive:                 cfg.CaseSensitive,
+	}
+	api.streamPool = &sync.Pool{
+		New: func() interface{} {
+			return NewStream(api, nil, 512)
+		},
+	}
+	api.iteratorPool = &sync.Pool{
+		New: func() interface{} {
+			return NewIterator(api)
+		},
+	}
+	api.initCache()
+	encoderExtension := EncoderExtension{}
+	decoderExtension := DecoderExtension{}
+	if cfg.MarshalFloatWith6Digits {
+		api.marshalFloatWith6Digits(encoderExtension)
+	}
+	if cfg.EscapeHTML {
+		api.escapeHTML(encoderExtension)
+	}
+	if cfg.UseNumber {
+		api.useNumber(decoderExtension)
+	}
+	if cfg.ValidateJsonRawMessage {
+		api.validateJsonRawMessage(encoderExtension)
+	}
+	api.encoderExtension = encoderExtension
+	api.decoderExtension = decoderExtension
+	api.configBeforeFrozen = cfg
+	return api
+}
+
+func (cfg Config) frozeWithCacheReuse(extraExtensions []Extension) *frozenConfig {
+	api := getFrozenConfigFromCache(cfg)
+	if api != nil {
+		return api
+	}
+	api = cfg.Froze().(*frozenConfig)
+	for _, extension := range extraExtensions {
+		api.RegisterExtension(extension)
+	}
+	addFrozenConfigToCache(cfg, api)
+	return api
+}
+
+func (cfg *frozenConfig) validateJsonRawMessage(extension EncoderExtension) {
+	encoder := &funcEncoder{func(ptr unsafe.Pointer, stream *Stream) {
+		rawMessage := *(*json.RawMessage)(ptr)
+		iter := cfg.BorrowIterator([]byte(rawMessage))
+		iter.Read()
+		if iter.Error != nil {
+			stream.WriteRaw("null")
+		} else {
+			cfg.ReturnIterator(iter)
+			stream.WriteRaw(string(rawMessage))
+		}
+	}, func(ptr unsafe.Pointer) bool {
+		return len(*((*json.RawMessage)(ptr))) == 0
+	}}
+	extension[reflect2.TypeOfPtr((*json.RawMessage)(nil)).Elem()] = encoder
+	extension[reflect2.TypeOfPtr((*RawMessage)(nil)).Elem()] = encoder
+}
+
+func (cfg *frozenConfig) useNumber(extension DecoderExtension) {
+	extension[reflect2.TypeOfPtr((*interface{})(nil)).Elem()] = &funcDecoder{func(ptr unsafe.Pointer, iter *Iterator) {
+		exitingValue := *((*interface{})(ptr))
+		if exitingValue != nil && reflect.TypeOf(exitingValue).Kind() == reflect.Ptr {
+			iter.ReadVal(exitingValue)
+			return
+		}
+		if iter.WhatIsNext() == NumberValue {
+			*((*interface{})(ptr)) = json.Number(iter.readNumberAsString())
+		} else {
+			*((*interface{})(ptr)) = iter.Read()
+		}
+	}}
+}
+func (cfg *frozenConfig) getTagKey() string {
+	tagKey := cfg.configBeforeFrozen.TagKey
+	if tagKey == "" {
+		return "json"
+	}
+	return tagKey
+}
+
+func (cfg *frozenConfig) RegisterExtension(extension Extension) {
+	cfg.extraExtensions = append(cfg.extraExtensions, extension)
+	copied := cfg.configBeforeFrozen
+	cfg.configBeforeFrozen = copied
+}
+
+type lossyFloat32Encoder struct {
+}
+
+func (encoder *lossyFloat32Encoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteFloat32Lossy(*((*float32)(ptr)))
+}
+
+func (encoder *lossyFloat32Encoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return *((*float32)(ptr)) == 0
+}
+
+type lossyFloat64Encoder struct {
+}
+
+func (encoder *lossyFloat64Encoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteFloat64Lossy(*((*float64)(ptr)))
+}
+
+func (encoder *lossyFloat64Encoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return *((*float64)(ptr)) == 0
+}
+
+// EnableLossyFloatMarshalling keeps 10**(-6) precision
+// for float variables for better performance.
+func (cfg *frozenConfig) marshalFloatWith6Digits(extension EncoderExtension) {
+	// for better performance
+	extension[reflect2.TypeOfPtr((*float32)(nil)).Elem()] = &lossyFloat32Encoder{}
+	extension[reflect2.TypeOfPtr((*float64)(nil)).Elem()] = &lossyFloat64Encoder{}
+}
+
+type htmlEscapedStringEncoder struct {
+}
+
+func (encoder *htmlEscapedStringEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	str := *((*string)(ptr))
+	stream.WriteStringWithHTMLEscaped(str)
+}
+
+func (encoder *htmlEscapedStringEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return *((*string)(ptr)) == ""
+}
+
+func (cfg *frozenConfig) escapeHTML(encoderExtension EncoderExtension) {
+	encoderExtension[reflect2.TypeOfPtr((*string)(nil)).Elem()] = &htmlEscapedStringEncoder{}
+}
+
+func (cfg *frozenConfig) cleanDecoders() {
+	typeDecoders = map[string]ValDecoder{}
+	fieldDecoders = map[string]ValDecoder{}
+	*cfg = *(cfg.configBeforeFrozen.Froze().(*frozenConfig))
+}
+
+func (cfg *frozenConfig) cleanEncoders() {
+	typeEncoders = map[string]ValEncoder{}
+	fieldEncoders = map[string]ValEncoder{}
+	*cfg = *(cfg.configBeforeFrozen.Froze().(*frozenConfig))
+}
+
+func (cfg *frozenConfig) MarshalToString(v interface{}) (string, error) {
+	stream := cfg.BorrowStream(nil)
+	defer cfg.ReturnStream(stream)
+	stream.WriteVal(v)
+	if stream.Error != nil {
+		return "", stream.Error
+	}
+	return string(stream.Buffer()), nil
+}
+
+func (cfg *frozenConfig) Marshal(v interface{}) ([]byte, error) {
+	stream := cfg.BorrowStream(nil)
+	defer cfg.ReturnStream(stream)
+	stream.WriteVal(v)
+	if stream.Error != nil {
+		return nil, stream.Error
+	}
+	result := stream.Buffer()
+	copied := make([]byte, len(result))
+	copy(copied, result)
+	return copied, nil
+}
+
+func (cfg *frozenConfig) MarshalIndent(v interface{}, prefix, indent string) ([]byte, error) {
+	if prefix != "" {
+		panic("prefix is not supported")
+	}
+	for _, r := range indent {
+		if r != ' ' {
+			panic("indent can only be space")
+		}
+	}
+	newCfg := cfg.configBeforeFrozen
+	newCfg.IndentionStep = len(indent)
+	return newCfg.frozeWithCacheReuse(cfg.extraExtensions).Marshal(v)
+}
+
+func (cfg *frozenConfig) UnmarshalFromString(str string, v interface{}) error {
+	data := []byte(str)
+	iter := cfg.BorrowIterator(data)
+	defer cfg.ReturnIterator(iter)
+	iter.ReadVal(v)
+	c := iter.nextToken()
+	if c == 0 {
+		if iter.Error == io.EOF {
+			return nil
+		}
+		return iter.Error
+	}
+	iter.ReportError("Unmarshal", "there are bytes left after unmarshal")
+	return iter.Error
+}
+
+func (cfg *frozenConfig) Get(data []byte, path ...interface{}) Any {
+	iter := cfg.BorrowIterator(data)
+	defer cfg.ReturnIterator(iter)
+	return locatePath(iter, path)
+}
+
+func (cfg *frozenConfig) Unmarshal(data []byte, v interface{}) error {
+	iter := cfg.BorrowIterator(data)
+	defer cfg.ReturnIterator(iter)
+	iter.ReadVal(v)
+	c := iter.nextToken()
+	if c == 0 {
+		if iter.Error == io.EOF {
+			return nil
+		}
+		return iter.Error
+	}
+	iter.ReportError("Unmarshal", "there are bytes left after unmarshal")
+	return iter.Error
+}
+
+func (cfg *frozenConfig) NewEncoder(writer io.Writer) *Encoder {
+	stream := NewStream(cfg, writer, 512)
+	return &Encoder{stream}
+}
+
+func (cfg *frozenConfig) NewDecoder(reader io.Reader) *Decoder {
+	iter := Parse(cfg, reader, 512)
+	return &Decoder{iter}
+}
+
+func (cfg *frozenConfig) Valid(data []byte) bool {
+	iter := cfg.BorrowIterator(data)
+	defer cfg.ReturnIterator(iter)
+	iter.Skip()
+	return iter.Error == nil
+}
diff --git a/vendor/github.com/json-iterator/go/fuzzy_mode_convert_table.md b/vendor/github.com/json-iterator/go/fuzzy_mode_convert_table.md
new file mode 100644
index 0000000..3095662
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/fuzzy_mode_convert_table.md
@@ -0,0 +1,7 @@
+| json type \ dest type | bool | int | uint | float |string|
+| --- | --- | --- | --- |--|--|
+| number | positive => true <br/> negative => true <br/> zero => false| 23.2 => 23 <br/> -32.1 => -32| 12.1 => 12 <br/> -12.1 => 0|as normal|same as origin|
+| string | empty string => false <br/> string "0" => false <br/> other strings => true | "123.32" => 123 <br/> "-123.4" => -123 <br/> "123.23xxxw" => 123 <br/>  "abcde12" => 0 <br/> "-32.1" => -32| 13.2 => 13 <br/> -1.1 => 0 |12.1 => 12.1 <br/> -12.3 => -12.3<br/> 12.4xxa => 12.4 <br/> +1.1e2 =>110 |same as origin|
+| bool | true => true <br/> false => false| true => 1 <br/> false => 0 | true => 1 <br/> false => 0 |true => 1 <br/>false => 0|true => "true" <br/> false => "false"|
+| object | true | 0 | 0 |0|originnal json|
+| array | empty array => false <br/> nonempty array => true| [] => 0 <br/> [1,2] => 1 | [] => 0 <br/> [1,2] => 1 |[] => 0<br/>[1,2] => 1|original json|
\ No newline at end of file
diff --git a/vendor/github.com/json-iterator/go/go.mod b/vendor/github.com/json-iterator/go/go.mod
new file mode 100644
index 0000000..e05c42f
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/go.mod
@@ -0,0 +1,11 @@
+module github.com/json-iterator/go
+
+go 1.12
+
+require (
+	github.com/davecgh/go-spew v1.1.1
+	github.com/google/gofuzz v1.0.0
+	github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421
+	github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742
+	github.com/stretchr/testify v1.3.0
+)
diff --git a/vendor/github.com/json-iterator/go/go.sum b/vendor/github.com/json-iterator/go/go.sum
new file mode 100644
index 0000000..d778b5a
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/go.sum
@@ -0,0 +1,14 @@
+github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
+github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
+github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
+github.com/google/gofuzz v1.0.0 h1:A8PeW59pxE9IoFRqBp37U+mSNaQoZ46F1f0f863XSXw=
+github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg=
+github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421 h1:ZqeYNhU3OHLH3mGKHDcjJRFFRrJa6eAM5H+CtDdOsPc=
+github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
+github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742 h1:Esafd1046DLDQ0W1YjYsBW+p8U2u7vzgW2SQVmlNazg=
+github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
+github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM=
+github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
+github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
+github.com/stretchr/testify v1.3.0 h1:TivCn/peBQ7UY8ooIcPgZFpTNSz0Q2U6UrFlUfqbe0Q=
+github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI=
diff --git a/vendor/github.com/json-iterator/go/iter.go b/vendor/github.com/json-iterator/go/iter.go
new file mode 100644
index 0000000..95ae54f
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/iter.go
@@ -0,0 +1,322 @@
+package jsoniter
+
+import (
+	"encoding/json"
+	"fmt"
+	"io"
+)
+
+// ValueType the type for JSON element
+type ValueType int
+
+const (
+	// InvalidValue invalid JSON element
+	InvalidValue ValueType = iota
+	// StringValue JSON element "string"
+	StringValue
+	// NumberValue JSON element 100 or 0.10
+	NumberValue
+	// NilValue JSON element null
+	NilValue
+	// BoolValue JSON element true or false
+	BoolValue
+	// ArrayValue JSON element []
+	ArrayValue
+	// ObjectValue JSON element {}
+	ObjectValue
+)
+
+var hexDigits []byte
+var valueTypes []ValueType
+
+func init() {
+	hexDigits = make([]byte, 256)
+	for i := 0; i < len(hexDigits); i++ {
+		hexDigits[i] = 255
+	}
+	for i := '0'; i <= '9'; i++ {
+		hexDigits[i] = byte(i - '0')
+	}
+	for i := 'a'; i <= 'f'; i++ {
+		hexDigits[i] = byte((i - 'a') + 10)
+	}
+	for i := 'A'; i <= 'F'; i++ {
+		hexDigits[i] = byte((i - 'A') + 10)
+	}
+	valueTypes = make([]ValueType, 256)
+	for i := 0; i < len(valueTypes); i++ {
+		valueTypes[i] = InvalidValue
+	}
+	valueTypes['"'] = StringValue
+	valueTypes['-'] = NumberValue
+	valueTypes['0'] = NumberValue
+	valueTypes['1'] = NumberValue
+	valueTypes['2'] = NumberValue
+	valueTypes['3'] = NumberValue
+	valueTypes['4'] = NumberValue
+	valueTypes['5'] = NumberValue
+	valueTypes['6'] = NumberValue
+	valueTypes['7'] = NumberValue
+	valueTypes['8'] = NumberValue
+	valueTypes['9'] = NumberValue
+	valueTypes['t'] = BoolValue
+	valueTypes['f'] = BoolValue
+	valueTypes['n'] = NilValue
+	valueTypes['['] = ArrayValue
+	valueTypes['{'] = ObjectValue
+}
+
+// Iterator is a io.Reader like object, with JSON specific read functions.
+// Error is not returned as return value, but stored as Error member on this iterator instance.
+type Iterator struct {
+	cfg              *frozenConfig
+	reader           io.Reader
+	buf              []byte
+	head             int
+	tail             int
+	captureStartedAt int
+	captured         []byte
+	Error            error
+	Attachment       interface{} // open for customized decoder
+}
+
+// NewIterator creates an empty Iterator instance
+func NewIterator(cfg API) *Iterator {
+	return &Iterator{
+		cfg:    cfg.(*frozenConfig),
+		reader: nil,
+		buf:    nil,
+		head:   0,
+		tail:   0,
+	}
+}
+
+// Parse creates an Iterator instance from io.Reader
+func Parse(cfg API, reader io.Reader, bufSize int) *Iterator {
+	return &Iterator{
+		cfg:    cfg.(*frozenConfig),
+		reader: reader,
+		buf:    make([]byte, bufSize),
+		head:   0,
+		tail:   0,
+	}
+}
+
+// ParseBytes creates an Iterator instance from byte array
+func ParseBytes(cfg API, input []byte) *Iterator {
+	return &Iterator{
+		cfg:    cfg.(*frozenConfig),
+		reader: nil,
+		buf:    input,
+		head:   0,
+		tail:   len(input),
+	}
+}
+
+// ParseString creates an Iterator instance from string
+func ParseString(cfg API, input string) *Iterator {
+	return ParseBytes(cfg, []byte(input))
+}
+
+// Pool returns a pool can provide more iterator with same configuration
+func (iter *Iterator) Pool() IteratorPool {
+	return iter.cfg
+}
+
+// Reset reuse iterator instance by specifying another reader
+func (iter *Iterator) Reset(reader io.Reader) *Iterator {
+	iter.reader = reader
+	iter.head = 0
+	iter.tail = 0
+	return iter
+}
+
+// ResetBytes reuse iterator instance by specifying another byte array as input
+func (iter *Iterator) ResetBytes(input []byte) *Iterator {
+	iter.reader = nil
+	iter.buf = input
+	iter.head = 0
+	iter.tail = len(input)
+	return iter
+}
+
+// WhatIsNext gets ValueType of relatively next json element
+func (iter *Iterator) WhatIsNext() ValueType {
+	valueType := valueTypes[iter.nextToken()]
+	iter.unreadByte()
+	return valueType
+}
+
+func (iter *Iterator) skipWhitespacesWithoutLoadMore() bool {
+	for i := iter.head; i < iter.tail; i++ {
+		c := iter.buf[i]
+		switch c {
+		case ' ', '\n', '\t', '\r':
+			continue
+		}
+		iter.head = i
+		return false
+	}
+	return true
+}
+
+func (iter *Iterator) isObjectEnd() bool {
+	c := iter.nextToken()
+	if c == ',' {
+		return false
+	}
+	if c == '}' {
+		return true
+	}
+	iter.ReportError("isObjectEnd", "object ended prematurely, unexpected char "+string([]byte{c}))
+	return true
+}
+
+func (iter *Iterator) nextToken() byte {
+	// a variation of skip whitespaces, returning the next non-whitespace token
+	for {
+		for i := iter.head; i < iter.tail; i++ {
+			c := iter.buf[i]
+			switch c {
+			case ' ', '\n', '\t', '\r':
+				continue
+			}
+			iter.head = i + 1
+			return c
+		}
+		if !iter.loadMore() {
+			return 0
+		}
+	}
+}
+
+// ReportError record a error in iterator instance with current position.
+func (iter *Iterator) ReportError(operation string, msg string) {
+	if iter.Error != nil {
+		if iter.Error != io.EOF {
+			return
+		}
+	}
+	peekStart := iter.head - 10
+	if peekStart < 0 {
+		peekStart = 0
+	}
+	peekEnd := iter.head + 10
+	if peekEnd > iter.tail {
+		peekEnd = iter.tail
+	}
+	parsing := string(iter.buf[peekStart:peekEnd])
+	contextStart := iter.head - 50
+	if contextStart < 0 {
+		contextStart = 0
+	}
+	contextEnd := iter.head + 50
+	if contextEnd > iter.tail {
+		contextEnd = iter.tail
+	}
+	context := string(iter.buf[contextStart:contextEnd])
+	iter.Error = fmt.Errorf("%s: %s, error found in #%v byte of ...|%s|..., bigger context ...|%s|...",
+		operation, msg, iter.head-peekStart, parsing, context)
+}
+
+// CurrentBuffer gets current buffer as string for debugging purpose
+func (iter *Iterator) CurrentBuffer() string {
+	peekStart := iter.head - 10
+	if peekStart < 0 {
+		peekStart = 0
+	}
+	return fmt.Sprintf("parsing #%v byte, around ...|%s|..., whole buffer ...|%s|...", iter.head,
+		string(iter.buf[peekStart:iter.head]), string(iter.buf[0:iter.tail]))
+}
+
+func (iter *Iterator) readByte() (ret byte) {
+	if iter.head == iter.tail {
+		if iter.loadMore() {
+			ret = iter.buf[iter.head]
+			iter.head++
+			return ret
+		}
+		return 0
+	}
+	ret = iter.buf[iter.head]
+	iter.head++
+	return ret
+}
+
+func (iter *Iterator) loadMore() bool {
+	if iter.reader == nil {
+		if iter.Error == nil {
+			iter.head = iter.tail
+			iter.Error = io.EOF
+		}
+		return false
+	}
+	if iter.captured != nil {
+		iter.captured = append(iter.captured,
+			iter.buf[iter.captureStartedAt:iter.tail]...)
+		iter.captureStartedAt = 0
+	}
+	for {
+		n, err := iter.reader.Read(iter.buf)
+		if n == 0 {
+			if err != nil {
+				if iter.Error == nil {
+					iter.Error = err
+				}
+				return false
+			}
+		} else {
+			iter.head = 0
+			iter.tail = n
+			return true
+		}
+	}
+}
+
+func (iter *Iterator) unreadByte() {
+	if iter.Error != nil {
+		return
+	}
+	iter.head--
+	return
+}
+
+// Read read the next JSON element as generic interface{}.
+func (iter *Iterator) Read() interface{} {
+	valueType := iter.WhatIsNext()
+	switch valueType {
+	case StringValue:
+		return iter.ReadString()
+	case NumberValue:
+		if iter.cfg.configBeforeFrozen.UseNumber {
+			return json.Number(iter.readNumberAsString())
+		}
+		return iter.ReadFloat64()
+	case NilValue:
+		iter.skipFourBytes('n', 'u', 'l', 'l')
+		return nil
+	case BoolValue:
+		return iter.ReadBool()
+	case ArrayValue:
+		arr := []interface{}{}
+		iter.ReadArrayCB(func(iter *Iterator) bool {
+			var elem interface{}
+			iter.ReadVal(&elem)
+			arr = append(arr, elem)
+			return true
+		})
+		return arr
+	case ObjectValue:
+		obj := map[string]interface{}{}
+		iter.ReadMapCB(func(Iter *Iterator, field string) bool {
+			var elem interface{}
+			iter.ReadVal(&elem)
+			obj[field] = elem
+			return true
+		})
+		return obj
+	default:
+		iter.ReportError("Read", fmt.Sprintf("unexpected value type: %v", valueType))
+		return nil
+	}
+}
diff --git a/vendor/github.com/json-iterator/go/iter_array.go b/vendor/github.com/json-iterator/go/iter_array.go
new file mode 100644
index 0000000..6188cb4
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/iter_array.go
@@ -0,0 +1,58 @@
+package jsoniter
+
+// ReadArray read array element, tells if the array has more element to read.
+func (iter *Iterator) ReadArray() (ret bool) {
+	c := iter.nextToken()
+	switch c {
+	case 'n':
+		iter.skipThreeBytes('u', 'l', 'l')
+		return false // null
+	case '[':
+		c = iter.nextToken()
+		if c != ']' {
+			iter.unreadByte()
+			return true
+		}
+		return false
+	case ']':
+		return false
+	case ',':
+		return true
+	default:
+		iter.ReportError("ReadArray", "expect [ or , or ] or n, but found "+string([]byte{c}))
+		return
+	}
+}
+
+// ReadArrayCB read array with callback
+func (iter *Iterator) ReadArrayCB(callback func(*Iterator) bool) (ret bool) {
+	c := iter.nextToken()
+	if c == '[' {
+		c = iter.nextToken()
+		if c != ']' {
+			iter.unreadByte()
+			if !callback(iter) {
+				return false
+			}
+			c = iter.nextToken()
+			for c == ',' {
+				if !callback(iter) {
+					return false
+				}
+				c = iter.nextToken()
+			}
+			if c != ']' {
+				iter.ReportError("ReadArrayCB", "expect ] in the end, but found "+string([]byte{c}))
+				return false
+			}
+			return true
+		}
+		return true
+	}
+	if c == 'n' {
+		iter.skipThreeBytes('u', 'l', 'l')
+		return true // null
+	}
+	iter.ReportError("ReadArrayCB", "expect [ or n, but found "+string([]byte{c}))
+	return false
+}
diff --git a/vendor/github.com/json-iterator/go/iter_float.go b/vendor/github.com/json-iterator/go/iter_float.go
new file mode 100644
index 0000000..b975463
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/iter_float.go
@@ -0,0 +1,339 @@
+package jsoniter
+
+import (
+	"encoding/json"
+	"io"
+	"math/big"
+	"strconv"
+	"strings"
+	"unsafe"
+)
+
+var floatDigits []int8
+
+const invalidCharForNumber = int8(-1)
+const endOfNumber = int8(-2)
+const dotInNumber = int8(-3)
+
+func init() {
+	floatDigits = make([]int8, 256)
+	for i := 0; i < len(floatDigits); i++ {
+		floatDigits[i] = invalidCharForNumber
+	}
+	for i := int8('0'); i <= int8('9'); i++ {
+		floatDigits[i] = i - int8('0')
+	}
+	floatDigits[','] = endOfNumber
+	floatDigits[']'] = endOfNumber
+	floatDigits['}'] = endOfNumber
+	floatDigits[' '] = endOfNumber
+	floatDigits['\t'] = endOfNumber
+	floatDigits['\n'] = endOfNumber
+	floatDigits['.'] = dotInNumber
+}
+
+// ReadBigFloat read big.Float
+func (iter *Iterator) ReadBigFloat() (ret *big.Float) {
+	str := iter.readNumberAsString()
+	if iter.Error != nil && iter.Error != io.EOF {
+		return nil
+	}
+	prec := 64
+	if len(str) > prec {
+		prec = len(str)
+	}
+	val, _, err := big.ParseFloat(str, 10, uint(prec), big.ToZero)
+	if err != nil {
+		iter.Error = err
+		return nil
+	}
+	return val
+}
+
+// ReadBigInt read big.Int
+func (iter *Iterator) ReadBigInt() (ret *big.Int) {
+	str := iter.readNumberAsString()
+	if iter.Error != nil && iter.Error != io.EOF {
+		return nil
+	}
+	ret = big.NewInt(0)
+	var success bool
+	ret, success = ret.SetString(str, 10)
+	if !success {
+		iter.ReportError("ReadBigInt", "invalid big int")
+		return nil
+	}
+	return ret
+}
+
+//ReadFloat32 read float32
+func (iter *Iterator) ReadFloat32() (ret float32) {
+	c := iter.nextToken()
+	if c == '-' {
+		return -iter.readPositiveFloat32()
+	}
+	iter.unreadByte()
+	return iter.readPositiveFloat32()
+}
+
+func (iter *Iterator) readPositiveFloat32() (ret float32) {
+	i := iter.head
+	// first char
+	if i == iter.tail {
+		return iter.readFloat32SlowPath()
+	}
+	c := iter.buf[i]
+	i++
+	ind := floatDigits[c]
+	switch ind {
+	case invalidCharForNumber:
+		return iter.readFloat32SlowPath()
+	case endOfNumber:
+		iter.ReportError("readFloat32", "empty number")
+		return
+	case dotInNumber:
+		iter.ReportError("readFloat32", "leading dot is invalid")
+		return
+	case 0:
+		if i == iter.tail {
+			return iter.readFloat32SlowPath()
+		}
+		c = iter.buf[i]
+		switch c {
+		case '0', '1', '2', '3', '4', '5', '6', '7', '8', '9':
+			iter.ReportError("readFloat32", "leading zero is invalid")
+			return
+		}
+	}
+	value := uint64(ind)
+	// chars before dot
+non_decimal_loop:
+	for ; i < iter.tail; i++ {
+		c = iter.buf[i]
+		ind := floatDigits[c]
+		switch ind {
+		case invalidCharForNumber:
+			return iter.readFloat32SlowPath()
+		case endOfNumber:
+			iter.head = i
+			return float32(value)
+		case dotInNumber:
+			break non_decimal_loop
+		}
+		if value > uint64SafeToMultiple10 {
+			return iter.readFloat32SlowPath()
+		}
+		value = (value << 3) + (value << 1) + uint64(ind) // value = value * 10 + ind;
+	}
+	// chars after dot
+	if c == '.' {
+		i++
+		decimalPlaces := 0
+		if i == iter.tail {
+			return iter.readFloat32SlowPath()
+		}
+		for ; i < iter.tail; i++ {
+			c = iter.buf[i]
+			ind := floatDigits[c]
+			switch ind {
+			case endOfNumber:
+				if decimalPlaces > 0 && decimalPlaces < len(pow10) {
+					iter.head = i
+					return float32(float64(value) / float64(pow10[decimalPlaces]))
+				}
+				// too many decimal places
+				return iter.readFloat32SlowPath()
+			case invalidCharForNumber, dotInNumber:
+				return iter.readFloat32SlowPath()
+			}
+			decimalPlaces++
+			if value > uint64SafeToMultiple10 {
+				return iter.readFloat32SlowPath()
+			}
+			value = (value << 3) + (value << 1) + uint64(ind)
+		}
+	}
+	return iter.readFloat32SlowPath()
+}
+
+func (iter *Iterator) readNumberAsString() (ret string) {
+	strBuf := [16]byte{}
+	str := strBuf[0:0]
+load_loop:
+	for {
+		for i := iter.head; i < iter.tail; i++ {
+			c := iter.buf[i]
+			switch c {
+			case '+', '-', '.', 'e', 'E', '0', '1', '2', '3', '4', '5', '6', '7', '8', '9':
+				str = append(str, c)
+				continue
+			default:
+				iter.head = i
+				break load_loop
+			}
+		}
+		if !iter.loadMore() {
+			break
+		}
+	}
+	if iter.Error != nil && iter.Error != io.EOF {
+		return
+	}
+	if len(str) == 0 {
+		iter.ReportError("readNumberAsString", "invalid number")
+	}
+	return *(*string)(unsafe.Pointer(&str))
+}
+
+func (iter *Iterator) readFloat32SlowPath() (ret float32) {
+	str := iter.readNumberAsString()
+	if iter.Error != nil && iter.Error != io.EOF {
+		return
+	}
+	errMsg := validateFloat(str)
+	if errMsg != "" {
+		iter.ReportError("readFloat32SlowPath", errMsg)
+		return
+	}
+	val, err := strconv.ParseFloat(str, 32)
+	if err != nil {
+		iter.Error = err
+		return
+	}
+	return float32(val)
+}
+
+// ReadFloat64 read float64
+func (iter *Iterator) ReadFloat64() (ret float64) {
+	c := iter.nextToken()
+	if c == '-' {
+		return -iter.readPositiveFloat64()
+	}
+	iter.unreadByte()
+	return iter.readPositiveFloat64()
+}
+
+func (iter *Iterator) readPositiveFloat64() (ret float64) {
+	i := iter.head
+	// first char
+	if i == iter.tail {
+		return iter.readFloat64SlowPath()
+	}
+	c := iter.buf[i]
+	i++
+	ind := floatDigits[c]
+	switch ind {
+	case invalidCharForNumber:
+		return iter.readFloat64SlowPath()
+	case endOfNumber:
+		iter.ReportError("readFloat64", "empty number")
+		return
+	case dotInNumber:
+		iter.ReportError("readFloat64", "leading dot is invalid")
+		return
+	case 0:
+		if i == iter.tail {
+			return iter.readFloat64SlowPath()
+		}
+		c = iter.buf[i]
+		switch c {
+		case '0', '1', '2', '3', '4', '5', '6', '7', '8', '9':
+			iter.ReportError("readFloat64", "leading zero is invalid")
+			return
+		}
+	}
+	value := uint64(ind)
+	// chars before dot
+non_decimal_loop:
+	for ; i < iter.tail; i++ {
+		c = iter.buf[i]
+		ind := floatDigits[c]
+		switch ind {
+		case invalidCharForNumber:
+			return iter.readFloat64SlowPath()
+		case endOfNumber:
+			iter.head = i
+			return float64(value)
+		case dotInNumber:
+			break non_decimal_loop
+		}
+		if value > uint64SafeToMultiple10 {
+			return iter.readFloat64SlowPath()
+		}
+		value = (value << 3) + (value << 1) + uint64(ind) // value = value * 10 + ind;
+	}
+	// chars after dot
+	if c == '.' {
+		i++
+		decimalPlaces := 0
+		if i == iter.tail {
+			return iter.readFloat64SlowPath()
+		}
+		for ; i < iter.tail; i++ {
+			c = iter.buf[i]
+			ind := floatDigits[c]
+			switch ind {
+			case endOfNumber:
+				if decimalPlaces > 0 && decimalPlaces < len(pow10) {
+					iter.head = i
+					return float64(value) / float64(pow10[decimalPlaces])
+				}
+				// too many decimal places
+				return iter.readFloat64SlowPath()
+			case invalidCharForNumber, dotInNumber:
+				return iter.readFloat64SlowPath()
+			}
+			decimalPlaces++
+			if value > uint64SafeToMultiple10 {
+				return iter.readFloat64SlowPath()
+			}
+			value = (value << 3) + (value << 1) + uint64(ind)
+		}
+	}
+	return iter.readFloat64SlowPath()
+}
+
+func (iter *Iterator) readFloat64SlowPath() (ret float64) {
+	str := iter.readNumberAsString()
+	if iter.Error != nil && iter.Error != io.EOF {
+		return
+	}
+	errMsg := validateFloat(str)
+	if errMsg != "" {
+		iter.ReportError("readFloat64SlowPath", errMsg)
+		return
+	}
+	val, err := strconv.ParseFloat(str, 64)
+	if err != nil {
+		iter.Error = err
+		return
+	}
+	return val
+}
+
+func validateFloat(str string) string {
+	// strconv.ParseFloat is not validating `1.` or `1.e1`
+	if len(str) == 0 {
+		return "empty number"
+	}
+	if str[0] == '-' {
+		return "-- is not valid"
+	}
+	dotPos := strings.IndexByte(str, '.')
+	if dotPos != -1 {
+		if dotPos == len(str)-1 {
+			return "dot can not be last character"
+		}
+		switch str[dotPos+1] {
+		case '0', '1', '2', '3', '4', '5', '6', '7', '8', '9':
+		default:
+			return "missing digit after dot"
+		}
+	}
+	return ""
+}
+
+// ReadNumber read json.Number
+func (iter *Iterator) ReadNumber() (ret json.Number) {
+	return json.Number(iter.readNumberAsString())
+}
diff --git a/vendor/github.com/json-iterator/go/iter_int.go b/vendor/github.com/json-iterator/go/iter_int.go
new file mode 100644
index 0000000..2142320
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/iter_int.go
@@ -0,0 +1,345 @@
+package jsoniter
+
+import (
+	"math"
+	"strconv"
+)
+
+var intDigits []int8
+
+const uint32SafeToMultiply10 = uint32(0xffffffff)/10 - 1
+const uint64SafeToMultiple10 = uint64(0xffffffffffffffff)/10 - 1
+
+func init() {
+	intDigits = make([]int8, 256)
+	for i := 0; i < len(intDigits); i++ {
+		intDigits[i] = invalidCharForNumber
+	}
+	for i := int8('0'); i <= int8('9'); i++ {
+		intDigits[i] = i - int8('0')
+	}
+}
+
+// ReadUint read uint
+func (iter *Iterator) ReadUint() uint {
+	if strconv.IntSize == 32 {
+		return uint(iter.ReadUint32())
+	}
+	return uint(iter.ReadUint64())
+}
+
+// ReadInt read int
+func (iter *Iterator) ReadInt() int {
+	if strconv.IntSize == 32 {
+		return int(iter.ReadInt32())
+	}
+	return int(iter.ReadInt64())
+}
+
+// ReadInt8 read int8
+func (iter *Iterator) ReadInt8() (ret int8) {
+	c := iter.nextToken()
+	if c == '-' {
+		val := iter.readUint32(iter.readByte())
+		if val > math.MaxInt8+1 {
+			iter.ReportError("ReadInt8", "overflow: "+strconv.FormatInt(int64(val), 10))
+			return
+		}
+		return -int8(val)
+	}
+	val := iter.readUint32(c)
+	if val > math.MaxInt8 {
+		iter.ReportError("ReadInt8", "overflow: "+strconv.FormatInt(int64(val), 10))
+		return
+	}
+	return int8(val)
+}
+
+// ReadUint8 read uint8
+func (iter *Iterator) ReadUint8() (ret uint8) {
+	val := iter.readUint32(iter.nextToken())
+	if val > math.MaxUint8 {
+		iter.ReportError("ReadUint8", "overflow: "+strconv.FormatInt(int64(val), 10))
+		return
+	}
+	return uint8(val)
+}
+
+// ReadInt16 read int16
+func (iter *Iterator) ReadInt16() (ret int16) {
+	c := iter.nextToken()
+	if c == '-' {
+		val := iter.readUint32(iter.readByte())
+		if val > math.MaxInt16+1 {
+			iter.ReportError("ReadInt16", "overflow: "+strconv.FormatInt(int64(val), 10))
+			return
+		}
+		return -int16(val)
+	}
+	val := iter.readUint32(c)
+	if val > math.MaxInt16 {
+		iter.ReportError("ReadInt16", "overflow: "+strconv.FormatInt(int64(val), 10))
+		return
+	}
+	return int16(val)
+}
+
+// ReadUint16 read uint16
+func (iter *Iterator) ReadUint16() (ret uint16) {
+	val := iter.readUint32(iter.nextToken())
+	if val > math.MaxUint16 {
+		iter.ReportError("ReadUint16", "overflow: "+strconv.FormatInt(int64(val), 10))
+		return
+	}
+	return uint16(val)
+}
+
+// ReadInt32 read int32
+func (iter *Iterator) ReadInt32() (ret int32) {
+	c := iter.nextToken()
+	if c == '-' {
+		val := iter.readUint32(iter.readByte())
+		if val > math.MaxInt32+1 {
+			iter.ReportError("ReadInt32", "overflow: "+strconv.FormatInt(int64(val), 10))
+			return
+		}
+		return -int32(val)
+	}
+	val := iter.readUint32(c)
+	if val > math.MaxInt32 {
+		iter.ReportError("ReadInt32", "overflow: "+strconv.FormatInt(int64(val), 10))
+		return
+	}
+	return int32(val)
+}
+
+// ReadUint32 read uint32
+func (iter *Iterator) ReadUint32() (ret uint32) {
+	return iter.readUint32(iter.nextToken())
+}
+
+func (iter *Iterator) readUint32(c byte) (ret uint32) {
+	ind := intDigits[c]
+	if ind == 0 {
+		iter.assertInteger()
+		return 0 // single zero
+	}
+	if ind == invalidCharForNumber {
+		iter.ReportError("readUint32", "unexpected character: "+string([]byte{byte(ind)}))
+		return
+	}
+	value := uint32(ind)
+	if iter.tail-iter.head > 10 {
+		i := iter.head
+		ind2 := intDigits[iter.buf[i]]
+		if ind2 == invalidCharForNumber {
+			iter.head = i
+			iter.assertInteger()
+			return value
+		}
+		i++
+		ind3 := intDigits[iter.buf[i]]
+		if ind3 == invalidCharForNumber {
+			iter.head = i
+			iter.assertInteger()
+			return value*10 + uint32(ind2)
+		}
+		//iter.head = i + 1
+		//value = value * 100 + uint32(ind2) * 10 + uint32(ind3)
+		i++
+		ind4 := intDigits[iter.buf[i]]
+		if ind4 == invalidCharForNumber {
+			iter.head = i
+			iter.assertInteger()
+			return value*100 + uint32(ind2)*10 + uint32(ind3)
+		}
+		i++
+		ind5 := intDigits[iter.buf[i]]
+		if ind5 == invalidCharForNumber {
+			iter.head = i
+			iter.assertInteger()
+			return value*1000 + uint32(ind2)*100 + uint32(ind3)*10 + uint32(ind4)
+		}
+		i++
+		ind6 := intDigits[iter.buf[i]]
+		if ind6 == invalidCharForNumber {
+			iter.head = i
+			iter.assertInteger()
+			return value*10000 + uint32(ind2)*1000 + uint32(ind3)*100 + uint32(ind4)*10 + uint32(ind5)
+		}
+		i++
+		ind7 := intDigits[iter.buf[i]]
+		if ind7 == invalidCharForNumber {
+			iter.head = i
+			iter.assertInteger()
+			return value*100000 + uint32(ind2)*10000 + uint32(ind3)*1000 + uint32(ind4)*100 + uint32(ind5)*10 + uint32(ind6)
+		}
+		i++
+		ind8 := intDigits[iter.buf[i]]
+		if ind8 == invalidCharForNumber {
+			iter.head = i
+			iter.assertInteger()
+			return value*1000000 + uint32(ind2)*100000 + uint32(ind3)*10000 + uint32(ind4)*1000 + uint32(ind5)*100 + uint32(ind6)*10 + uint32(ind7)
+		}
+		i++
+		ind9 := intDigits[iter.buf[i]]
+		value = value*10000000 + uint32(ind2)*1000000 + uint32(ind3)*100000 + uint32(ind4)*10000 + uint32(ind5)*1000 + uint32(ind6)*100 + uint32(ind7)*10 + uint32(ind8)
+		iter.head = i
+		if ind9 == invalidCharForNumber {
+			iter.assertInteger()
+			return value
+		}
+	}
+	for {
+		for i := iter.head; i < iter.tail; i++ {
+			ind = intDigits[iter.buf[i]]
+			if ind == invalidCharForNumber {
+				iter.head = i
+				iter.assertInteger()
+				return value
+			}
+			if value > uint32SafeToMultiply10 {
+				value2 := (value << 3) + (value << 1) + uint32(ind)
+				if value2 < value {
+					iter.ReportError("readUint32", "overflow")
+					return
+				}
+				value = value2
+				continue
+			}
+			value = (value << 3) + (value << 1) + uint32(ind)
+		}
+		if !iter.loadMore() {
+			iter.assertInteger()
+			return value
+		}
+	}
+}
+
+// ReadInt64 read int64
+func (iter *Iterator) ReadInt64() (ret int64) {
+	c := iter.nextToken()
+	if c == '-' {
+		val := iter.readUint64(iter.readByte())
+		if val > math.MaxInt64+1 {
+			iter.ReportError("ReadInt64", "overflow: "+strconv.FormatUint(uint64(val), 10))
+			return
+		}
+		return -int64(val)
+	}
+	val := iter.readUint64(c)
+	if val > math.MaxInt64 {
+		iter.ReportError("ReadInt64", "overflow: "+strconv.FormatUint(uint64(val), 10))
+		return
+	}
+	return int64(val)
+}
+
+// ReadUint64 read uint64
+func (iter *Iterator) ReadUint64() uint64 {
+	return iter.readUint64(iter.nextToken())
+}
+
+func (iter *Iterator) readUint64(c byte) (ret uint64) {
+	ind := intDigits[c]
+	if ind == 0 {
+		iter.assertInteger()
+		return 0 // single zero
+	}
+	if ind == invalidCharForNumber {
+		iter.ReportError("readUint64", "unexpected character: "+string([]byte{byte(ind)}))
+		return
+	}
+	value := uint64(ind)
+	if iter.tail-iter.head > 10 {
+		i := iter.head
+		ind2 := intDigits[iter.buf[i]]
+		if ind2 == invalidCharForNumber {
+			iter.head = i
+			iter.assertInteger()
+			return value
+		}
+		i++
+		ind3 := intDigits[iter.buf[i]]
+		if ind3 == invalidCharForNumber {
+			iter.head = i
+			iter.assertInteger()
+			return value*10 + uint64(ind2)
+		}
+		//iter.head = i + 1
+		//value = value * 100 + uint32(ind2) * 10 + uint32(ind3)
+		i++
+		ind4 := intDigits[iter.buf[i]]
+		if ind4 == invalidCharForNumber {
+			iter.head = i
+			iter.assertInteger()
+			return value*100 + uint64(ind2)*10 + uint64(ind3)
+		}
+		i++
+		ind5 := intDigits[iter.buf[i]]
+		if ind5 == invalidCharForNumber {
+			iter.head = i
+			iter.assertInteger()
+			return value*1000 + uint64(ind2)*100 + uint64(ind3)*10 + uint64(ind4)
+		}
+		i++
+		ind6 := intDigits[iter.buf[i]]
+		if ind6 == invalidCharForNumber {
+			iter.head = i
+			iter.assertInteger()
+			return value*10000 + uint64(ind2)*1000 + uint64(ind3)*100 + uint64(ind4)*10 + uint64(ind5)
+		}
+		i++
+		ind7 := intDigits[iter.buf[i]]
+		if ind7 == invalidCharForNumber {
+			iter.head = i
+			iter.assertInteger()
+			return value*100000 + uint64(ind2)*10000 + uint64(ind3)*1000 + uint64(ind4)*100 + uint64(ind5)*10 + uint64(ind6)
+		}
+		i++
+		ind8 := intDigits[iter.buf[i]]
+		if ind8 == invalidCharForNumber {
+			iter.head = i
+			iter.assertInteger()
+			return value*1000000 + uint64(ind2)*100000 + uint64(ind3)*10000 + uint64(ind4)*1000 + uint64(ind5)*100 + uint64(ind6)*10 + uint64(ind7)
+		}
+		i++
+		ind9 := intDigits[iter.buf[i]]
+		value = value*10000000 + uint64(ind2)*1000000 + uint64(ind3)*100000 + uint64(ind4)*10000 + uint64(ind5)*1000 + uint64(ind6)*100 + uint64(ind7)*10 + uint64(ind8)
+		iter.head = i
+		if ind9 == invalidCharForNumber {
+			iter.assertInteger()
+			return value
+		}
+	}
+	for {
+		for i := iter.head; i < iter.tail; i++ {
+			ind = intDigits[iter.buf[i]]
+			if ind == invalidCharForNumber {
+				iter.head = i
+				iter.assertInteger()
+				return value
+			}
+			if value > uint64SafeToMultiple10 {
+				value2 := (value << 3) + (value << 1) + uint64(ind)
+				if value2 < value {
+					iter.ReportError("readUint64", "overflow")
+					return
+				}
+				value = value2
+				continue
+			}
+			value = (value << 3) + (value << 1) + uint64(ind)
+		}
+		if !iter.loadMore() {
+			iter.assertInteger()
+			return value
+		}
+	}
+}
+
+func (iter *Iterator) assertInteger() {
+	if iter.head < len(iter.buf) && iter.buf[iter.head] == '.' {
+		iter.ReportError("assertInteger", "can not decode float as int")
+	}
+}
diff --git a/vendor/github.com/json-iterator/go/iter_object.go b/vendor/github.com/json-iterator/go/iter_object.go
new file mode 100644
index 0000000..1c57576
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/iter_object.go
@@ -0,0 +1,251 @@
+package jsoniter
+
+import (
+	"fmt"
+	"strings"
+)
+
+// ReadObject read one field from object.
+// If object ended, returns empty string.
+// Otherwise, returns the field name.
+func (iter *Iterator) ReadObject() (ret string) {
+	c := iter.nextToken()
+	switch c {
+	case 'n':
+		iter.skipThreeBytes('u', 'l', 'l')
+		return "" // null
+	case '{':
+		c = iter.nextToken()
+		if c == '"' {
+			iter.unreadByte()
+			field := iter.ReadString()
+			c = iter.nextToken()
+			if c != ':' {
+				iter.ReportError("ReadObject", "expect : after object field, but found "+string([]byte{c}))
+			}
+			return field
+		}
+		if c == '}' {
+			return "" // end of object
+		}
+		iter.ReportError("ReadObject", `expect " after {, but found `+string([]byte{c}))
+		return
+	case ',':
+		field := iter.ReadString()
+		c = iter.nextToken()
+		if c != ':' {
+			iter.ReportError("ReadObject", "expect : after object field, but found "+string([]byte{c}))
+		}
+		return field
+	case '}':
+		return "" // end of object
+	default:
+		iter.ReportError("ReadObject", fmt.Sprintf(`expect { or , or } or n, but found %s`, string([]byte{c})))
+		return
+	}
+}
+
+// CaseInsensitive
+func (iter *Iterator) readFieldHash() int64 {
+	hash := int64(0x811c9dc5)
+	c := iter.nextToken()
+	if c != '"' {
+		iter.ReportError("readFieldHash", `expect ", but found `+string([]byte{c}))
+		return 0
+	}
+	for {
+		for i := iter.head; i < iter.tail; i++ {
+			// require ascii string and no escape
+			b := iter.buf[i]
+			if b == '\\' {
+				iter.head = i
+				for _, b := range iter.readStringSlowPath() {
+					if 'A' <= b && b <= 'Z' && !iter.cfg.caseSensitive {
+						b += 'a' - 'A'
+					}
+					hash ^= int64(b)
+					hash *= 0x1000193
+				}
+				c = iter.nextToken()
+				if c != ':' {
+					iter.ReportError("readFieldHash", `expect :, but found `+string([]byte{c}))
+					return 0
+				}
+				return hash
+			}
+			if b == '"' {
+				iter.head = i + 1
+				c = iter.nextToken()
+				if c != ':' {
+					iter.ReportError("readFieldHash", `expect :, but found `+string([]byte{c}))
+					return 0
+				}
+				return hash
+			}
+			if 'A' <= b && b <= 'Z' && !iter.cfg.caseSensitive {
+				b += 'a' - 'A'
+			}
+			hash ^= int64(b)
+			hash *= 0x1000193
+		}
+		if !iter.loadMore() {
+			iter.ReportError("readFieldHash", `incomplete field name`)
+			return 0
+		}
+	}
+}
+
+func calcHash(str string, caseSensitive bool) int64 {
+	if !caseSensitive {
+		str = strings.ToLower(str)
+	}
+	hash := int64(0x811c9dc5)
+	for _, b := range []byte(str) {
+		hash ^= int64(b)
+		hash *= 0x1000193
+	}
+	return int64(hash)
+}
+
+// ReadObjectCB read object with callback, the key is ascii only and field name not copied
+func (iter *Iterator) ReadObjectCB(callback func(*Iterator, string) bool) bool {
+	c := iter.nextToken()
+	var field string
+	if c == '{' {
+		c = iter.nextToken()
+		if c == '"' {
+			iter.unreadByte()
+			field = iter.ReadString()
+			c = iter.nextToken()
+			if c != ':' {
+				iter.ReportError("ReadObject", "expect : after object field, but found "+string([]byte{c}))
+			}
+			if !callback(iter, field) {
+				return false
+			}
+			c = iter.nextToken()
+			for c == ',' {
+				field = iter.ReadString()
+				c = iter.nextToken()
+				if c != ':' {
+					iter.ReportError("ReadObject", "expect : after object field, but found "+string([]byte{c}))
+				}
+				if !callback(iter, field) {
+					return false
+				}
+				c = iter.nextToken()
+			}
+			if c != '}' {
+				iter.ReportError("ReadObjectCB", `object not ended with }`)
+				return false
+			}
+			return true
+		}
+		if c == '}' {
+			return true
+		}
+		iter.ReportError("ReadObjectCB", `expect " after }, but found `+string([]byte{c}))
+		return false
+	}
+	if c == 'n' {
+		iter.skipThreeBytes('u', 'l', 'l')
+		return true // null
+	}
+	iter.ReportError("ReadObjectCB", `expect { or n, but found `+string([]byte{c}))
+	return false
+}
+
+// ReadMapCB read map with callback, the key can be any string
+func (iter *Iterator) ReadMapCB(callback func(*Iterator, string) bool) bool {
+	c := iter.nextToken()
+	if c == '{' {
+		c = iter.nextToken()
+		if c == '"' {
+			iter.unreadByte()
+			field := iter.ReadString()
+			if iter.nextToken() != ':' {
+				iter.ReportError("ReadMapCB", "expect : after object field, but found "+string([]byte{c}))
+				return false
+			}
+			if !callback(iter, field) {
+				return false
+			}
+			c = iter.nextToken()
+			for c == ',' {
+				field = iter.ReadString()
+				if iter.nextToken() != ':' {
+					iter.ReportError("ReadMapCB", "expect : after object field, but found "+string([]byte{c}))
+					return false
+				}
+				if !callback(iter, field) {
+					return false
+				}
+				c = iter.nextToken()
+			}
+			if c != '}' {
+				iter.ReportError("ReadMapCB", `object not ended with }`)
+				return false
+			}
+			return true
+		}
+		if c == '}' {
+			return true
+		}
+		iter.ReportError("ReadMapCB", `expect " after }, but found `+string([]byte{c}))
+		return false
+	}
+	if c == 'n' {
+		iter.skipThreeBytes('u', 'l', 'l')
+		return true // null
+	}
+	iter.ReportError("ReadMapCB", `expect { or n, but found `+string([]byte{c}))
+	return false
+}
+
+func (iter *Iterator) readObjectStart() bool {
+	c := iter.nextToken()
+	if c == '{' {
+		c = iter.nextToken()
+		if c == '}' {
+			return false
+		}
+		iter.unreadByte()
+		return true
+	} else if c == 'n' {
+		iter.skipThreeBytes('u', 'l', 'l')
+		return false
+	}
+	iter.ReportError("readObjectStart", "expect { or n, but found "+string([]byte{c}))
+	return false
+}
+
+func (iter *Iterator) readObjectFieldAsBytes() (ret []byte) {
+	str := iter.ReadStringAsSlice()
+	if iter.skipWhitespacesWithoutLoadMore() {
+		if ret == nil {
+			ret = make([]byte, len(str))
+			copy(ret, str)
+		}
+		if !iter.loadMore() {
+			return
+		}
+	}
+	if iter.buf[iter.head] != ':' {
+		iter.ReportError("readObjectFieldAsBytes", "expect : after object field, but found "+string([]byte{iter.buf[iter.head]}))
+		return
+	}
+	iter.head++
+	if iter.skipWhitespacesWithoutLoadMore() {
+		if ret == nil {
+			ret = make([]byte, len(str))
+			copy(ret, str)
+		}
+		if !iter.loadMore() {
+			return
+		}
+	}
+	if ret == nil {
+		return str
+	}
+	return ret
+}
diff --git a/vendor/github.com/json-iterator/go/iter_skip.go b/vendor/github.com/json-iterator/go/iter_skip.go
new file mode 100644
index 0000000..e91eefb
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/iter_skip.go
@@ -0,0 +1,130 @@
+package jsoniter
+
+import "fmt"
+
+// ReadNil reads a json object as nil and
+// returns whether it's a nil or not
+func (iter *Iterator) ReadNil() (ret bool) {
+	c := iter.nextToken()
+	if c == 'n' {
+		iter.skipThreeBytes('u', 'l', 'l') // null
+		return true
+	}
+	iter.unreadByte()
+	return false
+}
+
+// ReadBool reads a json object as BoolValue
+func (iter *Iterator) ReadBool() (ret bool) {
+	c := iter.nextToken()
+	if c == 't' {
+		iter.skipThreeBytes('r', 'u', 'e')
+		return true
+	}
+	if c == 'f' {
+		iter.skipFourBytes('a', 'l', 's', 'e')
+		return false
+	}
+	iter.ReportError("ReadBool", "expect t or f, but found "+string([]byte{c}))
+	return
+}
+
+// SkipAndReturnBytes skip next JSON element, and return its content as []byte.
+// The []byte can be kept, it is a copy of data.
+func (iter *Iterator) SkipAndReturnBytes() []byte {
+	iter.startCapture(iter.head)
+	iter.Skip()
+	return iter.stopCapture()
+}
+
+// SkipAndAppendBytes skips next JSON element and appends its content to
+// buffer, returning the result.
+func (iter *Iterator) SkipAndAppendBytes(buf []byte) []byte {
+	iter.startCaptureTo(buf, iter.head)
+	iter.Skip()
+	return iter.stopCapture()
+}
+
+func (iter *Iterator) startCaptureTo(buf []byte, captureStartedAt int) {
+	if iter.captured != nil {
+		panic("already in capture mode")
+	}
+	iter.captureStartedAt = captureStartedAt
+	iter.captured = buf
+}
+
+func (iter *Iterator) startCapture(captureStartedAt int) {
+	iter.startCaptureTo(make([]byte, 0, 32), captureStartedAt)
+}
+
+func (iter *Iterator) stopCapture() []byte {
+	if iter.captured == nil {
+		panic("not in capture mode")
+	}
+	captured := iter.captured
+	remaining := iter.buf[iter.captureStartedAt:iter.head]
+	iter.captureStartedAt = -1
+	iter.captured = nil
+	return append(captured, remaining...)
+}
+
+// Skip skips a json object and positions to relatively the next json object
+func (iter *Iterator) Skip() {
+	c := iter.nextToken()
+	switch c {
+	case '"':
+		iter.skipString()
+	case 'n':
+		iter.skipThreeBytes('u', 'l', 'l') // null
+	case 't':
+		iter.skipThreeBytes('r', 'u', 'e') // true
+	case 'f':
+		iter.skipFourBytes('a', 'l', 's', 'e') // false
+	case '0':
+		iter.unreadByte()
+		iter.ReadFloat32()
+	case '-', '1', '2', '3', '4', '5', '6', '7', '8', '9':
+		iter.skipNumber()
+	case '[':
+		iter.skipArray()
+	case '{':
+		iter.skipObject()
+	default:
+		iter.ReportError("Skip", fmt.Sprintf("do not know how to skip: %v", c))
+		return
+	}
+}
+
+func (iter *Iterator) skipFourBytes(b1, b2, b3, b4 byte) {
+	if iter.readByte() != b1 {
+		iter.ReportError("skipFourBytes", fmt.Sprintf("expect %s", string([]byte{b1, b2, b3, b4})))
+		return
+	}
+	if iter.readByte() != b2 {
+		iter.ReportError("skipFourBytes", fmt.Sprintf("expect %s", string([]byte{b1, b2, b3, b4})))
+		return
+	}
+	if iter.readByte() != b3 {
+		iter.ReportError("skipFourBytes", fmt.Sprintf("expect %s", string([]byte{b1, b2, b3, b4})))
+		return
+	}
+	if iter.readByte() != b4 {
+		iter.ReportError("skipFourBytes", fmt.Sprintf("expect %s", string([]byte{b1, b2, b3, b4})))
+		return
+	}
+}
+
+func (iter *Iterator) skipThreeBytes(b1, b2, b3 byte) {
+	if iter.readByte() != b1 {
+		iter.ReportError("skipThreeBytes", fmt.Sprintf("expect %s", string([]byte{b1, b2, b3})))
+		return
+	}
+	if iter.readByte() != b2 {
+		iter.ReportError("skipThreeBytes", fmt.Sprintf("expect %s", string([]byte{b1, b2, b3})))
+		return
+	}
+	if iter.readByte() != b3 {
+		iter.ReportError("skipThreeBytes", fmt.Sprintf("expect %s", string([]byte{b1, b2, b3})))
+		return
+	}
+}
diff --git a/vendor/github.com/json-iterator/go/iter_skip_sloppy.go b/vendor/github.com/json-iterator/go/iter_skip_sloppy.go
new file mode 100644
index 0000000..8fcdc3b
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/iter_skip_sloppy.go
@@ -0,0 +1,144 @@
+//+build jsoniter_sloppy
+
+package jsoniter
+
+// sloppy but faster implementation, do not validate the input json
+
+func (iter *Iterator) skipNumber() {
+	for {
+		for i := iter.head; i < iter.tail; i++ {
+			c := iter.buf[i]
+			switch c {
+			case ' ', '\n', '\r', '\t', ',', '}', ']':
+				iter.head = i
+				return
+			}
+		}
+		if !iter.loadMore() {
+			return
+		}
+	}
+}
+
+func (iter *Iterator) skipArray() {
+	level := 1
+	for {
+		for i := iter.head; i < iter.tail; i++ {
+			switch iter.buf[i] {
+			case '"': // If inside string, skip it
+				iter.head = i + 1
+				iter.skipString()
+				i = iter.head - 1 // it will be i++ soon
+			case '[': // If open symbol, increase level
+				level++
+			case ']': // If close symbol, increase level
+				level--
+
+				// If we have returned to the original level, we're done
+				if level == 0 {
+					iter.head = i + 1
+					return
+				}
+			}
+		}
+		if !iter.loadMore() {
+			iter.ReportError("skipObject", "incomplete array")
+			return
+		}
+	}
+}
+
+func (iter *Iterator) skipObject() {
+	level := 1
+	for {
+		for i := iter.head; i < iter.tail; i++ {
+			switch iter.buf[i] {
+			case '"': // If inside string, skip it
+				iter.head = i + 1
+				iter.skipString()
+				i = iter.head - 1 // it will be i++ soon
+			case '{': // If open symbol, increase level
+				level++
+			case '}': // If close symbol, increase level
+				level--
+
+				// If we have returned to the original level, we're done
+				if level == 0 {
+					iter.head = i + 1
+					return
+				}
+			}
+		}
+		if !iter.loadMore() {
+			iter.ReportError("skipObject", "incomplete object")
+			return
+		}
+	}
+}
+
+func (iter *Iterator) skipString() {
+	for {
+		end, escaped := iter.findStringEnd()
+		if end == -1 {
+			if !iter.loadMore() {
+				iter.ReportError("skipString", "incomplete string")
+				return
+			}
+			if escaped {
+				iter.head = 1 // skip the first char as last char read is \
+			}
+		} else {
+			iter.head = end
+			return
+		}
+	}
+}
+
+// adapted from: https://github.com/buger/jsonparser/blob/master/parser.go
+// Tries to find the end of string
+// Support if string contains escaped quote symbols.
+func (iter *Iterator) findStringEnd() (int, bool) {
+	escaped := false
+	for i := iter.head; i < iter.tail; i++ {
+		c := iter.buf[i]
+		if c == '"' {
+			if !escaped {
+				return i + 1, false
+			}
+			j := i - 1
+			for {
+				if j < iter.head || iter.buf[j] != '\\' {
+					// even number of backslashes
+					// either end of buffer, or " found
+					return i + 1, true
+				}
+				j--
+				if j < iter.head || iter.buf[j] != '\\' {
+					// odd number of backslashes
+					// it is \" or \\\"
+					break
+				}
+				j--
+			}
+		} else if c == '\\' {
+			escaped = true
+		}
+	}
+	j := iter.tail - 1
+	for {
+		if j < iter.head || iter.buf[j] != '\\' {
+			// even number of backslashes
+			// either end of buffer, or " found
+			return -1, false // do not end with \
+		}
+		j--
+		if j < iter.head || iter.buf[j] != '\\' {
+			// odd number of backslashes
+			// it is \" or \\\"
+			break
+		}
+		j--
+
+	}
+	return -1, true // end with \
+}
diff --git a/vendor/github.com/json-iterator/go/iter_skip_strict.go b/vendor/github.com/json-iterator/go/iter_skip_strict.go
new file mode 100644
index 0000000..6cf66d0
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/iter_skip_strict.go
@@ -0,0 +1,99 @@
+//+build !jsoniter_sloppy
+
+package jsoniter
+
+import (
+	"fmt"
+	"io"
+)
+
+func (iter *Iterator) skipNumber() {
+	if !iter.trySkipNumber() {
+		iter.unreadByte()
+		if iter.Error != nil && iter.Error != io.EOF {
+			return
+		}
+		iter.ReadFloat64()
+		if iter.Error != nil && iter.Error != io.EOF {
+			iter.Error = nil
+			iter.ReadBigFloat()
+		}
+	}
+}
+
+func (iter *Iterator) trySkipNumber() bool {
+	dotFound := false
+	for i := iter.head; i < iter.tail; i++ {
+		c := iter.buf[i]
+		switch c {
+		case '0', '1', '2', '3', '4', '5', '6', '7', '8', '9':
+		case '.':
+			if dotFound {
+				iter.ReportError("validateNumber", `more than one dot found in number`)
+				return true // already failed
+			}
+			if i+1 == iter.tail {
+				return false
+			}
+			c = iter.buf[i+1]
+			switch c {
+			case '0', '1', '2', '3', '4', '5', '6', '7', '8', '9':
+			default:
+				iter.ReportError("validateNumber", `missing digit after dot`)
+				return true // already failed
+			}
+			dotFound = true
+		default:
+			switch c {
+			case ',', ']', '}', ' ', '\t', '\n', '\r':
+				if iter.head == i {
+					return false // if - without following digits
+				}
+				iter.head = i
+				return true // must be valid
+			}
+			return false // may be invalid
+		}
+	}
+	return false
+}
+
+func (iter *Iterator) skipString() {
+	if !iter.trySkipString() {
+		iter.unreadByte()
+		iter.ReadString()
+	}
+}
+
+func (iter *Iterator) trySkipString() bool {
+	for i := iter.head; i < iter.tail; i++ {
+		c := iter.buf[i]
+		if c == '"' {
+			iter.head = i + 1
+			return true // valid
+		} else if c == '\\' {
+			return false
+		} else if c < ' ' {
+			iter.ReportError("trySkipString",
+				fmt.Sprintf(`invalid control character found: %d`, c))
+			return true // already failed
+		}
+	}
+	return false
+}
+
+func (iter *Iterator) skipObject() {
+	iter.unreadByte()
+	iter.ReadObjectCB(func(iter *Iterator, field string) bool {
+		iter.Skip()
+		return true
+	})
+}
+
+func (iter *Iterator) skipArray() {
+	iter.unreadByte()
+	iter.ReadArrayCB(func(iter *Iterator) bool {
+		iter.Skip()
+		return true
+	})
+}
diff --git a/vendor/github.com/json-iterator/go/iter_str.go b/vendor/github.com/json-iterator/go/iter_str.go
new file mode 100644
index 0000000..adc487e
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/iter_str.go
@@ -0,0 +1,215 @@
+package jsoniter
+
+import (
+	"fmt"
+	"unicode/utf16"
+)
+
+// ReadString read string from iterator
+func (iter *Iterator) ReadString() (ret string) {
+	c := iter.nextToken()
+	if c == '"' {
+		for i := iter.head; i < iter.tail; i++ {
+			c := iter.buf[i]
+			if c == '"' {
+				ret = string(iter.buf[iter.head:i])
+				iter.head = i + 1
+				return ret
+			} else if c == '\\' {
+				break
+			} else if c < ' ' {
+				iter.ReportError("ReadString",
+					fmt.Sprintf(`invalid control character found: %d`, c))
+				return
+			}
+		}
+		return iter.readStringSlowPath()
+	} else if c == 'n' {
+		iter.skipThreeBytes('u', 'l', 'l')
+		return ""
+	}
+	iter.ReportError("ReadString", `expects " or n, but found `+string([]byte{c}))
+	return
+}
+
+func (iter *Iterator) readStringSlowPath() (ret string) {
+	var str []byte
+	var c byte
+	for iter.Error == nil {
+		c = iter.readByte()
+		if c == '"' {
+			return string(str)
+		}
+		if c == '\\' {
+			c = iter.readByte()
+			str = iter.readEscapedChar(c, str)
+		} else {
+			str = append(str, c)
+		}
+	}
+	iter.ReportError("readStringSlowPath", "unexpected end of input")
+	return
+}
+
+func (iter *Iterator) readEscapedChar(c byte, str []byte) []byte {
+	switch c {
+	case 'u':
+		r := iter.readU4()
+		if utf16.IsSurrogate(r) {
+			c = iter.readByte()
+			if iter.Error != nil {
+				return nil
+			}
+			if c != '\\' {
+				iter.unreadByte()
+				str = appendRune(str, r)
+				return str
+			}
+			c = iter.readByte()
+			if iter.Error != nil {
+				return nil
+			}
+			if c != 'u' {
+				str = appendRune(str, r)
+				return iter.readEscapedChar(c, str)
+			}
+			r2 := iter.readU4()
+			if iter.Error != nil {
+				return nil
+			}
+			combined := utf16.DecodeRune(r, r2)
+			if combined == '\uFFFD' {
+				str = appendRune(str, r)
+				str = appendRune(str, r2)
+			} else {
+				str = appendRune(str, combined)
+			}
+		} else {
+			str = appendRune(str, r)
+		}
+	case '"':
+		str = append(str, '"')
+	case '\\':
+		str = append(str, '\\')
+	case '/':
+		str = append(str, '/')
+	case 'b':
+		str = append(str, '\b')
+	case 'f':
+		str = append(str, '\f')
+	case 'n':
+		str = append(str, '\n')
+	case 'r':
+		str = append(str, '\r')
+	case 't':
+		str = append(str, '\t')
+	default:
+		iter.ReportError("readEscapedChar",
+			`invalid escape char after \`)
+		return nil
+	}
+	return str
+}
+
+// ReadStringAsSlice read string from iterator without copying into string form.
+// The []byte can not be kept, as it will change after next iterator call.
+func (iter *Iterator) ReadStringAsSlice() (ret []byte) {
+	c := iter.nextToken()
+	if c == '"' {
+		for i := iter.head; i < iter.tail; i++ {
+			// require ascii string and no escape
+			// for: field name, base64, number
+			if iter.buf[i] == '"' {
+				// fast path: reuse the underlying buffer
+				ret = iter.buf[iter.head:i]
+				iter.head = i + 1
+				return ret
+			}
+		}
+		readLen := iter.tail - iter.head
+		copied := make([]byte, readLen, readLen*2)
+		copy(copied, iter.buf[iter.head:iter.tail])
+		iter.head = iter.tail
+		for iter.Error == nil {
+			c := iter.readByte()
+			if c == '"' {
+				return copied
+			}
+			copied = append(copied, c)
+		}
+		return copied
+	}
+	iter.ReportError("ReadStringAsSlice", `expects " or n, but found `+string([]byte{c}))
+	return
+}
+
+func (iter *Iterator) readU4() (ret rune) {
+	for i := 0; i < 4; i++ {
+		c := iter.readByte()
+		if iter.Error != nil {
+			return
+		}
+		if c >= '0' && c <= '9' {
+			ret = ret*16 + rune(c-'0')
+		} else if c >= 'a' && c <= 'f' {
+			ret = ret*16 + rune(c-'a'+10)
+		} else if c >= 'A' && c <= 'F' {
+			ret = ret*16 + rune(c-'A'+10)
+		} else {
+			iter.ReportError("readU4", "expects 0~9 or a~f, but found "+string([]byte{c}))
+			return
+		}
+	}
+	return ret
+}
+
+const (
+	t1 = 0x00 // 0000 0000
+	tx = 0x80 // 1000 0000
+	t2 = 0xC0 // 1100 0000
+	t3 = 0xE0 // 1110 0000
+	t4 = 0xF0 // 1111 0000
+	t5 = 0xF8 // 1111 1000
+
+	maskx = 0x3F // 0011 1111
+	mask2 = 0x1F // 0001 1111
+	mask3 = 0x0F // 0000 1111
+	mask4 = 0x07 // 0000 0111
+
+	rune1Max = 1<<7 - 1
+	rune2Max = 1<<11 - 1
+	rune3Max = 1<<16 - 1
+
+	surrogateMin = 0xD800
+	surrogateMax = 0xDFFF
+
+	maxRune   = '\U0010FFFF' // Maximum valid Unicode code point.
+	runeError = '\uFFFD'     // the "error" Rune or "Unicode replacement character"
+)
+
+func appendRune(p []byte, r rune) []byte {
+	// Negative values are erroneous. Making it unsigned addresses the problem.
+	switch i := uint32(r); {
+	case i <= rune1Max:
+		p = append(p, byte(r))
+		return p
+	case i <= rune2Max:
+		p = append(p, t2|byte(r>>6))
+		p = append(p, tx|byte(r)&maskx)
+		return p
+	case i > maxRune, surrogateMin <= i && i <= surrogateMax:
+		r = runeError
+		fallthrough
+	case i <= rune3Max:
+		p = append(p, t3|byte(r>>12))
+		p = append(p, tx|byte(r>>6)&maskx)
+		p = append(p, tx|byte(r)&maskx)
+		return p
+	default:
+		p = append(p, t4|byte(r>>18))
+		p = append(p, tx|byte(r>>12)&maskx)
+		p = append(p, tx|byte(r>>6)&maskx)
+		p = append(p, tx|byte(r)&maskx)
+		return p
+	}
+}
diff --git a/vendor/github.com/json-iterator/go/jsoniter.go b/vendor/github.com/json-iterator/go/jsoniter.go
new file mode 100644
index 0000000..c2934f9
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/jsoniter.go
@@ -0,0 +1,18 @@
+// Package jsoniter implements encoding and decoding of JSON as defined in
+// RFC 4627 and provides interfaces with identical syntax of standard lib encoding/json.
+// Converting from encoding/json to jsoniter is no more than replacing the package with jsoniter
+// and variable type declarations (if any).
+// jsoniter interfaces gives 100% compatibility with code using standard lib.
+//
+// "JSON and Go"
+// (https://golang.org/doc/articles/json_and_go.html)
+// gives a description of how Marshal/Unmarshal operate
+// between arbitrary or predefined json objects and bytes,
+// and it applies to jsoniter.Marshal/Unmarshal as well.
+//
+// Besides, jsoniter.Iterator provides a different set of interfaces
+// iterating given bytes/string/reader
+// and yielding parsed elements one by one.
+// This set of interfaces reads input as required and gives
+// better performance.
+package jsoniter
diff --git a/vendor/github.com/json-iterator/go/pool.go b/vendor/github.com/json-iterator/go/pool.go
new file mode 100644
index 0000000..e2389b5
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/pool.go
@@ -0,0 +1,42 @@
+package jsoniter
+
+import (
+	"io"
+)
+
+// IteratorPool a thread safe pool of iterators with same configuration
+type IteratorPool interface {
+	BorrowIterator(data []byte) *Iterator
+	ReturnIterator(iter *Iterator)
+}
+
+// StreamPool a thread safe pool of streams with same configuration
+type StreamPool interface {
+	BorrowStream(writer io.Writer) *Stream
+	ReturnStream(stream *Stream)
+}
+
+func (cfg *frozenConfig) BorrowStream(writer io.Writer) *Stream {
+	stream := cfg.streamPool.Get().(*Stream)
+	stream.Reset(writer)
+	return stream
+}
+
+func (cfg *frozenConfig) ReturnStream(stream *Stream) {
+	stream.out = nil
+	stream.Error = nil
+	stream.Attachment = nil
+	cfg.streamPool.Put(stream)
+}
+
+func (cfg *frozenConfig) BorrowIterator(data []byte) *Iterator {
+	iter := cfg.iteratorPool.Get().(*Iterator)
+	iter.ResetBytes(data)
+	return iter
+}
+
+func (cfg *frozenConfig) ReturnIterator(iter *Iterator) {
+	iter.Error = nil
+	iter.Attachment = nil
+	cfg.iteratorPool.Put(iter)
+}
diff --git a/vendor/github.com/json-iterator/go/reflect.go b/vendor/github.com/json-iterator/go/reflect.go
new file mode 100644
index 0000000..4459e20
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/reflect.go
@@ -0,0 +1,332 @@
+package jsoniter
+
+import (
+	"fmt"
+	"reflect"
+	"unsafe"
+
+	"github.com/modern-go/reflect2"
+)
+
+// ValDecoder is an internal type registered to cache as needed.
+// Don't confuse jsoniter.ValDecoder with json.Decoder.
+// For json.Decoder's adapter, refer to jsoniter.AdapterDecoder(todo link).
+//
+// Reflection on type to create decoders, which is then cached
+// Reflection on value is avoided as we can, as the reflect.Value itself will allocate, with following exceptions
+// 1. create instance of new value, for example *int will need a int to be allocated
+// 2. append to slice, if the existing cap is not enough, allocate will be done using Reflect.New
+// 3. assignment to map, both key and value will be reflect.Value
+// For a simple struct binding, it will be reflect.Value free and allocation free
+type ValDecoder interface {
+	Decode(ptr unsafe.Pointer, iter *Iterator)
+}
+
+// ValEncoder is an internal type registered to cache as needed.
+// Don't confuse jsoniter.ValEncoder with json.Encoder.
+// For json.Encoder's adapter, refer to jsoniter.AdapterEncoder(todo godoc link).
+type ValEncoder interface {
+	IsEmpty(ptr unsafe.Pointer) bool
+	Encode(ptr unsafe.Pointer, stream *Stream)
+}
+
+type checkIsEmpty interface {
+	IsEmpty(ptr unsafe.Pointer) bool
+}
+
+type ctx struct {
+	*frozenConfig
+	prefix   string
+	encoders map[reflect2.Type]ValEncoder
+	decoders map[reflect2.Type]ValDecoder
+}
+
+func (b *ctx) caseSensitive() bool {
+	if b.frozenConfig == nil {
+		// default is case-insensitive
+		return false
+	}
+	return b.frozenConfig.caseSensitive
+}
+
+func (b *ctx) append(prefix string) *ctx {
+	return &ctx{
+		frozenConfig: b.frozenConfig,
+		prefix:       b.prefix + " " + prefix,
+		encoders:     b.encoders,
+		decoders:     b.decoders,
+	}
+}
+
+// ReadVal copy the underlying JSON into go interface, same as json.Unmarshal
+func (iter *Iterator) ReadVal(obj interface{}) {
+	cacheKey := reflect2.RTypeOf(obj)
+	decoder := iter.cfg.getDecoderFromCache(cacheKey)
+	if decoder == nil {
+		typ := reflect2.TypeOf(obj)
+		if typ.Kind() != reflect.Ptr {
+			iter.ReportError("ReadVal", "can only unmarshal into pointer")
+			return
+		}
+		decoder = iter.cfg.DecoderOf(typ)
+	}
+	ptr := reflect2.PtrOf(obj)
+	if ptr == nil {
+		iter.ReportError("ReadVal", "can not read into nil pointer")
+		return
+	}
+	decoder.Decode(ptr, iter)
+}
+
+// WriteVal copy the go interface into underlying JSON, same as json.Marshal
+func (stream *Stream) WriteVal(val interface{}) {
+	if nil == val {
+		stream.WriteNil()
+		return
+	}
+	cacheKey := reflect2.RTypeOf(val)
+	encoder := stream.cfg.getEncoderFromCache(cacheKey)
+	if encoder == nil {
+		typ := reflect2.TypeOf(val)
+		encoder = stream.cfg.EncoderOf(typ)
+	}
+	encoder.Encode(reflect2.PtrOf(val), stream)
+}
+
+func (cfg *frozenConfig) DecoderOf(typ reflect2.Type) ValDecoder {
+	cacheKey := typ.RType()
+	decoder := cfg.getDecoderFromCache(cacheKey)
+	if decoder != nil {
+		return decoder
+	}
+	ctx := &ctx{
+		frozenConfig: cfg,
+		prefix:       "",
+		decoders:     map[reflect2.Type]ValDecoder{},
+		encoders:     map[reflect2.Type]ValEncoder{},
+	}
+	ptrType := typ.(*reflect2.UnsafePtrType)
+	decoder = decoderOfType(ctx, ptrType.Elem())
+	cfg.addDecoderToCache(cacheKey, decoder)
+	return decoder
+}
+
+func decoderOfType(ctx *ctx, typ reflect2.Type) ValDecoder {
+	decoder := getTypeDecoderFromExtension(ctx, typ)
+	if decoder != nil {
+		return decoder
+	}
+	decoder = createDecoderOfType(ctx, typ)
+	for _, extension := range extensions {
+		decoder = extension.DecorateDecoder(typ, decoder)
+	}
+	decoder = ctx.decoderExtension.DecorateDecoder(typ, decoder)
+	for _, extension := range ctx.extraExtensions {
+		decoder = extension.DecorateDecoder(typ, decoder)
+	}
+	return decoder
+}
+
+func createDecoderOfType(ctx *ctx, typ reflect2.Type) ValDecoder {
+	decoder := ctx.decoders[typ]
+	if decoder != nil {
+		return decoder
+	}
+	placeholder := &placeholderDecoder{}
+	ctx.decoders[typ] = placeholder
+	decoder = _createDecoderOfType(ctx, typ)
+	placeholder.decoder = decoder
+	return decoder
+}
+
+func _createDecoderOfType(ctx *ctx, typ reflect2.Type) ValDecoder {
+	decoder := createDecoderOfJsonRawMessage(ctx, typ)
+	if decoder != nil {
+		return decoder
+	}
+	decoder = createDecoderOfJsonNumber(ctx, typ)
+	if decoder != nil {
+		return decoder
+	}
+	decoder = createDecoderOfMarshaler(ctx, typ)
+	if decoder != nil {
+		return decoder
+	}
+	decoder = createDecoderOfAny(ctx, typ)
+	if decoder != nil {
+		return decoder
+	}
+	decoder = createDecoderOfNative(ctx, typ)
+	if decoder != nil {
+		return decoder
+	}
+	switch typ.Kind() {
+	case reflect.Interface:
+		ifaceType, isIFace := typ.(*reflect2.UnsafeIFaceType)
+		if isIFace {
+			return &ifaceDecoder{valType: ifaceType}
+		}
+		return &efaceDecoder{}
+	case reflect.Struct:
+		return decoderOfStruct(ctx, typ)
+	case reflect.Array:
+		return decoderOfArray(ctx, typ)
+	case reflect.Slice:
+		return decoderOfSlice(ctx, typ)
+	case reflect.Map:
+		return decoderOfMap(ctx, typ)
+	case reflect.Ptr:
+		return decoderOfOptional(ctx, typ)
+	default:
+		return &lazyErrorDecoder{err: fmt.Errorf("%s%s is unsupported type", ctx.prefix, typ.String())}
+	}
+}
+
+func (cfg *frozenConfig) EncoderOf(typ reflect2.Type) ValEncoder {
+	cacheKey := typ.RType()
+	encoder := cfg.getEncoderFromCache(cacheKey)
+	if encoder != nil {
+		return encoder
+	}
+	ctx := &ctx{
+		frozenConfig: cfg,
+		prefix:       "",
+		decoders:     map[reflect2.Type]ValDecoder{},
+		encoders:     map[reflect2.Type]ValEncoder{},
+	}
+	encoder = encoderOfType(ctx, typ)
+	if typ.LikePtr() {
+		encoder = &onePtrEncoder{encoder}
+	}
+	cfg.addEncoderToCache(cacheKey, encoder)
+	return encoder
+}
+
+type onePtrEncoder struct {
+	encoder ValEncoder
+}
+
+func (encoder *onePtrEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return encoder.encoder.IsEmpty(unsafe.Pointer(&ptr))
+}
+
+func (encoder *onePtrEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	encoder.encoder.Encode(unsafe.Pointer(&ptr), stream)
+}
+
+func encoderOfType(ctx *ctx, typ reflect2.Type) ValEncoder {
+	encoder := getTypeEncoderFromExtension(ctx, typ)
+	if encoder != nil {
+		return encoder
+	}
+	encoder = createEncoderOfType(ctx, typ)
+	for _, extension := range extensions {
+		encoder = extension.DecorateEncoder(typ, encoder)
+	}
+	encoder = ctx.encoderExtension.DecorateEncoder(typ, encoder)
+	for _, extension := range ctx.extraExtensions {
+		encoder = extension.DecorateEncoder(typ, encoder)
+	}
+	return encoder
+}
+
+func createEncoderOfType(ctx *ctx, typ reflect2.Type) ValEncoder {
+	encoder := ctx.encoders[typ]
+	if encoder != nil {
+		return encoder
+	}
+	placeholder := &placeholderEncoder{}
+	ctx.encoders[typ] = placeholder
+	encoder = _createEncoderOfType(ctx, typ)
+	placeholder.encoder = encoder
+	return encoder
+}
+func _createEncoderOfType(ctx *ctx, typ reflect2.Type) ValEncoder {
+	encoder := createEncoderOfJsonRawMessage(ctx, typ)
+	if encoder != nil {
+		return encoder
+	}
+	encoder = createEncoderOfJsonNumber(ctx, typ)
+	if encoder != nil {
+		return encoder
+	}
+	encoder = createEncoderOfMarshaler(ctx, typ)
+	if encoder != nil {
+		return encoder
+	}
+	encoder = createEncoderOfAny(ctx, typ)
+	if encoder != nil {
+		return encoder
+	}
+	encoder = createEncoderOfNative(ctx, typ)
+	if encoder != nil {
+		return encoder
+	}
+	kind := typ.Kind()
+	switch kind {
+	case reflect.Interface:
+		return &dynamicEncoder{typ}
+	case reflect.Struct:
+		return encoderOfStruct(ctx, typ)
+	case reflect.Array:
+		return encoderOfArray(ctx, typ)
+	case reflect.Slice:
+		return encoderOfSlice(ctx, typ)
+	case reflect.Map:
+		return encoderOfMap(ctx, typ)
+	case reflect.Ptr:
+		return encoderOfOptional(ctx, typ)
+	default:
+		return &lazyErrorEncoder{err: fmt.Errorf("%s%s is unsupported type", ctx.prefix, typ.String())}
+	}
+}
+
+type lazyErrorDecoder struct {
+	err error
+}
+
+func (decoder *lazyErrorDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if iter.WhatIsNext() != NilValue {
+		if iter.Error == nil {
+			iter.Error = decoder.err
+		}
+	} else {
+		iter.Skip()
+	}
+}
+
+type lazyErrorEncoder struct {
+	err error
+}
+
+func (encoder *lazyErrorEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	if ptr == nil {
+		stream.WriteNil()
+	} else if stream.Error == nil {
+		stream.Error = encoder.err
+	}
+}
+
+func (encoder *lazyErrorEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return false
+}
+
+type placeholderDecoder struct {
+	decoder ValDecoder
+}
+
+func (decoder *placeholderDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	decoder.decoder.Decode(ptr, iter)
+}
+
+type placeholderEncoder struct {
+	encoder ValEncoder
+}
+
+func (encoder *placeholderEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	encoder.encoder.Encode(ptr, stream)
+}
+
+func (encoder *placeholderEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return encoder.encoder.IsEmpty(ptr)
+}
diff --git a/vendor/github.com/json-iterator/go/reflect_array.go b/vendor/github.com/json-iterator/go/reflect_array.go
new file mode 100644
index 0000000..13a0b7b
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/reflect_array.go
@@ -0,0 +1,104 @@
+package jsoniter
+
+import (
+	"fmt"
+	"github.com/modern-go/reflect2"
+	"io"
+	"unsafe"
+)
+
+func decoderOfArray(ctx *ctx, typ reflect2.Type) ValDecoder {
+	arrayType := typ.(*reflect2.UnsafeArrayType)
+	decoder := decoderOfType(ctx.append("[arrayElem]"), arrayType.Elem())
+	return &arrayDecoder{arrayType, decoder}
+}
+
+func encoderOfArray(ctx *ctx, typ reflect2.Type) ValEncoder {
+	arrayType := typ.(*reflect2.UnsafeArrayType)
+	if arrayType.Len() == 0 {
+		return emptyArrayEncoder{}
+	}
+	encoder := encoderOfType(ctx.append("[arrayElem]"), arrayType.Elem())
+	return &arrayEncoder{arrayType, encoder}
+}
+
+type emptyArrayEncoder struct{}
+
+func (encoder emptyArrayEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteEmptyArray()
+}
+
+func (encoder emptyArrayEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return true
+}
+
+type arrayEncoder struct {
+	arrayType   *reflect2.UnsafeArrayType
+	elemEncoder ValEncoder
+}
+
+func (encoder *arrayEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteArrayStart()
+	elemPtr := unsafe.Pointer(ptr)
+	encoder.elemEncoder.Encode(elemPtr, stream)
+	for i := 1; i < encoder.arrayType.Len(); i++ {
+		stream.WriteMore()
+		elemPtr = encoder.arrayType.UnsafeGetIndex(ptr, i)
+		encoder.elemEncoder.Encode(elemPtr, stream)
+	}
+	stream.WriteArrayEnd()
+	if stream.Error != nil && stream.Error != io.EOF {
+		stream.Error = fmt.Errorf("%v: %s", encoder.arrayType, stream.Error.Error())
+	}
+}
+
+func (encoder *arrayEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return false
+}
+
+type arrayDecoder struct {
+	arrayType   *reflect2.UnsafeArrayType
+	elemDecoder ValDecoder
+}
+
+func (decoder *arrayDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	decoder.doDecode(ptr, iter)
+	if iter.Error != nil && iter.Error != io.EOF {
+		iter.Error = fmt.Errorf("%v: %s", decoder.arrayType, iter.Error.Error())
+	}
+}
+
+func (decoder *arrayDecoder) doDecode(ptr unsafe.Pointer, iter *Iterator) {
+	c := iter.nextToken()
+	arrayType := decoder.arrayType
+	if c == 'n' {
+		iter.skipThreeBytes('u', 'l', 'l')
+		return
+	}
+	if c != '[' {
+		iter.ReportError("decode array", "expect [ or n, but found "+string([]byte{c}))
+		return
+	}
+	c = iter.nextToken()
+	if c == ']' {
+		return
+	}
+	iter.unreadByte()
+	elemPtr := arrayType.UnsafeGetIndex(ptr, 0)
+	decoder.elemDecoder.Decode(elemPtr, iter)
+	length := 1
+	for c = iter.nextToken(); c == ','; c = iter.nextToken() {
+		if length >= arrayType.Len() {
+			iter.Skip()
+			continue
+		}
+		idx := length
+		length += 1
+		elemPtr = arrayType.UnsafeGetIndex(ptr, idx)
+		decoder.elemDecoder.Decode(elemPtr, iter)
+	}
+	if c != ']' {
+		iter.ReportError("decode array", "expect ], but found "+string([]byte{c}))
+		return
+	}
+}
diff --git a/vendor/github.com/json-iterator/go/reflect_dynamic.go b/vendor/github.com/json-iterator/go/reflect_dynamic.go
new file mode 100644
index 0000000..8b6bc8b
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/reflect_dynamic.go
@@ -0,0 +1,70 @@
+package jsoniter
+
+import (
+	"github.com/modern-go/reflect2"
+	"reflect"
+	"unsafe"
+)
+
+type dynamicEncoder struct {
+	valType reflect2.Type
+}
+
+func (encoder *dynamicEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	obj := encoder.valType.UnsafeIndirect(ptr)
+	stream.WriteVal(obj)
+}
+
+func (encoder *dynamicEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return encoder.valType.UnsafeIndirect(ptr) == nil
+}
+
+type efaceDecoder struct {
+}
+
+func (decoder *efaceDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	pObj := (*interface{})(ptr)
+	obj := *pObj
+	if obj == nil {
+		*pObj = iter.Read()
+		return
+	}
+	typ := reflect2.TypeOf(obj)
+	if typ.Kind() != reflect.Ptr {
+		*pObj = iter.Read()
+		return
+	}
+	ptrType := typ.(*reflect2.UnsafePtrType)
+	ptrElemType := ptrType.Elem()
+	if iter.WhatIsNext() == NilValue {
+		if ptrElemType.Kind() != reflect.Ptr {
+			iter.skipFourBytes('n', 'u', 'l', 'l')
+			*pObj = nil
+			return
+		}
+	}
+	if reflect2.IsNil(obj) {
+		obj := ptrElemType.New()
+		iter.ReadVal(obj)
+		*pObj = obj
+		return
+	}
+	iter.ReadVal(obj)
+}
+
+type ifaceDecoder struct {
+	valType *reflect2.UnsafeIFaceType
+}
+
+func (decoder *ifaceDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if iter.ReadNil() {
+		decoder.valType.UnsafeSet(ptr, decoder.valType.UnsafeNew())
+		return
+	}
+	obj := decoder.valType.UnsafeIndirect(ptr)
+	if reflect2.IsNil(obj) {
+		iter.ReportError("decode non empty interface", "can not unmarshal into nil")
+		return
+	}
+	iter.ReadVal(obj)
+}
diff --git a/vendor/github.com/json-iterator/go/reflect_extension.go b/vendor/github.com/json-iterator/go/reflect_extension.go
new file mode 100644
index 0000000..05e8fbf
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/reflect_extension.go
@@ -0,0 +1,483 @@
+package jsoniter
+
+import (
+	"fmt"
+	"github.com/modern-go/reflect2"
+	"reflect"
+	"sort"
+	"strings"
+	"unicode"
+	"unsafe"
+)
+
+var typeDecoders = map[string]ValDecoder{}
+var fieldDecoders = map[string]ValDecoder{}
+var typeEncoders = map[string]ValEncoder{}
+var fieldEncoders = map[string]ValEncoder{}
+var extensions = []Extension{}
+
+// StructDescriptor describe how should we encode/decode the struct
+type StructDescriptor struct {
+	Type   reflect2.Type
+	Fields []*Binding
+}
+
+// GetField get one field from the descriptor by its name.
+// Can not use map here to keep field orders.
+func (structDescriptor *StructDescriptor) GetField(fieldName string) *Binding {
+	for _, binding := range structDescriptor.Fields {
+		if binding.Field.Name() == fieldName {
+			return binding
+		}
+	}
+	return nil
+}
+
+// Binding describe how should we encode/decode the struct field
+type Binding struct {
+	levels    []int
+	Field     reflect2.StructField
+	FromNames []string
+	ToNames   []string
+	Encoder   ValEncoder
+	Decoder   ValDecoder
+}
+
+// Extension the one for all SPI. Customize encoding/decoding by specifying alternate encoder/decoder.
+// Can also rename fields by UpdateStructDescriptor.
+type Extension interface {
+	UpdateStructDescriptor(structDescriptor *StructDescriptor)
+	CreateMapKeyDecoder(typ reflect2.Type) ValDecoder
+	CreateMapKeyEncoder(typ reflect2.Type) ValEncoder
+	CreateDecoder(typ reflect2.Type) ValDecoder
+	CreateEncoder(typ reflect2.Type) ValEncoder
+	DecorateDecoder(typ reflect2.Type, decoder ValDecoder) ValDecoder
+	DecorateEncoder(typ reflect2.Type, encoder ValEncoder) ValEncoder
+}
+
+// DummyExtension embed this type get dummy implementation for all methods of Extension
+type DummyExtension struct {
+}
+
+// UpdateStructDescriptor No-op
+func (extension *DummyExtension) UpdateStructDescriptor(structDescriptor *StructDescriptor) {
+}
+
+// CreateMapKeyDecoder No-op
+func (extension *DummyExtension) CreateMapKeyDecoder(typ reflect2.Type) ValDecoder {
+	return nil
+}
+
+// CreateMapKeyEncoder No-op
+func (extension *DummyExtension) CreateMapKeyEncoder(typ reflect2.Type) ValEncoder {
+	return nil
+}
+
+// CreateDecoder No-op
+func (extension *DummyExtension) CreateDecoder(typ reflect2.Type) ValDecoder {
+	return nil
+}
+
+// CreateEncoder No-op
+func (extension *DummyExtension) CreateEncoder(typ reflect2.Type) ValEncoder {
+	return nil
+}
+
+// DecorateDecoder No-op
+func (extension *DummyExtension) DecorateDecoder(typ reflect2.Type, decoder ValDecoder) ValDecoder {
+	return decoder
+}
+
+// DecorateEncoder No-op
+func (extension *DummyExtension) DecorateEncoder(typ reflect2.Type, encoder ValEncoder) ValEncoder {
+	return encoder
+}
+
+type EncoderExtension map[reflect2.Type]ValEncoder
+
+// UpdateStructDescriptor No-op
+func (extension EncoderExtension) UpdateStructDescriptor(structDescriptor *StructDescriptor) {
+}
+
+// CreateDecoder No-op
+func (extension EncoderExtension) CreateDecoder(typ reflect2.Type) ValDecoder {
+	return nil
+}
+
+// CreateEncoder get encoder from map
+func (extension EncoderExtension) CreateEncoder(typ reflect2.Type) ValEncoder {
+	return extension[typ]
+}
+
+// CreateMapKeyDecoder No-op
+func (extension EncoderExtension) CreateMapKeyDecoder(typ reflect2.Type) ValDecoder {
+	return nil
+}
+
+// CreateMapKeyEncoder No-op
+func (extension EncoderExtension) CreateMapKeyEncoder(typ reflect2.Type) ValEncoder {
+	return nil
+}
+
+// DecorateDecoder No-op
+func (extension EncoderExtension) DecorateDecoder(typ reflect2.Type, decoder ValDecoder) ValDecoder {
+	return decoder
+}
+
+// DecorateEncoder No-op
+func (extension EncoderExtension) DecorateEncoder(typ reflect2.Type, encoder ValEncoder) ValEncoder {
+	return encoder
+}
+
+type DecoderExtension map[reflect2.Type]ValDecoder
+
+// UpdateStructDescriptor No-op
+func (extension DecoderExtension) UpdateStructDescriptor(structDescriptor *StructDescriptor) {
+}
+
+// CreateMapKeyDecoder No-op
+func (extension DecoderExtension) CreateMapKeyDecoder(typ reflect2.Type) ValDecoder {
+	return nil
+}
+
+// CreateMapKeyEncoder No-op
+func (extension DecoderExtension) CreateMapKeyEncoder(typ reflect2.Type) ValEncoder {
+	return nil
+}
+
+// CreateDecoder get decoder from map
+func (extension DecoderExtension) CreateDecoder(typ reflect2.Type) ValDecoder {
+	return extension[typ]
+}
+
+// CreateEncoder No-op
+func (extension DecoderExtension) CreateEncoder(typ reflect2.Type) ValEncoder {
+	return nil
+}
+
+// DecorateDecoder No-op
+func (extension DecoderExtension) DecorateDecoder(typ reflect2.Type, decoder ValDecoder) ValDecoder {
+	return decoder
+}
+
+// DecorateEncoder No-op
+func (extension DecoderExtension) DecorateEncoder(typ reflect2.Type, encoder ValEncoder) ValEncoder {
+	return encoder
+}
+
+type funcDecoder struct {
+	fun DecoderFunc
+}
+
+func (decoder *funcDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	decoder.fun(ptr, iter)
+}
+
+type funcEncoder struct {
+	fun         EncoderFunc
+	isEmptyFunc func(ptr unsafe.Pointer) bool
+}
+
+func (encoder *funcEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	encoder.fun(ptr, stream)
+}
+
+func (encoder *funcEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	if encoder.isEmptyFunc == nil {
+		return false
+	}
+	return encoder.isEmptyFunc(ptr)
+}
+
+// DecoderFunc the function form of TypeDecoder
+type DecoderFunc func(ptr unsafe.Pointer, iter *Iterator)
+
+// EncoderFunc the function form of TypeEncoder
+type EncoderFunc func(ptr unsafe.Pointer, stream *Stream)
+
+// RegisterTypeDecoderFunc register TypeDecoder for a type with function
+func RegisterTypeDecoderFunc(typ string, fun DecoderFunc) {
+	typeDecoders[typ] = &funcDecoder{fun}
+}
+
+// RegisterTypeDecoder register TypeDecoder for a typ
+func RegisterTypeDecoder(typ string, decoder ValDecoder) {
+	typeDecoders[typ] = decoder
+}
+
+// RegisterFieldDecoderFunc register TypeDecoder for a struct field with function
+func RegisterFieldDecoderFunc(typ string, field string, fun DecoderFunc) {
+	RegisterFieldDecoder(typ, field, &funcDecoder{fun})
+}
+
+// RegisterFieldDecoder register TypeDecoder for a struct field
+func RegisterFieldDecoder(typ string, field string, decoder ValDecoder) {
+	fieldDecoders[fmt.Sprintf("%s/%s", typ, field)] = decoder
+}
+
+// RegisterTypeEncoderFunc register TypeEncoder for a type with encode/isEmpty function
+func RegisterTypeEncoderFunc(typ string, fun EncoderFunc, isEmptyFunc func(unsafe.Pointer) bool) {
+	typeEncoders[typ] = &funcEncoder{fun, isEmptyFunc}
+}
+
+// RegisterTypeEncoder register TypeEncoder for a type
+func RegisterTypeEncoder(typ string, encoder ValEncoder) {
+	typeEncoders[typ] = encoder
+}
+
+// RegisterFieldEncoderFunc register TypeEncoder for a struct field with encode/isEmpty function
+func RegisterFieldEncoderFunc(typ string, field string, fun EncoderFunc, isEmptyFunc func(unsafe.Pointer) bool) {
+	RegisterFieldEncoder(typ, field, &funcEncoder{fun, isEmptyFunc})
+}
+
+// RegisterFieldEncoder register TypeEncoder for a struct field
+func RegisterFieldEncoder(typ string, field string, encoder ValEncoder) {
+	fieldEncoders[fmt.Sprintf("%s/%s", typ, field)] = encoder
+}
+
+// RegisterExtension register extension
+func RegisterExtension(extension Extension) {
+	extensions = append(extensions, extension)
+}
+
+func getTypeDecoderFromExtension(ctx *ctx, typ reflect2.Type) ValDecoder {
+	decoder := _getTypeDecoderFromExtension(ctx, typ)
+	if decoder != nil {
+		for _, extension := range extensions {
+			decoder = extension.DecorateDecoder(typ, decoder)
+		}
+		decoder = ctx.decoderExtension.DecorateDecoder(typ, decoder)
+		for _, extension := range ctx.extraExtensions {
+			decoder = extension.DecorateDecoder(typ, decoder)
+		}
+	}
+	return decoder
+}
+func _getTypeDecoderFromExtension(ctx *ctx, typ reflect2.Type) ValDecoder {
+	for _, extension := range extensions {
+		decoder := extension.CreateDecoder(typ)
+		if decoder != nil {
+			return decoder
+		}
+	}
+	decoder := ctx.decoderExtension.CreateDecoder(typ)
+	if decoder != nil {
+		return decoder
+	}
+	for _, extension := range ctx.extraExtensions {
+		decoder := extension.CreateDecoder(typ)
+		if decoder != nil {
+			return decoder
+		}
+	}
+	typeName := typ.String()
+	decoder = typeDecoders[typeName]
+	if decoder != nil {
+		return decoder
+	}
+	if typ.Kind() == reflect.Ptr {
+		ptrType := typ.(*reflect2.UnsafePtrType)
+		decoder := typeDecoders[ptrType.Elem().String()]
+		if decoder != nil {
+			return &OptionalDecoder{ptrType.Elem(), decoder}
+		}
+	}
+	return nil
+}
+
+func getTypeEncoderFromExtension(ctx *ctx, typ reflect2.Type) ValEncoder {
+	encoder := _getTypeEncoderFromExtension(ctx, typ)
+	if encoder != nil {
+		for _, extension := range extensions {
+			encoder = extension.DecorateEncoder(typ, encoder)
+		}
+		encoder = ctx.encoderExtension.DecorateEncoder(typ, encoder)
+		for _, extension := range ctx.extraExtensions {
+			encoder = extension.DecorateEncoder(typ, encoder)
+		}
+	}
+	return encoder
+}
+
+func _getTypeEncoderFromExtension(ctx *ctx, typ reflect2.Type) ValEncoder {
+	for _, extension := range extensions {
+		encoder := extension.CreateEncoder(typ)
+		if encoder != nil {
+			return encoder
+		}
+	}
+	encoder := ctx.encoderExtension.CreateEncoder(typ)
+	if encoder != nil {
+		return encoder
+	}
+	for _, extension := range ctx.extraExtensions {
+		encoder := extension.CreateEncoder(typ)
+		if encoder != nil {
+			return encoder
+		}
+	}
+	typeName := typ.String()
+	encoder = typeEncoders[typeName]
+	if encoder != nil {
+		return encoder
+	}
+	if typ.Kind() == reflect.Ptr {
+		typePtr := typ.(*reflect2.UnsafePtrType)
+		encoder := typeEncoders[typePtr.Elem().String()]
+		if encoder != nil {
+			return &OptionalEncoder{encoder}
+		}
+	}
+	return nil
+}
+
+func describeStruct(ctx *ctx, typ reflect2.Type) *StructDescriptor {
+	structType := typ.(*reflect2.UnsafeStructType)
+	embeddedBindings := []*Binding{}
+	bindings := []*Binding{}
+	for i := 0; i < structType.NumField(); i++ {
+		field := structType.Field(i)
+		tag, hastag := field.Tag().Lookup(ctx.getTagKey())
+		if ctx.onlyTaggedField && !hastag && !field.Anonymous() {
+			continue
+		}
+		tagParts := strings.Split(tag, ",")
+		if tag == "-" {
+			continue
+		}
+		if field.Anonymous() && (tag == "" || tagParts[0] == "") {
+			if field.Type().Kind() == reflect.Struct {
+				structDescriptor := describeStruct(ctx, field.Type())
+				for _, binding := range structDescriptor.Fields {
+					binding.levels = append([]int{i}, binding.levels...)
+					omitempty := binding.Encoder.(*structFieldEncoder).omitempty
+					binding.Encoder = &structFieldEncoder{field, binding.Encoder, omitempty}
+					binding.Decoder = &structFieldDecoder{field, binding.Decoder}
+					embeddedBindings = append(embeddedBindings, binding)
+				}
+				continue
+			} else if field.Type().Kind() == reflect.Ptr {
+				ptrType := field.Type().(*reflect2.UnsafePtrType)
+				if ptrType.Elem().Kind() == reflect.Struct {
+					structDescriptor := describeStruct(ctx, ptrType.Elem())
+					for _, binding := range structDescriptor.Fields {
+						binding.levels = append([]int{i}, binding.levels...)
+						omitempty := binding.Encoder.(*structFieldEncoder).omitempty
+						binding.Encoder = &dereferenceEncoder{binding.Encoder}
+						binding.Encoder = &structFieldEncoder{field, binding.Encoder, omitempty}
+						binding.Decoder = &dereferenceDecoder{ptrType.Elem(), binding.Decoder}
+						binding.Decoder = &structFieldDecoder{field, binding.Decoder}
+						embeddedBindings = append(embeddedBindings, binding)
+					}
+					continue
+				}
+			}
+		}
+		fieldNames := calcFieldNames(field.Name(), tagParts[0], tag)
+		fieldCacheKey := fmt.Sprintf("%s/%s", typ.String(), field.Name())
+		decoder := fieldDecoders[fieldCacheKey]
+		if decoder == nil {
+			decoder = decoderOfType(ctx.append(field.Name()), field.Type())
+		}
+		encoder := fieldEncoders[fieldCacheKey]
+		if encoder == nil {
+			encoder = encoderOfType(ctx.append(field.Name()), field.Type())
+		}
+		binding := &Binding{
+			Field:     field,
+			FromNames: fieldNames,
+			ToNames:   fieldNames,
+			Decoder:   decoder,
+			Encoder:   encoder,
+		}
+		binding.levels = []int{i}
+		bindings = append(bindings, binding)
+	}
+	return createStructDescriptor(ctx, typ, bindings, embeddedBindings)
+}
+func createStructDescriptor(ctx *ctx, typ reflect2.Type, bindings []*Binding, embeddedBindings []*Binding) *StructDescriptor {
+	structDescriptor := &StructDescriptor{
+		Type:   typ,
+		Fields: bindings,
+	}
+	for _, extension := range extensions {
+		extension.UpdateStructDescriptor(structDescriptor)
+	}
+	ctx.encoderExtension.UpdateStructDescriptor(structDescriptor)
+	ctx.decoderExtension.UpdateStructDescriptor(structDescriptor)
+	for _, extension := range ctx.extraExtensions {
+		extension.UpdateStructDescriptor(structDescriptor)
+	}
+	processTags(structDescriptor, ctx.frozenConfig)
+	// merge normal & embedded bindings & sort with original order
+	allBindings := sortableBindings(append(embeddedBindings, structDescriptor.Fields...))
+	sort.Sort(allBindings)
+	structDescriptor.Fields = allBindings
+	return structDescriptor
+}
+
+type sortableBindings []*Binding
+
+func (bindings sortableBindings) Len() int {
+	return len(bindings)
+}
+
+func (bindings sortableBindings) Less(i, j int) bool {
+	left := bindings[i].levels
+	right := bindings[j].levels
+	k := 0
+	for {
+		if left[k] < right[k] {
+			return true
+		} else if left[k] > right[k] {
+			return false
+		}
+		k++
+	}
+}
+
+func (bindings sortableBindings) Swap(i, j int) {
+	bindings[i], bindings[j] = bindings[j], bindings[i]
+}
+
+func processTags(structDescriptor *StructDescriptor, cfg *frozenConfig) {
+	for _, binding := range structDescriptor.Fields {
+		shouldOmitEmpty := false
+		tagParts := strings.Split(binding.Field.Tag().Get(cfg.getTagKey()), ",")
+		for _, tagPart := range tagParts[1:] {
+			if tagPart == "omitempty" {
+				shouldOmitEmpty = true
+			} else if tagPart == "string" {
+				if binding.Field.Type().Kind() == reflect.String {
+					binding.Decoder = &stringModeStringDecoder{binding.Decoder, cfg}
+					binding.Encoder = &stringModeStringEncoder{binding.Encoder, cfg}
+				} else {
+					binding.Decoder = &stringModeNumberDecoder{binding.Decoder}
+					binding.Encoder = &stringModeNumberEncoder{binding.Encoder}
+				}
+			}
+		}
+		binding.Decoder = &structFieldDecoder{binding.Field, binding.Decoder}
+		binding.Encoder = &structFieldEncoder{binding.Field, binding.Encoder, shouldOmitEmpty}
+	}
+}
+
+func calcFieldNames(originalFieldName string, tagProvidedFieldName string, wholeTag string) []string {
+	// ignore?
+	if wholeTag == "-" {
+		return []string{}
+	}
+	// rename?
+	var fieldNames []string
+	if tagProvidedFieldName == "" {
+		fieldNames = []string{originalFieldName}
+	} else {
+		fieldNames = []string{tagProvidedFieldName}
+	}
+	// private?
+	isNotExported := unicode.IsLower(rune(originalFieldName[0]))
+	if isNotExported {
+		fieldNames = []string{}
+	}
+	return fieldNames
+}
diff --git a/vendor/github.com/json-iterator/go/reflect_json_number.go b/vendor/github.com/json-iterator/go/reflect_json_number.go
new file mode 100644
index 0000000..98d45c1
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/reflect_json_number.go
@@ -0,0 +1,112 @@
+package jsoniter
+
+import (
+	"encoding/json"
+	"github.com/modern-go/reflect2"
+	"strconv"
+	"unsafe"
+)
+
+type Number string
+
+// String returns the literal text of the number.
+func (n Number) String() string { return string(n) }
+
+// Float64 returns the number as a float64.
+func (n Number) Float64() (float64, error) {
+	return strconv.ParseFloat(string(n), 64)
+}
+
+// Int64 returns the number as an int64.
+func (n Number) Int64() (int64, error) {
+	return strconv.ParseInt(string(n), 10, 64)
+}
+
+func CastJsonNumber(val interface{}) (string, bool) {
+	switch typedVal := val.(type) {
+	case json.Number:
+		return string(typedVal), true
+	case Number:
+		return string(typedVal), true
+	}
+	return "", false
+}
+
+var jsonNumberType = reflect2.TypeOfPtr((*json.Number)(nil)).Elem()
+var jsoniterNumberType = reflect2.TypeOfPtr((*Number)(nil)).Elem()
+
+func createDecoderOfJsonNumber(ctx *ctx, typ reflect2.Type) ValDecoder {
+	if typ.AssignableTo(jsonNumberType) {
+		return &jsonNumberCodec{}
+	}
+	if typ.AssignableTo(jsoniterNumberType) {
+		return &jsoniterNumberCodec{}
+	}
+	return nil
+}
+
+func createEncoderOfJsonNumber(ctx *ctx, typ reflect2.Type) ValEncoder {
+	if typ.AssignableTo(jsonNumberType) {
+		return &jsonNumberCodec{}
+	}
+	if typ.AssignableTo(jsoniterNumberType) {
+		return &jsoniterNumberCodec{}
+	}
+	return nil
+}
+
+type jsonNumberCodec struct {
+}
+
+func (codec *jsonNumberCodec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	switch iter.WhatIsNext() {
+	case StringValue:
+		*((*json.Number)(ptr)) = json.Number(iter.ReadString())
+	case NilValue:
+		iter.skipFourBytes('n', 'u', 'l', 'l')
+		*((*json.Number)(ptr)) = ""
+	default:
+		*((*json.Number)(ptr)) = json.Number([]byte(iter.readNumberAsString()))
+	}
+}
+
+func (codec *jsonNumberCodec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	number := *((*json.Number)(ptr))
+	if len(number) == 0 {
+		stream.writeByte('0')
+	} else {
+		stream.WriteRaw(string(number))
+	}
+}
+
+func (codec *jsonNumberCodec) IsEmpty(ptr unsafe.Pointer) bool {
+	return len(*((*json.Number)(ptr))) == 0
+}
+
+type jsoniterNumberCodec struct {
+}
+
+func (codec *jsoniterNumberCodec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	switch iter.WhatIsNext() {
+	case StringValue:
+		*((*Number)(ptr)) = Number(iter.ReadString())
+	case NilValue:
+		iter.skipFourBytes('n', 'u', 'l', 'l')
+		*((*Number)(ptr)) = ""
+	default:
+		*((*Number)(ptr)) = Number([]byte(iter.readNumberAsString()))
+	}
+}
+
+func (codec *jsoniterNumberCodec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	number := *((*Number)(ptr))
+	if len(number) == 0 {
+		stream.writeByte('0')
+	} else {
+		stream.WriteRaw(string(number))
+	}
+}
+
+func (codec *jsoniterNumberCodec) IsEmpty(ptr unsafe.Pointer) bool {
+	return len(*((*Number)(ptr))) == 0
+}
diff --git a/vendor/github.com/json-iterator/go/reflect_json_raw_message.go b/vendor/github.com/json-iterator/go/reflect_json_raw_message.go
new file mode 100644
index 0000000..f261993
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/reflect_json_raw_message.go
@@ -0,0 +1,60 @@
+package jsoniter
+
+import (
+	"encoding/json"
+	"github.com/modern-go/reflect2"
+	"unsafe"
+)
+
+var jsonRawMessageType = reflect2.TypeOfPtr((*json.RawMessage)(nil)).Elem()
+var jsoniterRawMessageType = reflect2.TypeOfPtr((*RawMessage)(nil)).Elem()
+
+func createEncoderOfJsonRawMessage(ctx *ctx, typ reflect2.Type) ValEncoder {
+	if typ == jsonRawMessageType {
+		return &jsonRawMessageCodec{}
+	}
+	if typ == jsoniterRawMessageType {
+		return &jsoniterRawMessageCodec{}
+	}
+	return nil
+}
+
+func createDecoderOfJsonRawMessage(ctx *ctx, typ reflect2.Type) ValDecoder {
+	if typ == jsonRawMessageType {
+		return &jsonRawMessageCodec{}
+	}
+	if typ == jsoniterRawMessageType {
+		return &jsoniterRawMessageCodec{}
+	}
+	return nil
+}
+
+type jsonRawMessageCodec struct {
+}
+
+func (codec *jsonRawMessageCodec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	*((*json.RawMessage)(ptr)) = json.RawMessage(iter.SkipAndReturnBytes())
+}
+
+func (codec *jsonRawMessageCodec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteRaw(string(*((*json.RawMessage)(ptr))))
+}
+
+func (codec *jsonRawMessageCodec) IsEmpty(ptr unsafe.Pointer) bool {
+	return len(*((*json.RawMessage)(ptr))) == 0
+}
+
+type jsoniterRawMessageCodec struct {
+}
+
+func (codec *jsoniterRawMessageCodec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	*((*RawMessage)(ptr)) = RawMessage(iter.SkipAndReturnBytes())
+}
+
+func (codec *jsoniterRawMessageCodec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteRaw(string(*((*RawMessage)(ptr))))
+}
+
+func (codec *jsoniterRawMessageCodec) IsEmpty(ptr unsafe.Pointer) bool {
+	return len(*((*RawMessage)(ptr))) == 0
+}
diff --git a/vendor/github.com/json-iterator/go/reflect_map.go b/vendor/github.com/json-iterator/go/reflect_map.go
new file mode 100644
index 0000000..547b442
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/reflect_map.go
@@ -0,0 +1,338 @@
+package jsoniter
+
+import (
+	"fmt"
+	"github.com/modern-go/reflect2"
+	"io"
+	"reflect"
+	"sort"
+	"unsafe"
+)
+
+func decoderOfMap(ctx *ctx, typ reflect2.Type) ValDecoder {
+	mapType := typ.(*reflect2.UnsafeMapType)
+	keyDecoder := decoderOfMapKey(ctx.append("[mapKey]"), mapType.Key())
+	elemDecoder := decoderOfType(ctx.append("[mapElem]"), mapType.Elem())
+	return &mapDecoder{
+		mapType:     mapType,
+		keyType:     mapType.Key(),
+		elemType:    mapType.Elem(),
+		keyDecoder:  keyDecoder,
+		elemDecoder: elemDecoder,
+	}
+}
+
+func encoderOfMap(ctx *ctx, typ reflect2.Type) ValEncoder {
+	mapType := typ.(*reflect2.UnsafeMapType)
+	if ctx.sortMapKeys {
+		return &sortKeysMapEncoder{
+			mapType:     mapType,
+			keyEncoder:  encoderOfMapKey(ctx.append("[mapKey]"), mapType.Key()),
+			elemEncoder: encoderOfType(ctx.append("[mapElem]"), mapType.Elem()),
+		}
+	}
+	return &mapEncoder{
+		mapType:     mapType,
+		keyEncoder:  encoderOfMapKey(ctx.append("[mapKey]"), mapType.Key()),
+		elemEncoder: encoderOfType(ctx.append("[mapElem]"), mapType.Elem()),
+	}
+}
+
+func decoderOfMapKey(ctx *ctx, typ reflect2.Type) ValDecoder {
+	decoder := ctx.decoderExtension.CreateMapKeyDecoder(typ)
+	if decoder != nil {
+		return decoder
+	}
+	for _, extension := range ctx.extraExtensions {
+		decoder := extension.CreateMapKeyDecoder(typ)
+		if decoder != nil {
+			return decoder
+		}
+	}
+	switch typ.Kind() {
+	case reflect.String:
+		return decoderOfType(ctx, reflect2.DefaultTypeOfKind(reflect.String))
+	case reflect.Bool,
+		reflect.Uint8, reflect.Int8,
+		reflect.Uint16, reflect.Int16,
+		reflect.Uint32, reflect.Int32,
+		reflect.Uint64, reflect.Int64,
+		reflect.Uint, reflect.Int,
+		reflect.Float32, reflect.Float64,
+		reflect.Uintptr:
+		typ = reflect2.DefaultTypeOfKind(typ.Kind())
+		return &numericMapKeyDecoder{decoderOfType(ctx, typ)}
+	default:
+		ptrType := reflect2.PtrTo(typ)
+		if ptrType.Implements(unmarshalerType) {
+			return &referenceDecoder{
+				&unmarshalerDecoder{
+					valType: ptrType,
+				},
+			}
+		}
+		if typ.Implements(unmarshalerType) {
+			return &unmarshalerDecoder{
+				valType: typ,
+			}
+		}
+		if ptrType.Implements(textUnmarshalerType) {
+			return &referenceDecoder{
+				&textUnmarshalerDecoder{
+					valType: ptrType,
+				},
+			}
+		}
+		if typ.Implements(textUnmarshalerType) {
+			return &textUnmarshalerDecoder{
+				valType: typ,
+			}
+		}
+		return &lazyErrorDecoder{err: fmt.Errorf("unsupported map key type: %v", typ)}
+	}
+}
+
+func encoderOfMapKey(ctx *ctx, typ reflect2.Type) ValEncoder {
+	encoder := ctx.encoderExtension.CreateMapKeyEncoder(typ)
+	if encoder != nil {
+		return encoder
+	}
+	for _, extension := range ctx.extraExtensions {
+		encoder := extension.CreateMapKeyEncoder(typ)
+		if encoder != nil {
+			return encoder
+		}
+	}
+	switch typ.Kind() {
+	case reflect.String:
+		return encoderOfType(ctx, reflect2.DefaultTypeOfKind(reflect.String))
+	case reflect.Bool,
+		reflect.Uint8, reflect.Int8,
+		reflect.Uint16, reflect.Int16,
+		reflect.Uint32, reflect.Int32,
+		reflect.Uint64, reflect.Int64,
+		reflect.Uint, reflect.Int,
+		reflect.Float32, reflect.Float64,
+		reflect.Uintptr:
+		typ = reflect2.DefaultTypeOfKind(typ.Kind())
+		return &numericMapKeyEncoder{encoderOfType(ctx, typ)}
+	default:
+		if typ == textMarshalerType {
+			return &directTextMarshalerEncoder{
+				stringEncoder: ctx.EncoderOf(reflect2.TypeOf("")),
+			}
+		}
+		if typ.Implements(textMarshalerType) {
+			return &textMarshalerEncoder{
+				valType:       typ,
+				stringEncoder: ctx.EncoderOf(reflect2.TypeOf("")),
+			}
+		}
+		if typ.Kind() == reflect.Interface {
+			return &dynamicMapKeyEncoder{ctx, typ}
+		}
+		return &lazyErrorEncoder{err: fmt.Errorf("unsupported map key type: %v", typ)}
+	}
+}
+
+type mapDecoder struct {
+	mapType     *reflect2.UnsafeMapType
+	keyType     reflect2.Type
+	elemType    reflect2.Type
+	keyDecoder  ValDecoder
+	elemDecoder ValDecoder
+}
+
+func (decoder *mapDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	mapType := decoder.mapType
+	c := iter.nextToken()
+	if c == 'n' {
+		iter.skipThreeBytes('u', 'l', 'l')
+		*(*unsafe.Pointer)(ptr) = nil
+		mapType.UnsafeSet(ptr, mapType.UnsafeNew())
+		return
+	}
+	if mapType.UnsafeIsNil(ptr) {
+		mapType.UnsafeSet(ptr, mapType.UnsafeMakeMap(0))
+	}
+	if c != '{' {
+		iter.ReportError("ReadMapCB", `expect { or n, but found `+string([]byte{c}))
+		return
+	}
+	c = iter.nextToken()
+	if c == '}' {
+		return
+	}
+	if c != '"' {
+		iter.ReportError("ReadMapCB", `expect " after }, but found `+string([]byte{c}))
+		return
+	}
+	iter.unreadByte()
+	key := decoder.keyType.UnsafeNew()
+	decoder.keyDecoder.Decode(key, iter)
+	c = iter.nextToken()
+	if c != ':' {
+		iter.ReportError("ReadMapCB", "expect : after object field, but found "+string([]byte{c}))
+		return
+	}
+	elem := decoder.elemType.UnsafeNew()
+	decoder.elemDecoder.Decode(elem, iter)
+	decoder.mapType.UnsafeSetIndex(ptr, key, elem)
+	for c = iter.nextToken(); c == ','; c = iter.nextToken() {
+		key := decoder.keyType.UnsafeNew()
+		decoder.keyDecoder.Decode(key, iter)
+		c = iter.nextToken()
+		if c != ':' {
+			iter.ReportError("ReadMapCB", "expect : after object field, but found "+string([]byte{c}))
+			return
+		}
+		elem := decoder.elemType.UnsafeNew()
+		decoder.elemDecoder.Decode(elem, iter)
+		decoder.mapType.UnsafeSetIndex(ptr, key, elem)
+	}
+	if c != '}' {
+		iter.ReportError("ReadMapCB", `expect }, but found `+string([]byte{c}))
+	}
+}
+
+type numericMapKeyDecoder struct {
+	decoder ValDecoder
+}
+
+func (decoder *numericMapKeyDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	c := iter.nextToken()
+	if c != '"' {
+		iter.ReportError("ReadMapCB", `expect ", but found `+string([]byte{c}))
+		return
+	}
+	decoder.decoder.Decode(ptr, iter)
+	c = iter.nextToken()
+	if c != '"' {
+		iter.ReportError("ReadMapCB", `expect ", but found `+string([]byte{c}))
+		return
+	}
+}
+
+type numericMapKeyEncoder struct {
+	encoder ValEncoder
+}
+
+func (encoder *numericMapKeyEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.writeByte('"')
+	encoder.encoder.Encode(ptr, stream)
+	stream.writeByte('"')
+}
+
+func (encoder *numericMapKeyEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return false
+}
+
+type dynamicMapKeyEncoder struct {
+	ctx     *ctx
+	valType reflect2.Type
+}
+
+func (encoder *dynamicMapKeyEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	obj := encoder.valType.UnsafeIndirect(ptr)
+	encoderOfMapKey(encoder.ctx, reflect2.TypeOf(obj)).Encode(reflect2.PtrOf(obj), stream)
+}
+
+func (encoder *dynamicMapKeyEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	obj := encoder.valType.UnsafeIndirect(ptr)
+	return encoderOfMapKey(encoder.ctx, reflect2.TypeOf(obj)).IsEmpty(reflect2.PtrOf(obj))
+}
+
+type mapEncoder struct {
+	mapType     *reflect2.UnsafeMapType
+	keyEncoder  ValEncoder
+	elemEncoder ValEncoder
+}
+
+func (encoder *mapEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteObjectStart()
+	iter := encoder.mapType.UnsafeIterate(ptr)
+	for i := 0; iter.HasNext(); i++ {
+		if i != 0 {
+			stream.WriteMore()
+		}
+		key, elem := iter.UnsafeNext()
+		encoder.keyEncoder.Encode(key, stream)
+		if stream.indention > 0 {
+			stream.writeTwoBytes(byte(':'), byte(' '))
+		} else {
+			stream.writeByte(':')
+		}
+		encoder.elemEncoder.Encode(elem, stream)
+	}
+	stream.WriteObjectEnd()
+}
+
+func (encoder *mapEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	iter := encoder.mapType.UnsafeIterate(ptr)
+	return !iter.HasNext()
+}
+
+type sortKeysMapEncoder struct {
+	mapType     *reflect2.UnsafeMapType
+	keyEncoder  ValEncoder
+	elemEncoder ValEncoder
+}
+
+func (encoder *sortKeysMapEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	if *(*unsafe.Pointer)(ptr) == nil {
+		stream.WriteNil()
+		return
+	}
+	stream.WriteObjectStart()
+	mapIter := encoder.mapType.UnsafeIterate(ptr)
+	subStream := stream.cfg.BorrowStream(nil)
+	subIter := stream.cfg.BorrowIterator(nil)
+	keyValues := encodedKeyValues{}
+	for mapIter.HasNext() {
+		subStream.buf = make([]byte, 0, 64)
+		key, elem := mapIter.UnsafeNext()
+		encoder.keyEncoder.Encode(key, subStream)
+		if subStream.Error != nil && subStream.Error != io.EOF && stream.Error == nil {
+			stream.Error = subStream.Error
+		}
+		encodedKey := subStream.Buffer()
+		subIter.ResetBytes(encodedKey)
+		decodedKey := subIter.ReadString()
+		if stream.indention > 0 {
+			subStream.writeTwoBytes(byte(':'), byte(' '))
+		} else {
+			subStream.writeByte(':')
+		}
+		encoder.elemEncoder.Encode(elem, subStream)
+		keyValues = append(keyValues, encodedKV{
+			key:      decodedKey,
+			keyValue: subStream.Buffer(),
+		})
+	}
+	sort.Sort(keyValues)
+	for i, keyValue := range keyValues {
+		if i != 0 {
+			stream.WriteMore()
+		}
+		stream.Write(keyValue.keyValue)
+	}
+	stream.WriteObjectEnd()
+	stream.cfg.ReturnStream(subStream)
+	stream.cfg.ReturnIterator(subIter)
+}
+
+func (encoder *sortKeysMapEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	iter := encoder.mapType.UnsafeIterate(ptr)
+	return !iter.HasNext()
+}
+
+type encodedKeyValues []encodedKV
+
+type encodedKV struct {
+	key      string
+	keyValue []byte
+}
+
+func (sv encodedKeyValues) Len() int           { return len(sv) }
+func (sv encodedKeyValues) Swap(i, j int)      { sv[i], sv[j] = sv[j], sv[i] }
+func (sv encodedKeyValues) Less(i, j int) bool { return sv[i].key < sv[j].key }
diff --git a/vendor/github.com/json-iterator/go/reflect_marshaler.go b/vendor/github.com/json-iterator/go/reflect_marshaler.go
new file mode 100644
index 0000000..fea5071
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/reflect_marshaler.go
@@ -0,0 +1,217 @@
+package jsoniter
+
+import (
+	"encoding"
+	"encoding/json"
+	"github.com/modern-go/reflect2"
+	"unsafe"
+)
+
+var marshalerType = reflect2.TypeOfPtr((*json.Marshaler)(nil)).Elem()
+var unmarshalerType = reflect2.TypeOfPtr((*json.Unmarshaler)(nil)).Elem()
+var textMarshalerType = reflect2.TypeOfPtr((*encoding.TextMarshaler)(nil)).Elem()
+var textUnmarshalerType = reflect2.TypeOfPtr((*encoding.TextUnmarshaler)(nil)).Elem()
+
+func createDecoderOfMarshaler(ctx *ctx, typ reflect2.Type) ValDecoder {
+	ptrType := reflect2.PtrTo(typ)
+	if ptrType.Implements(unmarshalerType) {
+		return &referenceDecoder{
+			&unmarshalerDecoder{ptrType},
+		}
+	}
+	if ptrType.Implements(textUnmarshalerType) {
+		return &referenceDecoder{
+			&textUnmarshalerDecoder{ptrType},
+		}
+	}
+	return nil
+}
+
+func createEncoderOfMarshaler(ctx *ctx, typ reflect2.Type) ValEncoder {
+	if typ == marshalerType {
+		checkIsEmpty := createCheckIsEmpty(ctx, typ)
+		var encoder ValEncoder = &directMarshalerEncoder{
+			checkIsEmpty: checkIsEmpty,
+		}
+		return encoder
+	}
+	if typ.Implements(marshalerType) {
+		checkIsEmpty := createCheckIsEmpty(ctx, typ)
+		var encoder ValEncoder = &marshalerEncoder{
+			valType:      typ,
+			checkIsEmpty: checkIsEmpty,
+		}
+		return encoder
+	}
+	ptrType := reflect2.PtrTo(typ)
+	if ctx.prefix != "" && ptrType.Implements(marshalerType) {
+		checkIsEmpty := createCheckIsEmpty(ctx, ptrType)
+		var encoder ValEncoder = &marshalerEncoder{
+			valType:      ptrType,
+			checkIsEmpty: checkIsEmpty,
+		}
+		return &referenceEncoder{encoder}
+	}
+	if typ == textMarshalerType {
+		checkIsEmpty := createCheckIsEmpty(ctx, typ)
+		var encoder ValEncoder = &directTextMarshalerEncoder{
+			checkIsEmpty:  checkIsEmpty,
+			stringEncoder: ctx.EncoderOf(reflect2.TypeOf("")),
+		}
+		return encoder
+	}
+	if typ.Implements(textMarshalerType) {
+		checkIsEmpty := createCheckIsEmpty(ctx, typ)
+		var encoder ValEncoder = &textMarshalerEncoder{
+			valType:       typ,
+			stringEncoder: ctx.EncoderOf(reflect2.TypeOf("")),
+			checkIsEmpty:  checkIsEmpty,
+		}
+		return encoder
+	}
+	// if prefix is empty, the type is the root type
+	if ctx.prefix != "" && ptrType.Implements(textMarshalerType) {
+		checkIsEmpty := createCheckIsEmpty(ctx, ptrType)
+		var encoder ValEncoder = &textMarshalerEncoder{
+			valType:       ptrType,
+			stringEncoder: ctx.EncoderOf(reflect2.TypeOf("")),
+			checkIsEmpty:  checkIsEmpty,
+		}
+		return &referenceEncoder{encoder}
+	}
+	return nil
+}
+
+type marshalerEncoder struct {
+	checkIsEmpty checkIsEmpty
+	valType      reflect2.Type
+}
+
+func (encoder *marshalerEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	obj := encoder.valType.UnsafeIndirect(ptr)
+	if encoder.valType.IsNullable() && reflect2.IsNil(obj) {
+		stream.WriteNil()
+		return
+	}
+	bytes, err := json.Marshal(obj)
+	if err != nil {
+		stream.Error = err
+	} else {
+		stream.Write(bytes)
+	}
+}
+
+func (encoder *marshalerEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return encoder.checkIsEmpty.IsEmpty(ptr)
+}
+
+type directMarshalerEncoder struct {
+	checkIsEmpty checkIsEmpty
+}
+
+func (encoder *directMarshalerEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	marshaler := *(*json.Marshaler)(ptr)
+	if marshaler == nil {
+		stream.WriteNil()
+		return
+	}
+	bytes, err := marshaler.MarshalJSON()
+	if err != nil {
+		stream.Error = err
+	} else {
+		stream.Write(bytes)
+	}
+}
+
+func (encoder *directMarshalerEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return encoder.checkIsEmpty.IsEmpty(ptr)
+}
+
+type textMarshalerEncoder struct {
+	valType       reflect2.Type
+	stringEncoder ValEncoder
+	checkIsEmpty  checkIsEmpty
+}
+
+func (encoder *textMarshalerEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	obj := encoder.valType.UnsafeIndirect(ptr)
+	if encoder.valType.IsNullable() && reflect2.IsNil(obj) {
+		stream.WriteNil()
+		return
+	}
+	marshaler := (obj).(encoding.TextMarshaler)
+	bytes, err := marshaler.MarshalText()
+	if err != nil {
+		stream.Error = err
+	} else {
+		str := string(bytes)
+		encoder.stringEncoder.Encode(unsafe.Pointer(&str), stream)
+	}
+}
+
+func (encoder *textMarshalerEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return encoder.checkIsEmpty.IsEmpty(ptr)
+}
+
+type directTextMarshalerEncoder struct {
+	stringEncoder ValEncoder
+	checkIsEmpty  checkIsEmpty
+}
+
+func (encoder *directTextMarshalerEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	marshaler := *(*encoding.TextMarshaler)(ptr)
+	if marshaler == nil {
+		stream.WriteNil()
+		return
+	}
+	bytes, err := marshaler.MarshalText()
+	if err != nil {
+		stream.Error = err
+	} else {
+		str := string(bytes)
+		encoder.stringEncoder.Encode(unsafe.Pointer(&str), stream)
+	}
+}
+
+func (encoder *directTextMarshalerEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return encoder.checkIsEmpty.IsEmpty(ptr)
+}
+
+type unmarshalerDecoder struct {
+	valType reflect2.Type
+}
+
+func (decoder *unmarshalerDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	valType := decoder.valType
+	obj := valType.UnsafeIndirect(ptr)
+	unmarshaler := obj.(json.Unmarshaler)
+	iter.nextToken()
+	iter.unreadByte() // skip spaces
+	bytes := iter.SkipAndReturnBytes()
+	err := unmarshaler.UnmarshalJSON(bytes)
+	if err != nil {
+		iter.ReportError("unmarshalerDecoder", err.Error())
+	}
+}
+
+type textUnmarshalerDecoder struct {
+	valType reflect2.Type
+}
+
+func (decoder *textUnmarshalerDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	valType := decoder.valType
+	obj := valType.UnsafeIndirect(ptr)
+	if reflect2.IsNil(obj) {
+		ptrType := valType.(*reflect2.UnsafePtrType)
+		elemType := ptrType.Elem()
+		elem := elemType.UnsafeNew()
+		ptrType.UnsafeSet(ptr, unsafe.Pointer(&elem))
+		obj = valType.UnsafeIndirect(ptr)
+	}
+	unmarshaler := (obj).(encoding.TextUnmarshaler)
+	str := iter.ReadString()
+	err := unmarshaler.UnmarshalText([]byte(str))
+	if err != nil {
+		iter.ReportError("textUnmarshalerDecoder", err.Error())
+	}
+}
diff --git a/vendor/github.com/json-iterator/go/reflect_native.go b/vendor/github.com/json-iterator/go/reflect_native.go
new file mode 100644
index 0000000..f88722d
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/reflect_native.go
@@ -0,0 +1,453 @@
+package jsoniter
+
+import (
+	"encoding/base64"
+	"reflect"
+	"strconv"
+	"unsafe"
+
+	"github.com/modern-go/reflect2"
+)
+
+const ptrSize = 32 << uintptr(^uintptr(0)>>63)
+
+func createEncoderOfNative(ctx *ctx, typ reflect2.Type) ValEncoder {
+	if typ.Kind() == reflect.Slice && typ.(reflect2.SliceType).Elem().Kind() == reflect.Uint8 {
+		sliceDecoder := decoderOfSlice(ctx, typ)
+		return &base64Codec{sliceDecoder: sliceDecoder}
+	}
+	typeName := typ.String()
+	kind := typ.Kind()
+	switch kind {
+	case reflect.String:
+		if typeName != "string" {
+			return encoderOfType(ctx, reflect2.TypeOfPtr((*string)(nil)).Elem())
+		}
+		return &stringCodec{}
+	case reflect.Int:
+		if typeName != "int" {
+			return encoderOfType(ctx, reflect2.TypeOfPtr((*int)(nil)).Elem())
+		}
+		if strconv.IntSize == 32 {
+			return &int32Codec{}
+		}
+		return &int64Codec{}
+	case reflect.Int8:
+		if typeName != "int8" {
+			return encoderOfType(ctx, reflect2.TypeOfPtr((*int8)(nil)).Elem())
+		}
+		return &int8Codec{}
+	case reflect.Int16:
+		if typeName != "int16" {
+			return encoderOfType(ctx, reflect2.TypeOfPtr((*int16)(nil)).Elem())
+		}
+		return &int16Codec{}
+	case reflect.Int32:
+		if typeName != "int32" {
+			return encoderOfType(ctx, reflect2.TypeOfPtr((*int32)(nil)).Elem())
+		}
+		return &int32Codec{}
+	case reflect.Int64:
+		if typeName != "int64" {
+			return encoderOfType(ctx, reflect2.TypeOfPtr((*int64)(nil)).Elem())
+		}
+		return &int64Codec{}
+	case reflect.Uint:
+		if typeName != "uint" {
+			return encoderOfType(ctx, reflect2.TypeOfPtr((*uint)(nil)).Elem())
+		}
+		if strconv.IntSize == 32 {
+			return &uint32Codec{}
+		}
+		return &uint64Codec{}
+	case reflect.Uint8:
+		if typeName != "uint8" {
+			return encoderOfType(ctx, reflect2.TypeOfPtr((*uint8)(nil)).Elem())
+		}
+		return &uint8Codec{}
+	case reflect.Uint16:
+		if typeName != "uint16" {
+			return encoderOfType(ctx, reflect2.TypeOfPtr((*uint16)(nil)).Elem())
+		}
+		return &uint16Codec{}
+	case reflect.Uint32:
+		if typeName != "uint32" {
+			return encoderOfType(ctx, reflect2.TypeOfPtr((*uint32)(nil)).Elem())
+		}
+		return &uint32Codec{}
+	case reflect.Uintptr:
+		if typeName != "uintptr" {
+			return encoderOfType(ctx, reflect2.TypeOfPtr((*uintptr)(nil)).Elem())
+		}
+		if ptrSize == 32 {
+			return &uint32Codec{}
+		}
+		return &uint64Codec{}
+	case reflect.Uint64:
+		if typeName != "uint64" {
+			return encoderOfType(ctx, reflect2.TypeOfPtr((*uint64)(nil)).Elem())
+		}
+		return &uint64Codec{}
+	case reflect.Float32:
+		if typeName != "float32" {
+			return encoderOfType(ctx, reflect2.TypeOfPtr((*float32)(nil)).Elem())
+		}
+		return &float32Codec{}
+	case reflect.Float64:
+		if typeName != "float64" {
+			return encoderOfType(ctx, reflect2.TypeOfPtr((*float64)(nil)).Elem())
+		}
+		return &float64Codec{}
+	case reflect.Bool:
+		if typeName != "bool" {
+			return encoderOfType(ctx, reflect2.TypeOfPtr((*bool)(nil)).Elem())
+		}
+		return &boolCodec{}
+	}
+	return nil
+}
+
+func createDecoderOfNative(ctx *ctx, typ reflect2.Type) ValDecoder {
+	if typ.Kind() == reflect.Slice && typ.(reflect2.SliceType).Elem().Kind() == reflect.Uint8 {
+		sliceDecoder := decoderOfSlice(ctx, typ)
+		return &base64Codec{sliceDecoder: sliceDecoder}
+	}
+	typeName := typ.String()
+	switch typ.Kind() {
+	case reflect.String:
+		if typeName != "string" {
+			return decoderOfType(ctx, reflect2.TypeOfPtr((*string)(nil)).Elem())
+		}
+		return &stringCodec{}
+	case reflect.Int:
+		if typeName != "int" {
+			return decoderOfType(ctx, reflect2.TypeOfPtr((*int)(nil)).Elem())
+		}
+		if strconv.IntSize == 32 {
+			return &int32Codec{}
+		}
+		return &int64Codec{}
+	case reflect.Int8:
+		if typeName != "int8" {
+			return decoderOfType(ctx, reflect2.TypeOfPtr((*int8)(nil)).Elem())
+		}
+		return &int8Codec{}
+	case reflect.Int16:
+		if typeName != "int16" {
+			return decoderOfType(ctx, reflect2.TypeOfPtr((*int16)(nil)).Elem())
+		}
+		return &int16Codec{}
+	case reflect.Int32:
+		if typeName != "int32" {
+			return decoderOfType(ctx, reflect2.TypeOfPtr((*int32)(nil)).Elem())
+		}
+		return &int32Codec{}
+	case reflect.Int64:
+		if typeName != "int64" {
+			return decoderOfType(ctx, reflect2.TypeOfPtr((*int64)(nil)).Elem())
+		}
+		return &int64Codec{}
+	case reflect.Uint:
+		if typeName != "uint" {
+			return decoderOfType(ctx, reflect2.TypeOfPtr((*uint)(nil)).Elem())
+		}
+		if strconv.IntSize == 32 {
+			return &uint32Codec{}
+		}
+		return &uint64Codec{}
+	case reflect.Uint8:
+		if typeName != "uint8" {
+			return decoderOfType(ctx, reflect2.TypeOfPtr((*uint8)(nil)).Elem())
+		}
+		return &uint8Codec{}
+	case reflect.Uint16:
+		if typeName != "uint16" {
+			return decoderOfType(ctx, reflect2.TypeOfPtr((*uint16)(nil)).Elem())
+		}
+		return &uint16Codec{}
+	case reflect.Uint32:
+		if typeName != "uint32" {
+			return decoderOfType(ctx, reflect2.TypeOfPtr((*uint32)(nil)).Elem())
+		}
+		return &uint32Codec{}
+	case reflect.Uintptr:
+		if typeName != "uintptr" {
+			return decoderOfType(ctx, reflect2.TypeOfPtr((*uintptr)(nil)).Elem())
+		}
+		if ptrSize == 32 {
+			return &uint32Codec{}
+		}
+		return &uint64Codec{}
+	case reflect.Uint64:
+		if typeName != "uint64" {
+			return decoderOfType(ctx, reflect2.TypeOfPtr((*uint64)(nil)).Elem())
+		}
+		return &uint64Codec{}
+	case reflect.Float32:
+		if typeName != "float32" {
+			return decoderOfType(ctx, reflect2.TypeOfPtr((*float32)(nil)).Elem())
+		}
+		return &float32Codec{}
+	case reflect.Float64:
+		if typeName != "float64" {
+			return decoderOfType(ctx, reflect2.TypeOfPtr((*float64)(nil)).Elem())
+		}
+		return &float64Codec{}
+	case reflect.Bool:
+		if typeName != "bool" {
+			return decoderOfType(ctx, reflect2.TypeOfPtr((*bool)(nil)).Elem())
+		}
+		return &boolCodec{}
+	}
+	return nil
+}
+
+type stringCodec struct {
+}
+
+func (codec *stringCodec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	*((*string)(ptr)) = iter.ReadString()
+}
+
+func (codec *stringCodec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	str := *((*string)(ptr))
+	stream.WriteString(str)
+}
+
+func (codec *stringCodec) IsEmpty(ptr unsafe.Pointer) bool {
+	return *((*string)(ptr)) == ""
+}
+
+type int8Codec struct {
+}
+
+func (codec *int8Codec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.ReadNil() {
+		*((*int8)(ptr)) = iter.ReadInt8()
+	}
+}
+
+func (codec *int8Codec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteInt8(*((*int8)(ptr)))
+}
+
+func (codec *int8Codec) IsEmpty(ptr unsafe.Pointer) bool {
+	return *((*int8)(ptr)) == 0
+}
+
+type int16Codec struct {
+}
+
+func (codec *int16Codec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.ReadNil() {
+		*((*int16)(ptr)) = iter.ReadInt16()
+	}
+}
+
+func (codec *int16Codec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteInt16(*((*int16)(ptr)))
+}
+
+func (codec *int16Codec) IsEmpty(ptr unsafe.Pointer) bool {
+	return *((*int16)(ptr)) == 0
+}
+
+type int32Codec struct {
+}
+
+func (codec *int32Codec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.ReadNil() {
+		*((*int32)(ptr)) = iter.ReadInt32()
+	}
+}
+
+func (codec *int32Codec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteInt32(*((*int32)(ptr)))
+}
+
+func (codec *int32Codec) IsEmpty(ptr unsafe.Pointer) bool {
+	return *((*int32)(ptr)) == 0
+}
+
+type int64Codec struct {
+}
+
+func (codec *int64Codec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.ReadNil() {
+		*((*int64)(ptr)) = iter.ReadInt64()
+	}
+}
+
+func (codec *int64Codec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteInt64(*((*int64)(ptr)))
+}
+
+func (codec *int64Codec) IsEmpty(ptr unsafe.Pointer) bool {
+	return *((*int64)(ptr)) == 0
+}
+
+type uint8Codec struct {
+}
+
+func (codec *uint8Codec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.ReadNil() {
+		*((*uint8)(ptr)) = iter.ReadUint8()
+	}
+}
+
+func (codec *uint8Codec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteUint8(*((*uint8)(ptr)))
+}
+
+func (codec *uint8Codec) IsEmpty(ptr unsafe.Pointer) bool {
+	return *((*uint8)(ptr)) == 0
+}
+
+type uint16Codec struct {
+}
+
+func (codec *uint16Codec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.ReadNil() {
+		*((*uint16)(ptr)) = iter.ReadUint16()
+	}
+}
+
+func (codec *uint16Codec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteUint16(*((*uint16)(ptr)))
+}
+
+func (codec *uint16Codec) IsEmpty(ptr unsafe.Pointer) bool {
+	return *((*uint16)(ptr)) == 0
+}
+
+type uint32Codec struct {
+}
+
+func (codec *uint32Codec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.ReadNil() {
+		*((*uint32)(ptr)) = iter.ReadUint32()
+	}
+}
+
+func (codec *uint32Codec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteUint32(*((*uint32)(ptr)))
+}
+
+func (codec *uint32Codec) IsEmpty(ptr unsafe.Pointer) bool {
+	return *((*uint32)(ptr)) == 0
+}
+
+type uint64Codec struct {
+}
+
+func (codec *uint64Codec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.ReadNil() {
+		*((*uint64)(ptr)) = iter.ReadUint64()
+	}
+}
+
+func (codec *uint64Codec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteUint64(*((*uint64)(ptr)))
+}
+
+func (codec *uint64Codec) IsEmpty(ptr unsafe.Pointer) bool {
+	return *((*uint64)(ptr)) == 0
+}
+
+type float32Codec struct {
+}
+
+func (codec *float32Codec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.ReadNil() {
+		*((*float32)(ptr)) = iter.ReadFloat32()
+	}
+}
+
+func (codec *float32Codec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteFloat32(*((*float32)(ptr)))
+}
+
+func (codec *float32Codec) IsEmpty(ptr unsafe.Pointer) bool {
+	return *((*float32)(ptr)) == 0
+}
+
+type float64Codec struct {
+}
+
+func (codec *float64Codec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.ReadNil() {
+		*((*float64)(ptr)) = iter.ReadFloat64()
+	}
+}
+
+func (codec *float64Codec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteFloat64(*((*float64)(ptr)))
+}
+
+func (codec *float64Codec) IsEmpty(ptr unsafe.Pointer) bool {
+	return *((*float64)(ptr)) == 0
+}
+
+type boolCodec struct {
+}
+
+func (codec *boolCodec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.ReadNil() {
+		*((*bool)(ptr)) = iter.ReadBool()
+	}
+}
+
+func (codec *boolCodec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteBool(*((*bool)(ptr)))
+}
+
+func (codec *boolCodec) IsEmpty(ptr unsafe.Pointer) bool {
+	return !(*((*bool)(ptr)))
+}
+
+type base64Codec struct {
+	sliceType    *reflect2.UnsafeSliceType
+	sliceDecoder ValDecoder
+}
+
+func (codec *base64Codec) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if iter.ReadNil() {
+		codec.sliceType.UnsafeSetNil(ptr)
+		return
+	}
+	switch iter.WhatIsNext() {
+	case StringValue:
+		src := iter.ReadString()
+		dst, err := base64.StdEncoding.DecodeString(src)
+		if err != nil {
+			iter.ReportError("decode base64", err.Error())
+		} else {
+			codec.sliceType.UnsafeSet(ptr, unsafe.Pointer(&dst))
+		}
+	case ArrayValue:
+		codec.sliceDecoder.Decode(ptr, iter)
+	default:
+		iter.ReportError("base64Codec", "invalid input")
+	}
+}
+
+func (codec *base64Codec) Encode(ptr unsafe.Pointer, stream *Stream) {
+	if codec.sliceType.UnsafeIsNil(ptr) {
+		stream.WriteNil()
+		return
+	}
+	src := *((*[]byte)(ptr))
+	encoding := base64.StdEncoding
+	stream.writeByte('"')
+	if len(src) != 0 {
+		size := encoding.EncodedLen(len(src))
+		buf := make([]byte, size)
+		encoding.Encode(buf, src)
+		stream.buf = append(stream.buf, buf...)
+	}
+	stream.writeByte('"')
+}
+
+func (codec *base64Codec) IsEmpty(ptr unsafe.Pointer) bool {
+	return len(*((*[]byte)(ptr))) == 0
+}
diff --git a/vendor/github.com/json-iterator/go/reflect_optional.go b/vendor/github.com/json-iterator/go/reflect_optional.go
new file mode 100644
index 0000000..43ec71d
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/reflect_optional.go
@@ -0,0 +1,133 @@
+package jsoniter
+
+import (
+	"github.com/modern-go/reflect2"
+	"reflect"
+	"unsafe"
+)
+
+func decoderOfOptional(ctx *ctx, typ reflect2.Type) ValDecoder {
+	ptrType := typ.(*reflect2.UnsafePtrType)
+	elemType := ptrType.Elem()
+	decoder := decoderOfType(ctx, elemType)
+	if ctx.prefix == "" && elemType.Kind() == reflect.Ptr {
+		return &dereferenceDecoder{elemType, decoder}
+	}
+	return &OptionalDecoder{elemType, decoder}
+}
+
+func encoderOfOptional(ctx *ctx, typ reflect2.Type) ValEncoder {
+	ptrType := typ.(*reflect2.UnsafePtrType)
+	elemType := ptrType.Elem()
+	elemEncoder := encoderOfType(ctx, elemType)
+	encoder := &OptionalEncoder{elemEncoder}
+	return encoder
+}
+
+type OptionalDecoder struct {
+	ValueType    reflect2.Type
+	ValueDecoder ValDecoder
+}
+
+func (decoder *OptionalDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if iter.ReadNil() {
+		*((*unsafe.Pointer)(ptr)) = nil
+	} else {
+		if *((*unsafe.Pointer)(ptr)) == nil {
+			//pointer to null, we have to allocate memory to hold the value
+			newPtr := decoder.ValueType.UnsafeNew()
+			decoder.ValueDecoder.Decode(newPtr, iter)
+			*((*unsafe.Pointer)(ptr)) = newPtr
+		} else {
+			//reuse existing instance
+			decoder.ValueDecoder.Decode(*((*unsafe.Pointer)(ptr)), iter)
+		}
+	}
+}
+
+type dereferenceDecoder struct {
+	// only to deference a pointer
+	valueType    reflect2.Type
+	valueDecoder ValDecoder
+}
+
+func (decoder *dereferenceDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if *((*unsafe.Pointer)(ptr)) == nil {
+		//pointer to null, we have to allocate memory to hold the value
+		newPtr := decoder.valueType.UnsafeNew()
+		decoder.valueDecoder.Decode(newPtr, iter)
+		*((*unsafe.Pointer)(ptr)) = newPtr
+	} else {
+		//reuse existing instance
+		decoder.valueDecoder.Decode(*((*unsafe.Pointer)(ptr)), iter)
+	}
+}
+
+type OptionalEncoder struct {
+	ValueEncoder ValEncoder
+}
+
+func (encoder *OptionalEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	if *((*unsafe.Pointer)(ptr)) == nil {
+		stream.WriteNil()
+	} else {
+		encoder.ValueEncoder.Encode(*((*unsafe.Pointer)(ptr)), stream)
+	}
+}
+
+func (encoder *OptionalEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return *((*unsafe.Pointer)(ptr)) == nil
+}
+
+type dereferenceEncoder struct {
+	ValueEncoder ValEncoder
+}
+
+func (encoder *dereferenceEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	if *((*unsafe.Pointer)(ptr)) == nil {
+		stream.WriteNil()
+	} else {
+		encoder.ValueEncoder.Encode(*((*unsafe.Pointer)(ptr)), stream)
+	}
+}
+
+func (encoder *dereferenceEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	dePtr := *((*unsafe.Pointer)(ptr))
+	if dePtr == nil {
+		return true
+	}
+	return encoder.ValueEncoder.IsEmpty(dePtr)
+}
+
+func (encoder *dereferenceEncoder) IsEmbeddedPtrNil(ptr unsafe.Pointer) bool {
+	deReferenced := *((*unsafe.Pointer)(ptr))
+	if deReferenced == nil {
+		return true
+	}
+	isEmbeddedPtrNil, converted := encoder.ValueEncoder.(IsEmbeddedPtrNil)
+	if !converted {
+		return false
+	}
+	fieldPtr := unsafe.Pointer(deReferenced)
+	return isEmbeddedPtrNil.IsEmbeddedPtrNil(fieldPtr)
+}
+
+type referenceEncoder struct {
+	encoder ValEncoder
+}
+
+func (encoder *referenceEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	encoder.encoder.Encode(unsafe.Pointer(&ptr), stream)
+}
+
+func (encoder *referenceEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return encoder.encoder.IsEmpty(unsafe.Pointer(&ptr))
+}
+
+type referenceDecoder struct {
+	decoder ValDecoder
+}
+
+func (decoder *referenceDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	decoder.decoder.Decode(unsafe.Pointer(&ptr), iter)
+}
diff --git a/vendor/github.com/json-iterator/go/reflect_slice.go b/vendor/github.com/json-iterator/go/reflect_slice.go
new file mode 100644
index 0000000..9441d79
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/reflect_slice.go
@@ -0,0 +1,99 @@
+package jsoniter
+
+import (
+	"fmt"
+	"github.com/modern-go/reflect2"
+	"io"
+	"unsafe"
+)
+
+func decoderOfSlice(ctx *ctx, typ reflect2.Type) ValDecoder {
+	sliceType := typ.(*reflect2.UnsafeSliceType)
+	decoder := decoderOfType(ctx.append("[sliceElem]"), sliceType.Elem())
+	return &sliceDecoder{sliceType, decoder}
+}
+
+func encoderOfSlice(ctx *ctx, typ reflect2.Type) ValEncoder {
+	sliceType := typ.(*reflect2.UnsafeSliceType)
+	encoder := encoderOfType(ctx.append("[sliceElem]"), sliceType.Elem())
+	return &sliceEncoder{sliceType, encoder}
+}
+
+type sliceEncoder struct {
+	sliceType   *reflect2.UnsafeSliceType
+	elemEncoder ValEncoder
+}
+
+func (encoder *sliceEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	if encoder.sliceType.UnsafeIsNil(ptr) {
+		stream.WriteNil()
+		return
+	}
+	length := encoder.sliceType.UnsafeLengthOf(ptr)
+	if length == 0 {
+		stream.WriteEmptyArray()
+		return
+	}
+	stream.WriteArrayStart()
+	encoder.elemEncoder.Encode(encoder.sliceType.UnsafeGetIndex(ptr, 0), stream)
+	for i := 1; i < length; i++ {
+		stream.WriteMore()
+		elemPtr := encoder.sliceType.UnsafeGetIndex(ptr, i)
+		encoder.elemEncoder.Encode(elemPtr, stream)
+	}
+	stream.WriteArrayEnd()
+	if stream.Error != nil && stream.Error != io.EOF {
+		stream.Error = fmt.Errorf("%v: %s", encoder.sliceType, stream.Error.Error())
+	}
+}
+
+func (encoder *sliceEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return encoder.sliceType.UnsafeLengthOf(ptr) == 0
+}
+
+type sliceDecoder struct {
+	sliceType   *reflect2.UnsafeSliceType
+	elemDecoder ValDecoder
+}
+
+func (decoder *sliceDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	decoder.doDecode(ptr, iter)
+	if iter.Error != nil && iter.Error != io.EOF {
+		iter.Error = fmt.Errorf("%v: %s", decoder.sliceType, iter.Error.Error())
+	}
+}
+
+func (decoder *sliceDecoder) doDecode(ptr unsafe.Pointer, iter *Iterator) {
+	c := iter.nextToken()
+	sliceType := decoder.sliceType
+	if c == 'n' {
+		iter.skipThreeBytes('u', 'l', 'l')
+		sliceType.UnsafeSetNil(ptr)
+		return
+	}
+	if c != '[' {
+		iter.ReportError("decode slice", "expect [ or n, but found "+string([]byte{c}))
+		return
+	}
+	c = iter.nextToken()
+	if c == ']' {
+		sliceType.UnsafeSet(ptr, sliceType.UnsafeMakeSlice(0, 0))
+		return
+	}
+	iter.unreadByte()
+	sliceType.UnsafeGrow(ptr, 1)
+	elemPtr := sliceType.UnsafeGetIndex(ptr, 0)
+	decoder.elemDecoder.Decode(elemPtr, iter)
+	length := 1
+	for c = iter.nextToken(); c == ','; c = iter.nextToken() {
+		idx := length
+		length += 1
+		sliceType.UnsafeGrow(ptr, length)
+		elemPtr = sliceType.UnsafeGetIndex(ptr, idx)
+		decoder.elemDecoder.Decode(elemPtr, iter)
+	}
+	if c != ']' {
+		iter.ReportError("decode slice", "expect ], but found "+string([]byte{c}))
+		return
+	}
+}
diff --git a/vendor/github.com/json-iterator/go/reflect_struct_decoder.go b/vendor/github.com/json-iterator/go/reflect_struct_decoder.go
new file mode 100644
index 0000000..932641a
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/reflect_struct_decoder.go
@@ -0,0 +1,1048 @@
+package jsoniter
+
+import (
+	"fmt"
+	"io"
+	"strings"
+	"unsafe"
+
+	"github.com/modern-go/reflect2"
+)
+
+func decoderOfStruct(ctx *ctx, typ reflect2.Type) ValDecoder {
+	bindings := map[string]*Binding{}
+	structDescriptor := describeStruct(ctx, typ)
+	for _, binding := range structDescriptor.Fields {
+		for _, fromName := range binding.FromNames {
+			old := bindings[fromName]
+			if old == nil {
+				bindings[fromName] = binding
+				continue
+			}
+			ignoreOld, ignoreNew := resolveConflictBinding(ctx.frozenConfig, old, binding)
+			if ignoreOld {
+				delete(bindings, fromName)
+			}
+			if !ignoreNew {
+				bindings[fromName] = binding
+			}
+		}
+	}
+	fields := map[string]*structFieldDecoder{}
+	for k, binding := range bindings {
+		fields[k] = binding.Decoder.(*structFieldDecoder)
+	}
+
+	if !ctx.caseSensitive() {
+		for k, binding := range bindings {
+			if _, found := fields[strings.ToLower(k)]; !found {
+				fields[strings.ToLower(k)] = binding.Decoder.(*structFieldDecoder)
+			}
+		}
+	}
+
+	return createStructDecoder(ctx, typ, fields)
+}
+
+func createStructDecoder(ctx *ctx, typ reflect2.Type, fields map[string]*structFieldDecoder) ValDecoder {
+	if ctx.disallowUnknownFields {
+		return &generalStructDecoder{typ: typ, fields: fields, disallowUnknownFields: true}
+	}
+	knownHash := map[int64]struct{}{
+		0: {},
+	}
+
+	switch len(fields) {
+	case 0:
+		return &skipObjectDecoder{typ}
+	case 1:
+		for fieldName, fieldDecoder := range fields {
+			fieldHash := calcHash(fieldName, ctx.caseSensitive())
+			_, known := knownHash[fieldHash]
+			if known {
+				return &generalStructDecoder{typ, fields, false}
+			}
+			knownHash[fieldHash] = struct{}{}
+			return &oneFieldStructDecoder{typ, fieldHash, fieldDecoder}
+		}
+	case 2:
+		var fieldHash1 int64
+		var fieldHash2 int64
+		var fieldDecoder1 *structFieldDecoder
+		var fieldDecoder2 *structFieldDecoder
+		for fieldName, fieldDecoder := range fields {
+			fieldHash := calcHash(fieldName, ctx.caseSensitive())
+			_, known := knownHash[fieldHash]
+			if known {
+				return &generalStructDecoder{typ, fields, false}
+			}
+			knownHash[fieldHash] = struct{}{}
+			if fieldHash1 == 0 {
+				fieldHash1 = fieldHash
+				fieldDecoder1 = fieldDecoder
+			} else {
+				fieldHash2 = fieldHash
+				fieldDecoder2 = fieldDecoder
+			}
+		}
+		return &twoFieldsStructDecoder{typ, fieldHash1, fieldDecoder1, fieldHash2, fieldDecoder2}
+	case 3:
+		var fieldName1 int64
+		var fieldName2 int64
+		var fieldName3 int64
+		var fieldDecoder1 *structFieldDecoder
+		var fieldDecoder2 *structFieldDecoder
+		var fieldDecoder3 *structFieldDecoder
+		for fieldName, fieldDecoder := range fields {
+			fieldHash := calcHash(fieldName, ctx.caseSensitive())
+			_, known := knownHash[fieldHash]
+			if known {
+				return &generalStructDecoder{typ, fields, false}
+			}
+			knownHash[fieldHash] = struct{}{}
+			if fieldName1 == 0 {
+				fieldName1 = fieldHash
+				fieldDecoder1 = fieldDecoder
+			} else if fieldName2 == 0 {
+				fieldName2 = fieldHash
+				fieldDecoder2 = fieldDecoder
+			} else {
+				fieldName3 = fieldHash
+				fieldDecoder3 = fieldDecoder
+			}
+		}
+		return &threeFieldsStructDecoder{typ,
+			fieldName1, fieldDecoder1,
+			fieldName2, fieldDecoder2,
+			fieldName3, fieldDecoder3}
+	case 4:
+		var fieldName1 int64
+		var fieldName2 int64
+		var fieldName3 int64
+		var fieldName4 int64
+		var fieldDecoder1 *structFieldDecoder
+		var fieldDecoder2 *structFieldDecoder
+		var fieldDecoder3 *structFieldDecoder
+		var fieldDecoder4 *structFieldDecoder
+		for fieldName, fieldDecoder := range fields {
+			fieldHash := calcHash(fieldName, ctx.caseSensitive())
+			_, known := knownHash[fieldHash]
+			if known {
+				return &generalStructDecoder{typ, fields, false}
+			}
+			knownHash[fieldHash] = struct{}{}
+			if fieldName1 == 0 {
+				fieldName1 = fieldHash
+				fieldDecoder1 = fieldDecoder
+			} else if fieldName2 == 0 {
+				fieldName2 = fieldHash
+				fieldDecoder2 = fieldDecoder
+			} else if fieldName3 == 0 {
+				fieldName3 = fieldHash
+				fieldDecoder3 = fieldDecoder
+			} else {
+				fieldName4 = fieldHash
+				fieldDecoder4 = fieldDecoder
+			}
+		}
+		return &fourFieldsStructDecoder{typ,
+			fieldName1, fieldDecoder1,
+			fieldName2, fieldDecoder2,
+			fieldName3, fieldDecoder3,
+			fieldName4, fieldDecoder4}
+	case 5:
+		var fieldName1 int64
+		var fieldName2 int64
+		var fieldName3 int64
+		var fieldName4 int64
+		var fieldName5 int64
+		var fieldDecoder1 *structFieldDecoder
+		var fieldDecoder2 *structFieldDecoder
+		var fieldDecoder3 *structFieldDecoder
+		var fieldDecoder4 *structFieldDecoder
+		var fieldDecoder5 *structFieldDecoder
+		for fieldName, fieldDecoder := range fields {
+			fieldHash := calcHash(fieldName, ctx.caseSensitive())
+			_, known := knownHash[fieldHash]
+			if known {
+				return &generalStructDecoder{typ, fields, false}
+			}
+			knownHash[fieldHash] = struct{}{}
+			if fieldName1 == 0 {
+				fieldName1 = fieldHash
+				fieldDecoder1 = fieldDecoder
+			} else if fieldName2 == 0 {
+				fieldName2 = fieldHash
+				fieldDecoder2 = fieldDecoder
+			} else if fieldName3 == 0 {
+				fieldName3 = fieldHash
+				fieldDecoder3 = fieldDecoder
+			} else if fieldName4 == 0 {
+				fieldName4 = fieldHash
+				fieldDecoder4 = fieldDecoder
+			} else {
+				fieldName5 = fieldHash
+				fieldDecoder5 = fieldDecoder
+			}
+		}
+		return &fiveFieldsStructDecoder{typ,
+			fieldName1, fieldDecoder1,
+			fieldName2, fieldDecoder2,
+			fieldName3, fieldDecoder3,
+			fieldName4, fieldDecoder4,
+			fieldName5, fieldDecoder5}
+	case 6:
+		var fieldName1 int64
+		var fieldName2 int64
+		var fieldName3 int64
+		var fieldName4 int64
+		var fieldName5 int64
+		var fieldName6 int64
+		var fieldDecoder1 *structFieldDecoder
+		var fieldDecoder2 *structFieldDecoder
+		var fieldDecoder3 *structFieldDecoder
+		var fieldDecoder4 *structFieldDecoder
+		var fieldDecoder5 *structFieldDecoder
+		var fieldDecoder6 *structFieldDecoder
+		for fieldName, fieldDecoder := range fields {
+			fieldHash := calcHash(fieldName, ctx.caseSensitive())
+			_, known := knownHash[fieldHash]
+			if known {
+				return &generalStructDecoder{typ, fields, false}
+			}
+			knownHash[fieldHash] = struct{}{}
+			if fieldName1 == 0 {
+				fieldName1 = fieldHash
+				fieldDecoder1 = fieldDecoder
+			} else if fieldName2 == 0 {
+				fieldName2 = fieldHash
+				fieldDecoder2 = fieldDecoder
+			} else if fieldName3 == 0 {
+				fieldName3 = fieldHash
+				fieldDecoder3 = fieldDecoder
+			} else if fieldName4 == 0 {
+				fieldName4 = fieldHash
+				fieldDecoder4 = fieldDecoder
+			} else if fieldName5 == 0 {
+				fieldName5 = fieldHash
+				fieldDecoder5 = fieldDecoder
+			} else {
+				fieldName6 = fieldHash
+				fieldDecoder6 = fieldDecoder
+			}
+		}
+		return &sixFieldsStructDecoder{typ,
+			fieldName1, fieldDecoder1,
+			fieldName2, fieldDecoder2,
+			fieldName3, fieldDecoder3,
+			fieldName4, fieldDecoder4,
+			fieldName5, fieldDecoder5,
+			fieldName6, fieldDecoder6}
+	case 7:
+		var fieldName1 int64
+		var fieldName2 int64
+		var fieldName3 int64
+		var fieldName4 int64
+		var fieldName5 int64
+		var fieldName6 int64
+		var fieldName7 int64
+		var fieldDecoder1 *structFieldDecoder
+		var fieldDecoder2 *structFieldDecoder
+		var fieldDecoder3 *structFieldDecoder
+		var fieldDecoder4 *structFieldDecoder
+		var fieldDecoder5 *structFieldDecoder
+		var fieldDecoder6 *structFieldDecoder
+		var fieldDecoder7 *structFieldDecoder
+		for fieldName, fieldDecoder := range fields {
+			fieldHash := calcHash(fieldName, ctx.caseSensitive())
+			_, known := knownHash[fieldHash]
+			if known {
+				return &generalStructDecoder{typ, fields, false}
+			}
+			knownHash[fieldHash] = struct{}{}
+			if fieldName1 == 0 {
+				fieldName1 = fieldHash
+				fieldDecoder1 = fieldDecoder
+			} else if fieldName2 == 0 {
+				fieldName2 = fieldHash
+				fieldDecoder2 = fieldDecoder
+			} else if fieldName3 == 0 {
+				fieldName3 = fieldHash
+				fieldDecoder3 = fieldDecoder
+			} else if fieldName4 == 0 {
+				fieldName4 = fieldHash
+				fieldDecoder4 = fieldDecoder
+			} else if fieldName5 == 0 {
+				fieldName5 = fieldHash
+				fieldDecoder5 = fieldDecoder
+			} else if fieldName6 == 0 {
+				fieldName6 = fieldHash
+				fieldDecoder6 = fieldDecoder
+			} else {
+				fieldName7 = fieldHash
+				fieldDecoder7 = fieldDecoder
+			}
+		}
+		return &sevenFieldsStructDecoder{typ,
+			fieldName1, fieldDecoder1,
+			fieldName2, fieldDecoder2,
+			fieldName3, fieldDecoder3,
+			fieldName4, fieldDecoder4,
+			fieldName5, fieldDecoder5,
+			fieldName6, fieldDecoder6,
+			fieldName7, fieldDecoder7}
+	case 8:
+		var fieldName1 int64
+		var fieldName2 int64
+		var fieldName3 int64
+		var fieldName4 int64
+		var fieldName5 int64
+		var fieldName6 int64
+		var fieldName7 int64
+		var fieldName8 int64
+		var fieldDecoder1 *structFieldDecoder
+		var fieldDecoder2 *structFieldDecoder
+		var fieldDecoder3 *structFieldDecoder
+		var fieldDecoder4 *structFieldDecoder
+		var fieldDecoder5 *structFieldDecoder
+		var fieldDecoder6 *structFieldDecoder
+		var fieldDecoder7 *structFieldDecoder
+		var fieldDecoder8 *structFieldDecoder
+		for fieldName, fieldDecoder := range fields {
+			fieldHash := calcHash(fieldName, ctx.caseSensitive())
+			_, known := knownHash[fieldHash]
+			if known {
+				return &generalStructDecoder{typ, fields, false}
+			}
+			knownHash[fieldHash] = struct{}{}
+			if fieldName1 == 0 {
+				fieldName1 = fieldHash
+				fieldDecoder1 = fieldDecoder
+			} else if fieldName2 == 0 {
+				fieldName2 = fieldHash
+				fieldDecoder2 = fieldDecoder
+			} else if fieldName3 == 0 {
+				fieldName3 = fieldHash
+				fieldDecoder3 = fieldDecoder
+			} else if fieldName4 == 0 {
+				fieldName4 = fieldHash
+				fieldDecoder4 = fieldDecoder
+			} else if fieldName5 == 0 {
+				fieldName5 = fieldHash
+				fieldDecoder5 = fieldDecoder
+			} else if fieldName6 == 0 {
+				fieldName6 = fieldHash
+				fieldDecoder6 = fieldDecoder
+			} else if fieldName7 == 0 {
+				fieldName7 = fieldHash
+				fieldDecoder7 = fieldDecoder
+			} else {
+				fieldName8 = fieldHash
+				fieldDecoder8 = fieldDecoder
+			}
+		}
+		return &eightFieldsStructDecoder{typ,
+			fieldName1, fieldDecoder1,
+			fieldName2, fieldDecoder2,
+			fieldName3, fieldDecoder3,
+			fieldName4, fieldDecoder4,
+			fieldName5, fieldDecoder5,
+			fieldName6, fieldDecoder6,
+			fieldName7, fieldDecoder7,
+			fieldName8, fieldDecoder8}
+	case 9:
+		var fieldName1 int64
+		var fieldName2 int64
+		var fieldName3 int64
+		var fieldName4 int64
+		var fieldName5 int64
+		var fieldName6 int64
+		var fieldName7 int64
+		var fieldName8 int64
+		var fieldName9 int64
+		var fieldDecoder1 *structFieldDecoder
+		var fieldDecoder2 *structFieldDecoder
+		var fieldDecoder3 *structFieldDecoder
+		var fieldDecoder4 *structFieldDecoder
+		var fieldDecoder5 *structFieldDecoder
+		var fieldDecoder6 *structFieldDecoder
+		var fieldDecoder7 *structFieldDecoder
+		var fieldDecoder8 *structFieldDecoder
+		var fieldDecoder9 *structFieldDecoder
+		for fieldName, fieldDecoder := range fields {
+			fieldHash := calcHash(fieldName, ctx.caseSensitive())
+			_, known := knownHash[fieldHash]
+			if known {
+				return &generalStructDecoder{typ, fields, false}
+			}
+			knownHash[fieldHash] = struct{}{}
+			if fieldName1 == 0 {
+				fieldName1 = fieldHash
+				fieldDecoder1 = fieldDecoder
+			} else if fieldName2 == 0 {
+				fieldName2 = fieldHash
+				fieldDecoder2 = fieldDecoder
+			} else if fieldName3 == 0 {
+				fieldName3 = fieldHash
+				fieldDecoder3 = fieldDecoder
+			} else if fieldName4 == 0 {
+				fieldName4 = fieldHash
+				fieldDecoder4 = fieldDecoder
+			} else if fieldName5 == 0 {
+				fieldName5 = fieldHash
+				fieldDecoder5 = fieldDecoder
+			} else if fieldName6 == 0 {
+				fieldName6 = fieldHash
+				fieldDecoder6 = fieldDecoder
+			} else if fieldName7 == 0 {
+				fieldName7 = fieldHash
+				fieldDecoder7 = fieldDecoder
+			} else if fieldName8 == 0 {
+				fieldName8 = fieldHash
+				fieldDecoder8 = fieldDecoder
+			} else {
+				fieldName9 = fieldHash
+				fieldDecoder9 = fieldDecoder
+			}
+		}
+		return &nineFieldsStructDecoder{typ,
+			fieldName1, fieldDecoder1,
+			fieldName2, fieldDecoder2,
+			fieldName3, fieldDecoder3,
+			fieldName4, fieldDecoder4,
+			fieldName5, fieldDecoder5,
+			fieldName6, fieldDecoder6,
+			fieldName7, fieldDecoder7,
+			fieldName8, fieldDecoder8,
+			fieldName9, fieldDecoder9}
+	case 10:
+		var fieldName1 int64
+		var fieldName2 int64
+		var fieldName3 int64
+		var fieldName4 int64
+		var fieldName5 int64
+		var fieldName6 int64
+		var fieldName7 int64
+		var fieldName8 int64
+		var fieldName9 int64
+		var fieldName10 int64
+		var fieldDecoder1 *structFieldDecoder
+		var fieldDecoder2 *structFieldDecoder
+		var fieldDecoder3 *structFieldDecoder
+		var fieldDecoder4 *structFieldDecoder
+		var fieldDecoder5 *structFieldDecoder
+		var fieldDecoder6 *structFieldDecoder
+		var fieldDecoder7 *structFieldDecoder
+		var fieldDecoder8 *structFieldDecoder
+		var fieldDecoder9 *structFieldDecoder
+		var fieldDecoder10 *structFieldDecoder
+		for fieldName, fieldDecoder := range fields {
+			fieldHash := calcHash(fieldName, ctx.caseSensitive())
+			_, known := knownHash[fieldHash]
+			if known {
+				return &generalStructDecoder{typ, fields, false}
+			}
+			knownHash[fieldHash] = struct{}{}
+			if fieldName1 == 0 {
+				fieldName1 = fieldHash
+				fieldDecoder1 = fieldDecoder
+			} else if fieldName2 == 0 {
+				fieldName2 = fieldHash
+				fieldDecoder2 = fieldDecoder
+			} else if fieldName3 == 0 {
+				fieldName3 = fieldHash
+				fieldDecoder3 = fieldDecoder
+			} else if fieldName4 == 0 {
+				fieldName4 = fieldHash
+				fieldDecoder4 = fieldDecoder
+			} else if fieldName5 == 0 {
+				fieldName5 = fieldHash
+				fieldDecoder5 = fieldDecoder
+			} else if fieldName6 == 0 {
+				fieldName6 = fieldHash
+				fieldDecoder6 = fieldDecoder
+			} else if fieldName7 == 0 {
+				fieldName7 = fieldHash
+				fieldDecoder7 = fieldDecoder
+			} else if fieldName8 == 0 {
+				fieldName8 = fieldHash
+				fieldDecoder8 = fieldDecoder
+			} else if fieldName9 == 0 {
+				fieldName9 = fieldHash
+				fieldDecoder9 = fieldDecoder
+			} else {
+				fieldName10 = fieldHash
+				fieldDecoder10 = fieldDecoder
+			}
+		}
+		return &tenFieldsStructDecoder{typ,
+			fieldName1, fieldDecoder1,
+			fieldName2, fieldDecoder2,
+			fieldName3, fieldDecoder3,
+			fieldName4, fieldDecoder4,
+			fieldName5, fieldDecoder5,
+			fieldName6, fieldDecoder6,
+			fieldName7, fieldDecoder7,
+			fieldName8, fieldDecoder8,
+			fieldName9, fieldDecoder9,
+			fieldName10, fieldDecoder10}
+	}
+	return &generalStructDecoder{typ, fields, false}
+}
+
+type generalStructDecoder struct {
+	typ                   reflect2.Type
+	fields                map[string]*structFieldDecoder
+	disallowUnknownFields bool
+}
+
+func (decoder *generalStructDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.readObjectStart() {
+		return
+	}
+	var c byte
+	for c = ','; c == ','; c = iter.nextToken() {
+		decoder.decodeOneField(ptr, iter)
+	}
+	if iter.Error != nil && iter.Error != io.EOF {
+		iter.Error = fmt.Errorf("%v.%s", decoder.typ, iter.Error.Error())
+	}
+	if c != '}' {
+		iter.ReportError("struct Decode", `expect }, but found `+string([]byte{c}))
+	}
+}
+
+func (decoder *generalStructDecoder) decodeOneField(ptr unsafe.Pointer, iter *Iterator) {
+	var field string
+	var fieldDecoder *structFieldDecoder
+	if iter.cfg.objectFieldMustBeSimpleString {
+		fieldBytes := iter.ReadStringAsSlice()
+		field = *(*string)(unsafe.Pointer(&fieldBytes))
+		fieldDecoder = decoder.fields[field]
+		if fieldDecoder == nil && !iter.cfg.caseSensitive {
+			fieldDecoder = decoder.fields[strings.ToLower(field)]
+		}
+	} else {
+		field = iter.ReadString()
+		fieldDecoder = decoder.fields[field]
+		if fieldDecoder == nil && !iter.cfg.caseSensitive {
+			fieldDecoder = decoder.fields[strings.ToLower(field)]
+		}
+	}
+	if fieldDecoder == nil {
+		if decoder.disallowUnknownFields {
+			msg := "found unknown field: " + field
+			iter.ReportError("ReadObject", msg)
+		}
+		c := iter.nextToken()
+		if c != ':' {
+			iter.ReportError("ReadObject", "expect : after object field, but found "+string([]byte{c}))
+		}
+		iter.Skip()
+		return
+	}
+	c := iter.nextToken()
+	if c != ':' {
+		iter.ReportError("ReadObject", "expect : after object field, but found "+string([]byte{c}))
+	}
+	fieldDecoder.Decode(ptr, iter)
+}
+
+type skipObjectDecoder struct {
+	typ reflect2.Type
+}
+
+func (decoder *skipObjectDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	valueType := iter.WhatIsNext()
+	if valueType != ObjectValue && valueType != NilValue {
+		iter.ReportError("skipObjectDecoder", "expect object or null")
+		return
+	}
+	iter.Skip()
+}
+
+type oneFieldStructDecoder struct {
+	typ          reflect2.Type
+	fieldHash    int64
+	fieldDecoder *structFieldDecoder
+}
+
+func (decoder *oneFieldStructDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.readObjectStart() {
+		return
+	}
+	for {
+		if iter.readFieldHash() == decoder.fieldHash {
+			decoder.fieldDecoder.Decode(ptr, iter)
+		} else {
+			iter.Skip()
+		}
+		if iter.isObjectEnd() {
+			break
+		}
+	}
+	if iter.Error != nil && iter.Error != io.EOF {
+		iter.Error = fmt.Errorf("%v.%s", decoder.typ, iter.Error.Error())
+	}
+}
+
+type twoFieldsStructDecoder struct {
+	typ           reflect2.Type
+	fieldHash1    int64
+	fieldDecoder1 *structFieldDecoder
+	fieldHash2    int64
+	fieldDecoder2 *structFieldDecoder
+}
+
+func (decoder *twoFieldsStructDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.readObjectStart() {
+		return
+	}
+	for {
+		switch iter.readFieldHash() {
+		case decoder.fieldHash1:
+			decoder.fieldDecoder1.Decode(ptr, iter)
+		case decoder.fieldHash2:
+			decoder.fieldDecoder2.Decode(ptr, iter)
+		default:
+			iter.Skip()
+		}
+		if iter.isObjectEnd() {
+			break
+		}
+	}
+	if iter.Error != nil && iter.Error != io.EOF {
+		iter.Error = fmt.Errorf("%v.%s", decoder.typ, iter.Error.Error())
+	}
+}
+
+type threeFieldsStructDecoder struct {
+	typ           reflect2.Type
+	fieldHash1    int64
+	fieldDecoder1 *structFieldDecoder
+	fieldHash2    int64
+	fieldDecoder2 *structFieldDecoder
+	fieldHash3    int64
+	fieldDecoder3 *structFieldDecoder
+}
+
+func (decoder *threeFieldsStructDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.readObjectStart() {
+		return
+	}
+	for {
+		switch iter.readFieldHash() {
+		case decoder.fieldHash1:
+			decoder.fieldDecoder1.Decode(ptr, iter)
+		case decoder.fieldHash2:
+			decoder.fieldDecoder2.Decode(ptr, iter)
+		case decoder.fieldHash3:
+			decoder.fieldDecoder3.Decode(ptr, iter)
+		default:
+			iter.Skip()
+		}
+		if iter.isObjectEnd() {
+			break
+		}
+	}
+	if iter.Error != nil && iter.Error != io.EOF {
+		iter.Error = fmt.Errorf("%v.%s", decoder.typ, iter.Error.Error())
+	}
+}
+
+type fourFieldsStructDecoder struct {
+	typ           reflect2.Type
+	fieldHash1    int64
+	fieldDecoder1 *structFieldDecoder
+	fieldHash2    int64
+	fieldDecoder2 *structFieldDecoder
+	fieldHash3    int64
+	fieldDecoder3 *structFieldDecoder
+	fieldHash4    int64
+	fieldDecoder4 *structFieldDecoder
+}
+
+func (decoder *fourFieldsStructDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.readObjectStart() {
+		return
+	}
+	for {
+		switch iter.readFieldHash() {
+		case decoder.fieldHash1:
+			decoder.fieldDecoder1.Decode(ptr, iter)
+		case decoder.fieldHash2:
+			decoder.fieldDecoder2.Decode(ptr, iter)
+		case decoder.fieldHash3:
+			decoder.fieldDecoder3.Decode(ptr, iter)
+		case decoder.fieldHash4:
+			decoder.fieldDecoder4.Decode(ptr, iter)
+		default:
+			iter.Skip()
+		}
+		if iter.isObjectEnd() {
+			break
+		}
+	}
+	if iter.Error != nil && iter.Error != io.EOF {
+		iter.Error = fmt.Errorf("%v.%s", decoder.typ, iter.Error.Error())
+	}
+}
+
+type fiveFieldsStructDecoder struct {
+	typ           reflect2.Type
+	fieldHash1    int64
+	fieldDecoder1 *structFieldDecoder
+	fieldHash2    int64
+	fieldDecoder2 *structFieldDecoder
+	fieldHash3    int64
+	fieldDecoder3 *structFieldDecoder
+	fieldHash4    int64
+	fieldDecoder4 *structFieldDecoder
+	fieldHash5    int64
+	fieldDecoder5 *structFieldDecoder
+}
+
+func (decoder *fiveFieldsStructDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.readObjectStart() {
+		return
+	}
+	for {
+		switch iter.readFieldHash() {
+		case decoder.fieldHash1:
+			decoder.fieldDecoder1.Decode(ptr, iter)
+		case decoder.fieldHash2:
+			decoder.fieldDecoder2.Decode(ptr, iter)
+		case decoder.fieldHash3:
+			decoder.fieldDecoder3.Decode(ptr, iter)
+		case decoder.fieldHash4:
+			decoder.fieldDecoder4.Decode(ptr, iter)
+		case decoder.fieldHash5:
+			decoder.fieldDecoder5.Decode(ptr, iter)
+		default:
+			iter.Skip()
+		}
+		if iter.isObjectEnd() {
+			break
+		}
+	}
+	if iter.Error != nil && iter.Error != io.EOF {
+		iter.Error = fmt.Errorf("%v.%s", decoder.typ, iter.Error.Error())
+	}
+}
+
+type sixFieldsStructDecoder struct {
+	typ           reflect2.Type
+	fieldHash1    int64
+	fieldDecoder1 *structFieldDecoder
+	fieldHash2    int64
+	fieldDecoder2 *structFieldDecoder
+	fieldHash3    int64
+	fieldDecoder3 *structFieldDecoder
+	fieldHash4    int64
+	fieldDecoder4 *structFieldDecoder
+	fieldHash5    int64
+	fieldDecoder5 *structFieldDecoder
+	fieldHash6    int64
+	fieldDecoder6 *structFieldDecoder
+}
+
+func (decoder *sixFieldsStructDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.readObjectStart() {
+		return
+	}
+	for {
+		switch iter.readFieldHash() {
+		case decoder.fieldHash1:
+			decoder.fieldDecoder1.Decode(ptr, iter)
+		case decoder.fieldHash2:
+			decoder.fieldDecoder2.Decode(ptr, iter)
+		case decoder.fieldHash3:
+			decoder.fieldDecoder3.Decode(ptr, iter)
+		case decoder.fieldHash4:
+			decoder.fieldDecoder4.Decode(ptr, iter)
+		case decoder.fieldHash5:
+			decoder.fieldDecoder5.Decode(ptr, iter)
+		case decoder.fieldHash6:
+			decoder.fieldDecoder6.Decode(ptr, iter)
+		default:
+			iter.Skip()
+		}
+		if iter.isObjectEnd() {
+			break
+		}
+	}
+	if iter.Error != nil && iter.Error != io.EOF {
+		iter.Error = fmt.Errorf("%v.%s", decoder.typ, iter.Error.Error())
+	}
+}
+
+type sevenFieldsStructDecoder struct {
+	typ           reflect2.Type
+	fieldHash1    int64
+	fieldDecoder1 *structFieldDecoder
+	fieldHash2    int64
+	fieldDecoder2 *structFieldDecoder
+	fieldHash3    int64
+	fieldDecoder3 *structFieldDecoder
+	fieldHash4    int64
+	fieldDecoder4 *structFieldDecoder
+	fieldHash5    int64
+	fieldDecoder5 *structFieldDecoder
+	fieldHash6    int64
+	fieldDecoder6 *structFieldDecoder
+	fieldHash7    int64
+	fieldDecoder7 *structFieldDecoder
+}
+
+func (decoder *sevenFieldsStructDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.readObjectStart() {
+		return
+	}
+	for {
+		switch iter.readFieldHash() {
+		case decoder.fieldHash1:
+			decoder.fieldDecoder1.Decode(ptr, iter)
+		case decoder.fieldHash2:
+			decoder.fieldDecoder2.Decode(ptr, iter)
+		case decoder.fieldHash3:
+			decoder.fieldDecoder3.Decode(ptr, iter)
+		case decoder.fieldHash4:
+			decoder.fieldDecoder4.Decode(ptr, iter)
+		case decoder.fieldHash5:
+			decoder.fieldDecoder5.Decode(ptr, iter)
+		case decoder.fieldHash6:
+			decoder.fieldDecoder6.Decode(ptr, iter)
+		case decoder.fieldHash7:
+			decoder.fieldDecoder7.Decode(ptr, iter)
+		default:
+			iter.Skip()
+		}
+		if iter.isObjectEnd() {
+			break
+		}
+	}
+	if iter.Error != nil && iter.Error != io.EOF {
+		iter.Error = fmt.Errorf("%v.%s", decoder.typ, iter.Error.Error())
+	}
+}
+
+type eightFieldsStructDecoder struct {
+	typ           reflect2.Type
+	fieldHash1    int64
+	fieldDecoder1 *structFieldDecoder
+	fieldHash2    int64
+	fieldDecoder2 *structFieldDecoder
+	fieldHash3    int64
+	fieldDecoder3 *structFieldDecoder
+	fieldHash4    int64
+	fieldDecoder4 *structFieldDecoder
+	fieldHash5    int64
+	fieldDecoder5 *structFieldDecoder
+	fieldHash6    int64
+	fieldDecoder6 *structFieldDecoder
+	fieldHash7    int64
+	fieldDecoder7 *structFieldDecoder
+	fieldHash8    int64
+	fieldDecoder8 *structFieldDecoder
+}
+
+func (decoder *eightFieldsStructDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.readObjectStart() {
+		return
+	}
+	for {
+		switch iter.readFieldHash() {
+		case decoder.fieldHash1:
+			decoder.fieldDecoder1.Decode(ptr, iter)
+		case decoder.fieldHash2:
+			decoder.fieldDecoder2.Decode(ptr, iter)
+		case decoder.fieldHash3:
+			decoder.fieldDecoder3.Decode(ptr, iter)
+		case decoder.fieldHash4:
+			decoder.fieldDecoder4.Decode(ptr, iter)
+		case decoder.fieldHash5:
+			decoder.fieldDecoder5.Decode(ptr, iter)
+		case decoder.fieldHash6:
+			decoder.fieldDecoder6.Decode(ptr, iter)
+		case decoder.fieldHash7:
+			decoder.fieldDecoder7.Decode(ptr, iter)
+		case decoder.fieldHash8:
+			decoder.fieldDecoder8.Decode(ptr, iter)
+		default:
+			iter.Skip()
+		}
+		if iter.isObjectEnd() {
+			break
+		}
+	}
+	if iter.Error != nil && iter.Error != io.EOF {
+		iter.Error = fmt.Errorf("%v.%s", decoder.typ, iter.Error.Error())
+	}
+}
+
+type nineFieldsStructDecoder struct {
+	typ           reflect2.Type
+	fieldHash1    int64
+	fieldDecoder1 *structFieldDecoder
+	fieldHash2    int64
+	fieldDecoder2 *structFieldDecoder
+	fieldHash3    int64
+	fieldDecoder3 *structFieldDecoder
+	fieldHash4    int64
+	fieldDecoder4 *structFieldDecoder
+	fieldHash5    int64
+	fieldDecoder5 *structFieldDecoder
+	fieldHash6    int64
+	fieldDecoder6 *structFieldDecoder
+	fieldHash7    int64
+	fieldDecoder7 *structFieldDecoder
+	fieldHash8    int64
+	fieldDecoder8 *structFieldDecoder
+	fieldHash9    int64
+	fieldDecoder9 *structFieldDecoder
+}
+
+func (decoder *nineFieldsStructDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.readObjectStart() {
+		return
+	}
+	for {
+		switch iter.readFieldHash() {
+		case decoder.fieldHash1:
+			decoder.fieldDecoder1.Decode(ptr, iter)
+		case decoder.fieldHash2:
+			decoder.fieldDecoder2.Decode(ptr, iter)
+		case decoder.fieldHash3:
+			decoder.fieldDecoder3.Decode(ptr, iter)
+		case decoder.fieldHash4:
+			decoder.fieldDecoder4.Decode(ptr, iter)
+		case decoder.fieldHash5:
+			decoder.fieldDecoder5.Decode(ptr, iter)
+		case decoder.fieldHash6:
+			decoder.fieldDecoder6.Decode(ptr, iter)
+		case decoder.fieldHash7:
+			decoder.fieldDecoder7.Decode(ptr, iter)
+		case decoder.fieldHash8:
+			decoder.fieldDecoder8.Decode(ptr, iter)
+		case decoder.fieldHash9:
+			decoder.fieldDecoder9.Decode(ptr, iter)
+		default:
+			iter.Skip()
+		}
+		if iter.isObjectEnd() {
+			break
+		}
+	}
+	if iter.Error != nil && iter.Error != io.EOF {
+		iter.Error = fmt.Errorf("%v.%s", decoder.typ, iter.Error.Error())
+	}
+}
+
+type tenFieldsStructDecoder struct {
+	typ            reflect2.Type
+	fieldHash1     int64
+	fieldDecoder1  *structFieldDecoder
+	fieldHash2     int64
+	fieldDecoder2  *structFieldDecoder
+	fieldHash3     int64
+	fieldDecoder3  *structFieldDecoder
+	fieldHash4     int64
+	fieldDecoder4  *structFieldDecoder
+	fieldHash5     int64
+	fieldDecoder5  *structFieldDecoder
+	fieldHash6     int64
+	fieldDecoder6  *structFieldDecoder
+	fieldHash7     int64
+	fieldDecoder7  *structFieldDecoder
+	fieldHash8     int64
+	fieldDecoder8  *structFieldDecoder
+	fieldHash9     int64
+	fieldDecoder9  *structFieldDecoder
+	fieldHash10    int64
+	fieldDecoder10 *structFieldDecoder
+}
+
+func (decoder *tenFieldsStructDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	if !iter.readObjectStart() {
+		return
+	}
+	for {
+		switch iter.readFieldHash() {
+		case decoder.fieldHash1:
+			decoder.fieldDecoder1.Decode(ptr, iter)
+		case decoder.fieldHash2:
+			decoder.fieldDecoder2.Decode(ptr, iter)
+		case decoder.fieldHash3:
+			decoder.fieldDecoder3.Decode(ptr, iter)
+		case decoder.fieldHash4:
+			decoder.fieldDecoder4.Decode(ptr, iter)
+		case decoder.fieldHash5:
+			decoder.fieldDecoder5.Decode(ptr, iter)
+		case decoder.fieldHash6:
+			decoder.fieldDecoder6.Decode(ptr, iter)
+		case decoder.fieldHash7:
+			decoder.fieldDecoder7.Decode(ptr, iter)
+		case decoder.fieldHash8:
+			decoder.fieldDecoder8.Decode(ptr, iter)
+		case decoder.fieldHash9:
+			decoder.fieldDecoder9.Decode(ptr, iter)
+		case decoder.fieldHash10:
+			decoder.fieldDecoder10.Decode(ptr, iter)
+		default:
+			iter.Skip()
+		}
+		if iter.isObjectEnd() {
+			break
+		}
+	}
+	if iter.Error != nil && iter.Error != io.EOF {
+		iter.Error = fmt.Errorf("%v.%s", decoder.typ, iter.Error.Error())
+	}
+}
+
+type structFieldDecoder struct {
+	field        reflect2.StructField
+	fieldDecoder ValDecoder
+}
+
+func (decoder *structFieldDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	fieldPtr := decoder.field.UnsafeGet(ptr)
+	decoder.fieldDecoder.Decode(fieldPtr, iter)
+	if iter.Error != nil && iter.Error != io.EOF {
+		iter.Error = fmt.Errorf("%s: %s", decoder.field.Name(), iter.Error.Error())
+	}
+}
+
+type stringModeStringDecoder struct {
+	elemDecoder ValDecoder
+	cfg         *frozenConfig
+}
+
+func (decoder *stringModeStringDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	decoder.elemDecoder.Decode(ptr, iter)
+	str := *((*string)(ptr))
+	tempIter := decoder.cfg.BorrowIterator([]byte(str))
+	defer decoder.cfg.ReturnIterator(tempIter)
+	*((*string)(ptr)) = tempIter.ReadString()
+}
+
+type stringModeNumberDecoder struct {
+	elemDecoder ValDecoder
+}
+
+func (decoder *stringModeNumberDecoder) Decode(ptr unsafe.Pointer, iter *Iterator) {
+	c := iter.nextToken()
+	if c != '"' {
+		iter.ReportError("stringModeNumberDecoder", `expect ", but found `+string([]byte{c}))
+		return
+	}
+	decoder.elemDecoder.Decode(ptr, iter)
+	if iter.Error != nil {
+		return
+	}
+	c = iter.readByte()
+	if c != '"' {
+		iter.ReportError("stringModeNumberDecoder", `expect ", but found `+string([]byte{c}))
+		return
+	}
+}
diff --git a/vendor/github.com/json-iterator/go/reflect_struct_encoder.go b/vendor/github.com/json-iterator/go/reflect_struct_encoder.go
new file mode 100644
index 0000000..d0759cf
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/reflect_struct_encoder.go
@@ -0,0 +1,210 @@
+package jsoniter
+
+import (
+	"fmt"
+	"github.com/modern-go/reflect2"
+	"io"
+	"reflect"
+	"unsafe"
+)
+
+func encoderOfStruct(ctx *ctx, typ reflect2.Type) ValEncoder {
+	type bindingTo struct {
+		binding *Binding
+		toName  string
+		ignored bool
+	}
+	orderedBindings := []*bindingTo{}
+	structDescriptor := describeStruct(ctx, typ)
+	for _, binding := range structDescriptor.Fields {
+		for _, toName := range binding.ToNames {
+			new := &bindingTo{
+				binding: binding,
+				toName:  toName,
+			}
+			for _, old := range orderedBindings {
+				if old.toName != toName {
+					continue
+				}
+				old.ignored, new.ignored = resolveConflictBinding(ctx.frozenConfig, old.binding, new.binding)
+			}
+			orderedBindings = append(orderedBindings, new)
+		}
+	}
+	if len(orderedBindings) == 0 {
+		return &emptyStructEncoder{}
+	}
+	finalOrderedFields := []structFieldTo{}
+	for _, bindingTo := range orderedBindings {
+		if !bindingTo.ignored {
+			finalOrderedFields = append(finalOrderedFields, structFieldTo{
+				encoder: bindingTo.binding.Encoder.(*structFieldEncoder),
+				toName:  bindingTo.toName,
+			})
+		}
+	}
+	return &structEncoder{typ, finalOrderedFields}
+}
+
+func createCheckIsEmpty(ctx *ctx, typ reflect2.Type) checkIsEmpty {
+	encoder := createEncoderOfNative(ctx, typ)
+	if encoder != nil {
+		return encoder
+	}
+	kind := typ.Kind()
+	switch kind {
+	case reflect.Interface:
+		return &dynamicEncoder{typ}
+	case reflect.Struct:
+		return &structEncoder{typ: typ}
+	case reflect.Array:
+		return &arrayEncoder{}
+	case reflect.Slice:
+		return &sliceEncoder{}
+	case reflect.Map:
+		return encoderOfMap(ctx, typ)
+	case reflect.Ptr:
+		return &OptionalEncoder{}
+	default:
+		return &lazyErrorEncoder{err: fmt.Errorf("unsupported type: %v", typ)}
+	}
+}
+
+func resolveConflictBinding(cfg *frozenConfig, old, new *Binding) (ignoreOld, ignoreNew bool) {
+	newTagged := new.Field.Tag().Get(cfg.getTagKey()) != ""
+	oldTagged := old.Field.Tag().Get(cfg.getTagKey()) != ""
+	if newTagged {
+		if oldTagged {
+			if len(old.levels) > len(new.levels) {
+				return true, false
+			} else if len(new.levels) > len(old.levels) {
+				return false, true
+			} else {
+				return true, true
+			}
+		} else {
+			return true, false
+		}
+	} else {
+		if oldTagged {
+			return true, false
+		}
+		if len(old.levels) > len(new.levels) {
+			return true, false
+		} else if len(new.levels) > len(old.levels) {
+			return false, true
+		} else {
+			return true, true
+		}
+	}
+}
+
+type structFieldEncoder struct {
+	field        reflect2.StructField
+	fieldEncoder ValEncoder
+	omitempty    bool
+}
+
+func (encoder *structFieldEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	fieldPtr := encoder.field.UnsafeGet(ptr)
+	encoder.fieldEncoder.Encode(fieldPtr, stream)
+	if stream.Error != nil && stream.Error != io.EOF {
+		stream.Error = fmt.Errorf("%s: %s", encoder.field.Name(), stream.Error.Error())
+	}
+}
+
+func (encoder *structFieldEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	fieldPtr := encoder.field.UnsafeGet(ptr)
+	return encoder.fieldEncoder.IsEmpty(fieldPtr)
+}
+
+func (encoder *structFieldEncoder) IsEmbeddedPtrNil(ptr unsafe.Pointer) bool {
+	isEmbeddedPtrNil, converted := encoder.fieldEncoder.(IsEmbeddedPtrNil)
+	if !converted {
+		return false
+	}
+	fieldPtr := encoder.field.UnsafeGet(ptr)
+	return isEmbeddedPtrNil.IsEmbeddedPtrNil(fieldPtr)
+}
+
+type IsEmbeddedPtrNil interface {
+	IsEmbeddedPtrNil(ptr unsafe.Pointer) bool
+}
+
+type structEncoder struct {
+	typ    reflect2.Type
+	fields []structFieldTo
+}
+
+type structFieldTo struct {
+	encoder *structFieldEncoder
+	toName  string
+}
+
+func (encoder *structEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteObjectStart()
+	isNotFirst := false
+	for _, field := range encoder.fields {
+		if field.encoder.omitempty && field.encoder.IsEmpty(ptr) {
+			continue
+		}
+		if field.encoder.IsEmbeddedPtrNil(ptr) {
+			continue
+		}
+		if isNotFirst {
+			stream.WriteMore()
+		}
+		stream.WriteObjectField(field.toName)
+		field.encoder.Encode(ptr, stream)
+		isNotFirst = true
+	}
+	stream.WriteObjectEnd()
+	if stream.Error != nil && stream.Error != io.EOF {
+		stream.Error = fmt.Errorf("%v.%s", encoder.typ, stream.Error.Error())
+	}
+}
+
+func (encoder *structEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return false
+}
+
+type emptyStructEncoder struct {
+}
+
+func (encoder *emptyStructEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.WriteEmptyObject()
+}
+
+func (encoder *emptyStructEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return false
+}
+
+type stringModeNumberEncoder struct {
+	elemEncoder ValEncoder
+}
+
+func (encoder *stringModeNumberEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	stream.writeByte('"')
+	encoder.elemEncoder.Encode(ptr, stream)
+	stream.writeByte('"')
+}
+
+func (encoder *stringModeNumberEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return encoder.elemEncoder.IsEmpty(ptr)
+}
+
+type stringModeStringEncoder struct {
+	elemEncoder ValEncoder
+	cfg         *frozenConfig
+}
+
+func (encoder *stringModeStringEncoder) Encode(ptr unsafe.Pointer, stream *Stream) {
+	tempStream := encoder.cfg.BorrowStream(nil)
+	defer encoder.cfg.ReturnStream(tempStream)
+	encoder.elemEncoder.Encode(ptr, tempStream)
+	stream.WriteString(string(tempStream.Buffer()))
+}
+
+func (encoder *stringModeStringEncoder) IsEmpty(ptr unsafe.Pointer) bool {
+	return encoder.elemEncoder.IsEmpty(ptr)
+}
diff --git a/vendor/github.com/json-iterator/go/stream.go b/vendor/github.com/json-iterator/go/stream.go
new file mode 100644
index 0000000..17662fd
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/stream.go
@@ -0,0 +1,211 @@
+package jsoniter
+
+import (
+	"io"
+)
+
+// stream is a io.Writer like object, with JSON specific write functions.
+// Error is not returned as return value, but stored as Error member on this stream instance.
+type Stream struct {
+	cfg        *frozenConfig
+	out        io.Writer
+	buf        []byte
+	Error      error
+	indention  int
+	Attachment interface{} // open for customized encoder
+}
+
+// NewStream create new stream instance.
+// cfg can be jsoniter.ConfigDefault.
+// out can be nil if write to internal buffer.
+// bufSize is the initial size for the internal buffer in bytes.
+func NewStream(cfg API, out io.Writer, bufSize int) *Stream {
+	return &Stream{
+		cfg:       cfg.(*frozenConfig),
+		out:       out,
+		buf:       make([]byte, 0, bufSize),
+		Error:     nil,
+		indention: 0,
+	}
+}
+
+// Pool returns a pool can provide more stream with same configuration
+func (stream *Stream) Pool() StreamPool {
+	return stream.cfg
+}
+
+// Reset reuse this stream instance by assign a new writer
+func (stream *Stream) Reset(out io.Writer) {
+	stream.out = out
+	stream.buf = stream.buf[:0]
+}
+
+// Available returns how many bytes are unused in the buffer.
+func (stream *Stream) Available() int {
+	return cap(stream.buf) - len(stream.buf)
+}
+
+// Buffered returns the number of bytes that have been written into the current buffer.
+func (stream *Stream) Buffered() int {
+	return len(stream.buf)
+}
+
+// Buffer if writer is nil, use this method to take the result
+func (stream *Stream) Buffer() []byte {
+	return stream.buf
+}
+
+// SetBuffer allows to append to the internal buffer directly
+func (stream *Stream) SetBuffer(buf []byte) {
+	stream.buf = buf
+}
+
+// Write writes the contents of p into the buffer.
+// It returns the number of bytes written.
+// If nn < len(p), it also returns an error explaining
+// why the write is short.
+func (stream *Stream) Write(p []byte) (nn int, err error) {
+	stream.buf = append(stream.buf, p...)
+	if stream.out != nil {
+		nn, err = stream.out.Write(stream.buf)
+		stream.buf = stream.buf[nn:]
+		return
+	}
+	return len(p), nil
+}
+
+// WriteByte writes a single byte.
+func (stream *Stream) writeByte(c byte) {
+	stream.buf = append(stream.buf, c)
+}
+
+func (stream *Stream) writeTwoBytes(c1 byte, c2 byte) {
+	stream.buf = append(stream.buf, c1, c2)
+}
+
+func (stream *Stream) writeThreeBytes(c1 byte, c2 byte, c3 byte) {
+	stream.buf = append(stream.buf, c1, c2, c3)
+}
+
+func (stream *Stream) writeFourBytes(c1 byte, c2 byte, c3 byte, c4 byte) {
+	stream.buf = append(stream.buf, c1, c2, c3, c4)
+}
+
+func (stream *Stream) writeFiveBytes(c1 byte, c2 byte, c3 byte, c4 byte, c5 byte) {
+	stream.buf = append(stream.buf, c1, c2, c3, c4, c5)
+}
+
+// Flush writes any buffered data to the underlying io.Writer.
+func (stream *Stream) Flush() error {
+	if stream.out == nil {
+		return nil
+	}
+	if stream.Error != nil {
+		return stream.Error
+	}
+	n, err := stream.out.Write(stream.buf)
+	if err != nil {
+		if stream.Error == nil {
+			stream.Error = err
+		}
+		return err
+	}
+	stream.buf = stream.buf[n:]
+	return nil
+}
+
+// WriteRaw write string out without quotes, just like []byte
+func (stream *Stream) WriteRaw(s string) {
+	stream.buf = append(stream.buf, s...)
+}
+
+// WriteNil write null to stream
+func (stream *Stream) WriteNil() {
+	stream.writeFourBytes('n', 'u', 'l', 'l')
+}
+
+// WriteTrue write true to stream
+func (stream *Stream) WriteTrue() {
+	stream.writeFourBytes('t', 'r', 'u', 'e')
+}
+
+// WriteFalse write false to stream
+func (stream *Stream) WriteFalse() {
+	stream.writeFiveBytes('f', 'a', 'l', 's', 'e')
+}
+
+// WriteBool write true or false into stream
+func (stream *Stream) WriteBool(val bool) {
+	if val {
+		stream.WriteTrue()
+	} else {
+		stream.WriteFalse()
+	}
+}
+
+// WriteObjectStart write { with possible indention
+func (stream *Stream) WriteObjectStart() {
+	stream.indention += stream.cfg.indentionStep
+	stream.writeByte('{')
+	stream.writeIndention(0)
+}
+
+// WriteObjectField write "field": with possible indention
+func (stream *Stream) WriteObjectField(field string) {
+	stream.WriteString(field)
+	if stream.indention > 0 {
+		stream.writeTwoBytes(':', ' ')
+	} else {
+		stream.writeByte(':')
+	}
+}
+
+// WriteObjectEnd write } with possible indention
+func (stream *Stream) WriteObjectEnd() {
+	stream.writeIndention(stream.cfg.indentionStep)
+	stream.indention -= stream.cfg.indentionStep
+	stream.writeByte('}')
+}
+
+// WriteEmptyObject write {}
+func (stream *Stream) WriteEmptyObject() {
+	stream.writeByte('{')
+	stream.writeByte('}')
+}
+
+// WriteMore write , with possible indention
+func (stream *Stream) WriteMore() {
+	stream.writeByte(',')
+	stream.writeIndention(0)
+	stream.Flush()
+}
+
+// WriteArrayStart write [ with possible indention
+func (stream *Stream) WriteArrayStart() {
+	stream.indention += stream.cfg.indentionStep
+	stream.writeByte('[')
+	stream.writeIndention(0)
+}
+
+// WriteEmptyArray write []
+func (stream *Stream) WriteEmptyArray() {
+	stream.writeTwoBytes('[', ']')
+}
+
+// WriteArrayEnd write ] with possible indention
+func (stream *Stream) WriteArrayEnd() {
+	stream.writeIndention(stream.cfg.indentionStep)
+	stream.indention -= stream.cfg.indentionStep
+	stream.writeByte(']')
+}
+
+func (stream *Stream) writeIndention(delta int) {
+	if stream.indention == 0 {
+		return
+	}
+	stream.writeByte('\n')
+	toWrite := stream.indention - delta
+	for i := 0; i < toWrite; i++ {
+		stream.buf = append(stream.buf, ' ')
+	}
+}
diff --git a/vendor/github.com/json-iterator/go/stream_float.go b/vendor/github.com/json-iterator/go/stream_float.go
new file mode 100644
index 0000000..826aa59
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/stream_float.go
@@ -0,0 +1,111 @@
+package jsoniter
+
+import (
+	"fmt"
+	"math"
+	"strconv"
+)
+
+var pow10 []uint64
+
+func init() {
+	pow10 = []uint64{1, 10, 100, 1000, 10000, 100000, 1000000}
+}
+
+// WriteFloat32 write float32 to stream
+func (stream *Stream) WriteFloat32(val float32) {
+	if math.IsInf(float64(val), 0) || math.IsNaN(float64(val)) {
+		stream.Error = fmt.Errorf("unsupported value: %f", val)
+		return
+	}
+	abs := math.Abs(float64(val))
+	fmt := byte('f')
+	// Note: Must use float32 comparisons for underlying float32 value to get precise cutoffs right.
+	if abs != 0 {
+		if float32(abs) < 1e-6 || float32(abs) >= 1e21 {
+			fmt = 'e'
+		}
+	}
+	stream.buf = strconv.AppendFloat(stream.buf, float64(val), fmt, -1, 32)
+}
+
+// WriteFloat32Lossy write float32 to stream with ONLY 6 digits precision although much much faster
+func (stream *Stream) WriteFloat32Lossy(val float32) {
+	if math.IsInf(float64(val), 0) || math.IsNaN(float64(val)) {
+		stream.Error = fmt.Errorf("unsupported value: %f", val)
+		return
+	}
+	if val < 0 {
+		stream.writeByte('-')
+		val = -val
+	}
+	if val > 0x4ffffff {
+		stream.WriteFloat32(val)
+		return
+	}
+	precision := 6
+	exp := uint64(1000000) // 6
+	lval := uint64(float64(val)*float64(exp) + 0.5)
+	stream.WriteUint64(lval / exp)
+	fval := lval % exp
+	if fval == 0 {
+		return
+	}
+	stream.writeByte('.')
+	for p := precision - 1; p > 0 && fval < pow10[p]; p-- {
+		stream.writeByte('0')
+	}
+	stream.WriteUint64(fval)
+	for stream.buf[len(stream.buf)-1] == '0' {
+		stream.buf = stream.buf[:len(stream.buf)-1]
+	}
+}
+
+// WriteFloat64 write float64 to stream
+func (stream *Stream) WriteFloat64(val float64) {
+	if math.IsInf(val, 0) || math.IsNaN(val) {
+		stream.Error = fmt.Errorf("unsupported value: %f", val)
+		return
+	}
+	abs := math.Abs(val)
+	fmt := byte('f')
+	// Note: Must use float32 comparisons for underlying float32 value to get precise cutoffs right.
+	if abs != 0 {
+		if abs < 1e-6 || abs >= 1e21 {
+			fmt = 'e'
+		}
+	}
+	stream.buf = strconv.AppendFloat(stream.buf, float64(val), fmt, -1, 64)
+}
+
+// WriteFloat64Lossy write float64 to stream with ONLY 6 digits precision although much much faster
+func (stream *Stream) WriteFloat64Lossy(val float64) {
+	if math.IsInf(val, 0) || math.IsNaN(val) {
+		stream.Error = fmt.Errorf("unsupported value: %f", val)
+		return
+	}
+	if val < 0 {
+		stream.writeByte('-')
+		val = -val
+	}
+	if val > 0x4ffffff {
+		stream.WriteFloat64(val)
+		return
+	}
+	precision := 6
+	exp := uint64(1000000) // 6
+	lval := uint64(val*float64(exp) + 0.5)
+	stream.WriteUint64(lval / exp)
+	fval := lval % exp
+	if fval == 0 {
+		return
+	}
+	stream.writeByte('.')
+	for p := precision - 1; p > 0 && fval < pow10[p]; p-- {
+		stream.writeByte('0')
+	}
+	stream.WriteUint64(fval)
+	for stream.buf[len(stream.buf)-1] == '0' {
+		stream.buf = stream.buf[:len(stream.buf)-1]
+	}
+}
diff --git a/vendor/github.com/json-iterator/go/stream_int.go b/vendor/github.com/json-iterator/go/stream_int.go
new file mode 100644
index 0000000..d1059ee
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/stream_int.go
@@ -0,0 +1,190 @@
+package jsoniter
+
+var digits []uint32
+
+func init() {
+	digits = make([]uint32, 1000)
+	for i := uint32(0); i < 1000; i++ {
+		digits[i] = (((i / 100) + '0') << 16) + ((((i / 10) % 10) + '0') << 8) + i%10 + '0'
+		if i < 10 {
+			digits[i] += 2 << 24
+		} else if i < 100 {
+			digits[i] += 1 << 24
+		}
+	}
+}
+
+func writeFirstBuf(space []byte, v uint32) []byte {
+	start := v >> 24
+	if start == 0 {
+		space = append(space, byte(v>>16), byte(v>>8))
+	} else if start == 1 {
+		space = append(space, byte(v>>8))
+	}
+	space = append(space, byte(v))
+	return space
+}
+
+func writeBuf(buf []byte, v uint32) []byte {
+	return append(buf, byte(v>>16), byte(v>>8), byte(v))
+}
+
+// WriteUint8 write uint8 to stream
+func (stream *Stream) WriteUint8(val uint8) {
+	stream.buf = writeFirstBuf(stream.buf, digits[val])
+}
+
+// WriteInt8 write int8 to stream
+func (stream *Stream) WriteInt8(nval int8) {
+	var val uint8
+	if nval < 0 {
+		val = uint8(-nval)
+		stream.buf = append(stream.buf, '-')
+	} else {
+		val = uint8(nval)
+	}
+	stream.buf = writeFirstBuf(stream.buf, digits[val])
+}
+
+// WriteUint16 write uint16 to stream
+func (stream *Stream) WriteUint16(val uint16) {
+	q1 := val / 1000
+	if q1 == 0 {
+		stream.buf = writeFirstBuf(stream.buf, digits[val])
+		return
+	}
+	r1 := val - q1*1000
+	stream.buf = writeFirstBuf(stream.buf, digits[q1])
+	stream.buf = writeBuf(stream.buf, digits[r1])
+	return
+}
+
+// WriteInt16 write int16 to stream
+func (stream *Stream) WriteInt16(nval int16) {
+	var val uint16
+	if nval < 0 {
+		val = uint16(-nval)
+		stream.buf = append(stream.buf, '-')
+	} else {
+		val = uint16(nval)
+	}
+	stream.WriteUint16(val)
+}
+
+// WriteUint32 write uint32 to stream
+func (stream *Stream) WriteUint32(val uint32) {
+	q1 := val / 1000
+	if q1 == 0 {
+		stream.buf = writeFirstBuf(stream.buf, digits[val])
+		return
+	}
+	r1 := val - q1*1000
+	q2 := q1 / 1000
+	if q2 == 0 {
+		stream.buf = writeFirstBuf(stream.buf, digits[q1])
+		stream.buf = writeBuf(stream.buf, digits[r1])
+		return
+	}
+	r2 := q1 - q2*1000
+	q3 := q2 / 1000
+	if q3 == 0 {
+		stream.buf = writeFirstBuf(stream.buf, digits[q2])
+	} else {
+		r3 := q2 - q3*1000
+		stream.buf = append(stream.buf, byte(q3+'0'))
+		stream.buf = writeBuf(stream.buf, digits[r3])
+	}
+	stream.buf = writeBuf(stream.buf, digits[r2])
+	stream.buf = writeBuf(stream.buf, digits[r1])
+}
+
+// WriteInt32 write int32 to stream
+func (stream *Stream) WriteInt32(nval int32) {
+	var val uint32
+	if nval < 0 {
+		val = uint32(-nval)
+		stream.buf = append(stream.buf, '-')
+	} else {
+		val = uint32(nval)
+	}
+	stream.WriteUint32(val)
+}
+
+// WriteUint64 write uint64 to stream
+func (stream *Stream) WriteUint64(val uint64) {
+	q1 := val / 1000
+	if q1 == 0 {
+		stream.buf = writeFirstBuf(stream.buf, digits[val])
+		return
+	}
+	r1 := val - q1*1000
+	q2 := q1 / 1000
+	if q2 == 0 {
+		stream.buf = writeFirstBuf(stream.buf, digits[q1])
+		stream.buf = writeBuf(stream.buf, digits[r1])
+		return
+	}
+	r2 := q1 - q2*1000
+	q3 := q2 / 1000
+	if q3 == 0 {
+		stream.buf = writeFirstBuf(stream.buf, digits[q2])
+		stream.buf = writeBuf(stream.buf, digits[r2])
+		stream.buf = writeBuf(stream.buf, digits[r1])
+		return
+	}
+	r3 := q2 - q3*1000
+	q4 := q3 / 1000
+	if q4 == 0 {
+		stream.buf = writeFirstBuf(stream.buf, digits[q3])
+		stream.buf = writeBuf(stream.buf, digits[r3])
+		stream.buf = writeBuf(stream.buf, digits[r2])
+		stream.buf = writeBuf(stream.buf, digits[r1])
+		return
+	}
+	r4 := q3 - q4*1000
+	q5 := q4 / 1000
+	if q5 == 0 {
+		stream.buf = writeFirstBuf(stream.buf, digits[q4])
+		stream.buf = writeBuf(stream.buf, digits[r4])
+		stream.buf = writeBuf(stream.buf, digits[r3])
+		stream.buf = writeBuf(stream.buf, digits[r2])
+		stream.buf = writeBuf(stream.buf, digits[r1])
+		return
+	}
+	r5 := q4 - q5*1000
+	q6 := q5 / 1000
+	if q6 == 0 {
+		stream.buf = writeFirstBuf(stream.buf, digits[q5])
+	} else {
+		stream.buf = writeFirstBuf(stream.buf, digits[q6])
+		r6 := q5 - q6*1000
+		stream.buf = writeBuf(stream.buf, digits[r6])
+	}
+	stream.buf = writeBuf(stream.buf, digits[r5])
+	stream.buf = writeBuf(stream.buf, digits[r4])
+	stream.buf = writeBuf(stream.buf, digits[r3])
+	stream.buf = writeBuf(stream.buf, digits[r2])
+	stream.buf = writeBuf(stream.buf, digits[r1])
+}
+
+// WriteInt64 write int64 to stream
+func (stream *Stream) WriteInt64(nval int64) {
+	var val uint64
+	if nval < 0 {
+		val = uint64(-nval)
+		stream.buf = append(stream.buf, '-')
+	} else {
+		val = uint64(nval)
+	}
+	stream.WriteUint64(val)
+}
+
+// WriteInt write int to stream
+func (stream *Stream) WriteInt(val int) {
+	stream.WriteInt64(int64(val))
+}
+
+// WriteUint write uint to stream
+func (stream *Stream) WriteUint(val uint) {
+	stream.WriteUint64(uint64(val))
+}
diff --git a/vendor/github.com/json-iterator/go/stream_str.go b/vendor/github.com/json-iterator/go/stream_str.go
new file mode 100644
index 0000000..54c2ba0
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/stream_str.go
@@ -0,0 +1,372 @@
+package jsoniter
+
+import (
+	"unicode/utf8"
+)
+
+// htmlSafeSet holds the value true if the ASCII character with the given
+// array position can be safely represented inside a JSON string, embedded
+// inside of HTML <script> tags, without any additional escaping.
+//
+// All values are true except for the ASCII control characters (0-31), the
+// double quote ("), the backslash character ("\"), HTML opening and closing
+// tags ("<" and ">"), and the ampersand ("&").
+var htmlSafeSet = [utf8.RuneSelf]bool{
+	' ':      true,
+	'!':      true,
+	'"':      false,
+	'#':      true,
+	'$':      true,
+	'%':      true,
+	'&':      false,
+	'\'':     true,
+	'(':      true,
+	')':      true,
+	'*':      true,
+	'+':      true,
+	',':      true,
+	'-':      true,
+	'.':      true,
+	'/':      true,
+	'0':      true,
+	'1':      true,
+	'2':      true,
+	'3':      true,
+	'4':      true,
+	'5':      true,
+	'6':      true,
+	'7':      true,
+	'8':      true,
+	'9':      true,
+	':':      true,
+	';':      true,
+	'<':      false,
+	'=':      true,
+	'>':      false,
+	'?':      true,
+	'@':      true,
+	'A':      true,
+	'B':      true,
+	'C':      true,
+	'D':      true,
+	'E':      true,
+	'F':      true,
+	'G':      true,
+	'H':      true,
+	'I':      true,
+	'J':      true,
+	'K':      true,
+	'L':      true,
+	'M':      true,
+	'N':      true,
+	'O':      true,
+	'P':      true,
+	'Q':      true,
+	'R':      true,
+	'S':      true,
+	'T':      true,
+	'U':      true,
+	'V':      true,
+	'W':      true,
+	'X':      true,
+	'Y':      true,
+	'Z':      true,
+	'[':      true,
+	'\\':     false,
+	']':      true,
+	'^':      true,
+	'_':      true,
+	'`':      true,
+	'a':      true,
+	'b':      true,
+	'c':      true,
+	'd':      true,
+	'e':      true,
+	'f':      true,
+	'g':      true,
+	'h':      true,
+	'i':      true,
+	'j':      true,
+	'k':      true,
+	'l':      true,
+	'm':      true,
+	'n':      true,
+	'o':      true,
+	'p':      true,
+	'q':      true,
+	'r':      true,
+	's':      true,
+	't':      true,
+	'u':      true,
+	'v':      true,
+	'w':      true,
+	'x':      true,
+	'y':      true,
+	'z':      true,
+	'{':      true,
+	'|':      true,
+	'}':      true,
+	'~':      true,
+	'\u007f': true,
+}
+
+// safeSet holds the value true if the ASCII character with the given array
+// position can be represented inside a JSON string without any further
+// escaping.
+//
+// All values are true except for the ASCII control characters (0-31), the
+// double quote ("), and the backslash character ("\").
+var safeSet = [utf8.RuneSelf]bool{
+	' ':      true,
+	'!':      true,
+	'"':      false,
+	'#':      true,
+	'$':      true,
+	'%':      true,
+	'&':      true,
+	'\'':     true,
+	'(':      true,
+	')':      true,
+	'*':      true,
+	'+':      true,
+	',':      true,
+	'-':      true,
+	'.':      true,
+	'/':      true,
+	'0':      true,
+	'1':      true,
+	'2':      true,
+	'3':      true,
+	'4':      true,
+	'5':      true,
+	'6':      true,
+	'7':      true,
+	'8':      true,
+	'9':      true,
+	':':      true,
+	';':      true,
+	'<':      true,
+	'=':      true,
+	'>':      true,
+	'?':      true,
+	'@':      true,
+	'A':      true,
+	'B':      true,
+	'C':      true,
+	'D':      true,
+	'E':      true,
+	'F':      true,
+	'G':      true,
+	'H':      true,
+	'I':      true,
+	'J':      true,
+	'K':      true,
+	'L':      true,
+	'M':      true,
+	'N':      true,
+	'O':      true,
+	'P':      true,
+	'Q':      true,
+	'R':      true,
+	'S':      true,
+	'T':      true,
+	'U':      true,
+	'V':      true,
+	'W':      true,
+	'X':      true,
+	'Y':      true,
+	'Z':      true,
+	'[':      true,
+	'\\':     false,
+	']':      true,
+	'^':      true,
+	'_':      true,
+	'`':      true,
+	'a':      true,
+	'b':      true,
+	'c':      true,
+	'd':      true,
+	'e':      true,
+	'f':      true,
+	'g':      true,
+	'h':      true,
+	'i':      true,
+	'j':      true,
+	'k':      true,
+	'l':      true,
+	'm':      true,
+	'n':      true,
+	'o':      true,
+	'p':      true,
+	'q':      true,
+	'r':      true,
+	's':      true,
+	't':      true,
+	'u':      true,
+	'v':      true,
+	'w':      true,
+	'x':      true,
+	'y':      true,
+	'z':      true,
+	'{':      true,
+	'|':      true,
+	'}':      true,
+	'~':      true,
+	'\u007f': true,
+}
+
+var hex = "0123456789abcdef"
+
+// WriteStringWithHTMLEscaped write string to stream with html special characters escaped
+func (stream *Stream) WriteStringWithHTMLEscaped(s string) {
+	valLen := len(s)
+	stream.buf = append(stream.buf, '"')
+	// write string, the fast path, without utf8 and escape support
+	i := 0
+	for ; i < valLen; i++ {
+		c := s[i]
+		if c < utf8.RuneSelf && htmlSafeSet[c] {
+			stream.buf = append(stream.buf, c)
+		} else {
+			break
+		}
+	}
+	if i == valLen {
+		stream.buf = append(stream.buf, '"')
+		return
+	}
+	writeStringSlowPathWithHTMLEscaped(stream, i, s, valLen)
+}
+
+func writeStringSlowPathWithHTMLEscaped(stream *Stream, i int, s string, valLen int) {
+	start := i
+	// for the remaining parts, we process them char by char
+	for i < valLen {
+		if b := s[i]; b < utf8.RuneSelf {
+			if htmlSafeSet[b] {
+				i++
+				continue
+			}
+			if start < i {
+				stream.WriteRaw(s[start:i])
+			}
+			switch b {
+			case '\\', '"':
+				stream.writeTwoBytes('\\', b)
+			case '\n':
+				stream.writeTwoBytes('\\', 'n')
+			case '\r':
+				stream.writeTwoBytes('\\', 'r')
+			case '\t':
+				stream.writeTwoBytes('\\', 't')
+			default:
+				// This encodes bytes < 0x20 except for \t, \n and \r.
+				// If escapeHTML is set, it also escapes <, >, and &
+				// because they can lead to security holes when
+				// user-controlled strings are rendered into JSON
+				// and served to some browsers.
+				stream.WriteRaw(`\u00`)
+				stream.writeTwoBytes(hex[b>>4], hex[b&0xF])
+			}
+			i++
+			start = i
+			continue
+		}
+		c, size := utf8.DecodeRuneInString(s[i:])
+		if c == utf8.RuneError && size == 1 {
+			if start < i {
+				stream.WriteRaw(s[start:i])
+			}
+			stream.WriteRaw(`\ufffd`)
+			i++
+			start = i
+			continue
+		}
+		// U+2028 is LINE SEPARATOR.
+		// U+2029 is PARAGRAPH SEPARATOR.
+		// They are both technically valid characters in JSON strings,
+		// but don't work in JSONP, which has to be evaluated as JavaScript,
+		// and can lead to security holes there. It is valid JSON to
+		// escape them, so we do so unconditionally.
+		// See http://timelessrepo.com/json-isnt-a-javascript-subset for discussion.
+		if c == '\u2028' || c == '\u2029' {
+			if start < i {
+				stream.WriteRaw(s[start:i])
+			}
+			stream.WriteRaw(`\u202`)
+			stream.writeByte(hex[c&0xF])
+			i += size
+			start = i
+			continue
+		}
+		i += size
+	}
+	if start < len(s) {
+		stream.WriteRaw(s[start:])
+	}
+	stream.writeByte('"')
+}
+
+// WriteString write string to stream without html escape
+func (stream *Stream) WriteString(s string) {
+	valLen := len(s)
+	stream.buf = append(stream.buf, '"')
+	// write string, the fast path, without utf8 and escape support
+	i := 0
+	for ; i < valLen; i++ {
+		c := s[i]
+		if c > 31 && c != '"' && c != '\\' {
+			stream.buf = append(stream.buf, c)
+		} else {
+			break
+		}
+	}
+	if i == valLen {
+		stream.buf = append(stream.buf, '"')
+		return
+	}
+	writeStringSlowPath(stream, i, s, valLen)
+}
+
+func writeStringSlowPath(stream *Stream, i int, s string, valLen int) {
+	start := i
+	// for the remaining parts, we process them char by char
+	for i < valLen {
+		if b := s[i]; b < utf8.RuneSelf {
+			if safeSet[b] {
+				i++
+				continue
+			}
+			if start < i {
+				stream.WriteRaw(s[start:i])
+			}
+			switch b {
+			case '\\', '"':
+				stream.writeTwoBytes('\\', b)
+			case '\n':
+				stream.writeTwoBytes('\\', 'n')
+			case '\r':
+				stream.writeTwoBytes('\\', 'r')
+			case '\t':
+				stream.writeTwoBytes('\\', 't')
+			default:
+				// This encodes bytes < 0x20 except for \t, \n and \r.
+				// If escapeHTML is set, it also escapes <, >, and &
+				// because they can lead to security holes when
+				// user-controlled strings are rendered into JSON
+				// and served to some browsers.
+				stream.WriteRaw(`\u00`)
+				stream.writeTwoBytes(hex[b>>4], hex[b&0xF])
+			}
+			i++
+			start = i
+			continue
+		}
+		i++
+		continue
+	}
+	if start < len(s) {
+		stream.WriteRaw(s[start:])
+	}
+	stream.writeByte('"')
+}
diff --git a/vendor/github.com/json-iterator/go/test.sh b/vendor/github.com/json-iterator/go/test.sh
new file mode 100644
index 0000000..f4e7c0b
--- /dev/null
+++ b/vendor/github.com/json-iterator/go/test.sh
@@ -0,0 +1,12 @@
+#!/usr/bin/env bash
+
+set -e
+echo "" > coverage.txt
+
+for d in $(go list ./... | grep -v vendor); do
+    go test -coverprofile=profile.out -coverpkg=github.com/json-iterator/go $d
+    if [ -f profile.out ]; then
+        cat profile.out >> coverage.txt
+        rm profile.out
+    fi
+done
diff --git a/vendor/github.com/konsorten/go-windows-terminal-sequences/LICENSE b/vendor/github.com/konsorten/go-windows-terminal-sequences/LICENSE
new file mode 100644
index 0000000..14127cd
--- /dev/null
+++ b/vendor/github.com/konsorten/go-windows-terminal-sequences/LICENSE
@@ -0,0 +1,9 @@
+(The MIT License)
+
+Copyright (c) 2017 marvin + konsorten GmbH (open-source@konsorten.de)
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the 'Software'), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED 'AS IS', WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/vendor/github.com/konsorten/go-windows-terminal-sequences/README.md b/vendor/github.com/konsorten/go-windows-terminal-sequences/README.md
new file mode 100644
index 0000000..949b77e
--- /dev/null
+++ b/vendor/github.com/konsorten/go-windows-terminal-sequences/README.md
@@ -0,0 +1,40 @@
+# Windows Terminal Sequences
+
+This library allow for enabling Windows terminal color support for Go.
+
+See [Console Virtual Terminal Sequences](https://docs.microsoft.com/en-us/windows/console/console-virtual-terminal-sequences) for details.
+
+## Usage
+
+```go
+import (
+	"syscall"
+	
+	sequences "github.com/konsorten/go-windows-terminal-sequences"
+)
+
+func main() {
+	sequences.EnableVirtualTerminalProcessing(syscall.Stdout, true)
+}
+
+```
+
+## Authors
+
+The tool is sponsored by the [marvin + konsorten GmbH](http://www.konsorten.de).
+
+We thank all the authors who provided code to this library:
+
+* Felix Kollmann
+
+## License
+
+(The MIT License)
+
+Copyright (c) 2018 marvin + konsorten GmbH (open-source@konsorten.de)
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the 'Software'), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED 'AS IS', WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/vendor/github.com/konsorten/go-windows-terminal-sequences/go.mod b/vendor/github.com/konsorten/go-windows-terminal-sequences/go.mod
new file mode 100644
index 0000000..716c613
--- /dev/null
+++ b/vendor/github.com/konsorten/go-windows-terminal-sequences/go.mod
@@ -0,0 +1 @@
+module github.com/konsorten/go-windows-terminal-sequences
diff --git a/vendor/github.com/konsorten/go-windows-terminal-sequences/sequences.go b/vendor/github.com/konsorten/go-windows-terminal-sequences/sequences.go
new file mode 100644
index 0000000..ef18d8f
--- /dev/null
+++ b/vendor/github.com/konsorten/go-windows-terminal-sequences/sequences.go
@@ -0,0 +1,36 @@
+// +build windows
+
+package sequences
+
+import (
+	"syscall"
+	"unsafe"
+)
+
+var (
+	kernel32Dll    *syscall.LazyDLL  = syscall.NewLazyDLL("Kernel32.dll")
+	setConsoleMode *syscall.LazyProc = kernel32Dll.NewProc("SetConsoleMode")
+)
+
+func EnableVirtualTerminalProcessing(stream syscall.Handle, enable bool) error {
+	const ENABLE_VIRTUAL_TERMINAL_PROCESSING uint32 = 0x4
+
+	var mode uint32
+	err := syscall.GetConsoleMode(syscall.Stdout, &mode)
+	if err != nil {
+		return err
+	}
+
+	if enable {
+		mode |= ENABLE_VIRTUAL_TERMINAL_PROCESSING
+	} else {
+		mode &^= ENABLE_VIRTUAL_TERMINAL_PROCESSING
+	}
+
+	ret, _, err := setConsoleMode.Call(uintptr(unsafe.Pointer(stream)), uintptr(mode))
+	if ret == 0 {
+		return err
+	}
+
+	return nil
+}
diff --git a/vendor/github.com/matttproud/golang_protobuf_extensions/LICENSE b/vendor/github.com/matttproud/golang_protobuf_extensions/LICENSE
new file mode 100644
index 0000000..8dada3e
--- /dev/null
+++ b/vendor/github.com/matttproud/golang_protobuf_extensions/LICENSE
@@ -0,0 +1,201 @@
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "{}"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright {yyyy} {name of copyright owner}
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
diff --git a/vendor/github.com/matttproud/golang_protobuf_extensions/NOTICE b/vendor/github.com/matttproud/golang_protobuf_extensions/NOTICE
new file mode 100644
index 0000000..5d8cb5b
--- /dev/null
+++ b/vendor/github.com/matttproud/golang_protobuf_extensions/NOTICE
@@ -0,0 +1 @@
+Copyright 2012 Matt T. Proud (matt.proud@gmail.com)
diff --git a/vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/.gitignore b/vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/.gitignore
new file mode 100644
index 0000000..e16fb94
--- /dev/null
+++ b/vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/.gitignore
@@ -0,0 +1 @@
+cover.dat
diff --git a/vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/Makefile b/vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/Makefile
new file mode 100644
index 0000000..81be214
--- /dev/null
+++ b/vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/Makefile
@@ -0,0 +1,7 @@
+all:
+
+cover:
+	go test -cover -v -coverprofile=cover.dat ./...
+	go tool cover -func cover.dat
+
+.PHONY: cover
diff --git a/vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/decode.go b/vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/decode.go
new file mode 100644
index 0000000..258c063
--- /dev/null
+++ b/vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/decode.go
@@ -0,0 +1,75 @@
+// Copyright 2013 Matt T. Proud
+//
+// 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 pbutil
+
+import (
+	"encoding/binary"
+	"errors"
+	"io"
+
+	"github.com/golang/protobuf/proto"
+)
+
+var errInvalidVarint = errors.New("invalid varint32 encountered")
+
+// ReadDelimited decodes a message from the provided length-delimited stream,
+// where the length is encoded as 32-bit varint prefix to the message body.
+// It returns the total number of bytes read and any applicable error.  This is
+// roughly equivalent to the companion Java API's
+// MessageLite#parseDelimitedFrom.  As per the reader contract, this function
+// calls r.Read repeatedly as required until exactly one message including its
+// prefix is read and decoded (or an error has occurred).  The function never
+// reads more bytes from the stream than required.  The function never returns
+// an error if a message has been read and decoded correctly, even if the end
+// of the stream has been reached in doing so.  In that case, any subsequent
+// calls return (0, io.EOF).
+func ReadDelimited(r io.Reader, m proto.Message) (n int, err error) {
+	// Per AbstractParser#parsePartialDelimitedFrom with
+	// CodedInputStream#readRawVarint32.
+	var headerBuf [binary.MaxVarintLen32]byte
+	var bytesRead, varIntBytes int
+	var messageLength uint64
+	for varIntBytes == 0 { // i.e. no varint has been decoded yet.
+		if bytesRead >= len(headerBuf) {
+			return bytesRead, errInvalidVarint
+		}
+		// We have to read byte by byte here to avoid reading more bytes
+		// than required. Each read byte is appended to what we have
+		// read before.
+		newBytesRead, err := r.Read(headerBuf[bytesRead : bytesRead+1])
+		if newBytesRead == 0 {
+			if err != nil {
+				return bytesRead, err
+			}
+			// A Reader should not return (0, nil), but if it does,
+			// it should be treated as no-op (according to the
+			// Reader contract). So let's go on...
+			continue
+		}
+		bytesRead += newBytesRead
+		// Now present everything read so far to the varint decoder and
+		// see if a varint can be decoded already.
+		messageLength, varIntBytes = proto.DecodeVarint(headerBuf[:bytesRead])
+	}
+
+	messageBuf := make([]byte, messageLength)
+	newBytesRead, err := io.ReadFull(r, messageBuf)
+	bytesRead += newBytesRead
+	if err != nil {
+		return bytesRead, err
+	}
+
+	return bytesRead, proto.Unmarshal(messageBuf, m)
+}
diff --git a/vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/doc.go b/vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/doc.go
new file mode 100644
index 0000000..c318385
--- /dev/null
+++ b/vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/doc.go
@@ -0,0 +1,16 @@
+// Copyright 2013 Matt T. Proud
+//
+// 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 pbutil provides record length-delimited Protocol Buffer streaming.
+package pbutil
diff --git a/vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/encode.go b/vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/encode.go
new file mode 100644
index 0000000..8fb59ad
--- /dev/null
+++ b/vendor/github.com/matttproud/golang_protobuf_extensions/pbutil/encode.go
@@ -0,0 +1,46 @@
+// Copyright 2013 Matt T. Proud
+//
+// 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 pbutil
+
+import (
+	"encoding/binary"
+	"io"
+
+	"github.com/golang/protobuf/proto"
+)
+
+// WriteDelimited encodes and dumps a message to the provided writer prefixed
+// with a 32-bit varint indicating the length of the encoded message, producing
+// a length-delimited record stream, which can be used to chain together
+// encoded messages of the same type together in a file.  It returns the total
+// number of bytes written and any applicable error.  This is roughly
+// equivalent to the companion Java API's MessageLite#writeDelimitedTo.
+func WriteDelimited(w io.Writer, m proto.Message) (n int, err error) {
+	buffer, err := proto.Marshal(m)
+	if err != nil {
+		return 0, err
+	}
+
+	var buf [binary.MaxVarintLen32]byte
+	encodedLength := binary.PutUvarint(buf[:], uint64(len(buffer)))
+
+	sync, err := w.Write(buf[:encodedLength])
+	if err != nil {
+		return sync, err
+	}
+
+	n, err = w.Write(buffer)
+	return n + sync, err
+}
diff --git a/vendor/github.com/modern-go/concurrent/.gitignore b/vendor/github.com/modern-go/concurrent/.gitignore
new file mode 100644
index 0000000..3f2bc47
--- /dev/null
+++ b/vendor/github.com/modern-go/concurrent/.gitignore
@@ -0,0 +1 @@
+/coverage.txt
diff --git a/vendor/github.com/modern-go/concurrent/.travis.yml b/vendor/github.com/modern-go/concurrent/.travis.yml
new file mode 100644
index 0000000..449e67c
--- /dev/null
+++ b/vendor/github.com/modern-go/concurrent/.travis.yml
@@ -0,0 +1,14 @@
+language: go
+
+go:
+  - 1.8.x
+  - 1.x
+
+before_install:
+  - go get -t -v ./...
+
+script:
+  - ./test.sh
+
+after_success:
+  - bash <(curl -s https://codecov.io/bash)
diff --git a/vendor/github.com/modern-go/concurrent/LICENSE b/vendor/github.com/modern-go/concurrent/LICENSE
new file mode 100644
index 0000000..261eeb9
--- /dev/null
+++ b/vendor/github.com/modern-go/concurrent/LICENSE
@@ -0,0 +1,201 @@
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
diff --git a/vendor/github.com/modern-go/concurrent/README.md b/vendor/github.com/modern-go/concurrent/README.md
new file mode 100644
index 0000000..acab320
--- /dev/null
+++ b/vendor/github.com/modern-go/concurrent/README.md
@@ -0,0 +1,49 @@
+# concurrent
+
+[![Sourcegraph](https://sourcegraph.com/github.com/modern-go/concurrent/-/badge.svg)](https://sourcegraph.com/github.com/modern-go/concurrent?badge)
+[![GoDoc](http://img.shields.io/badge/go-documentation-blue.svg?style=flat-square)](http://godoc.org/github.com/modern-go/concurrent)
+[![Build Status](https://travis-ci.org/modern-go/concurrent.svg?branch=master)](https://travis-ci.org/modern-go/concurrent)
+[![codecov](https://codecov.io/gh/modern-go/concurrent/branch/master/graph/badge.svg)](https://codecov.io/gh/modern-go/concurrent)
+[![rcard](https://goreportcard.com/badge/github.com/modern-go/concurrent)](https://goreportcard.com/report/github.com/modern-go/concurrent)
+[![License](https://img.shields.io/badge/License-Apache%202.0-blue.svg)](https://raw.githubusercontent.com/modern-go/concurrent/master/LICENSE)
+
+* concurrent.Map: backport sync.Map for go below 1.9
+* concurrent.Executor: goroutine with explicit ownership and cancellable
+
+# concurrent.Map
+
+because sync.Map is only available in go 1.9, we can use concurrent.Map to make code portable
+
+```go
+m := concurrent.NewMap()
+m.Store("hello", "world")
+elem, found := m.Load("hello")
+// elem will be "world"
+// found will be true
+```
+
+# concurrent.Executor
+
+```go
+executor := concurrent.NewUnboundedExecutor()
+executor.Go(func(ctx context.Context) {
+    everyMillisecond := time.NewTicker(time.Millisecond)
+    for {
+        select {
+        case <-ctx.Done():
+            fmt.Println("goroutine exited")
+            return
+        case <-everyMillisecond.C:
+            // do something
+        }
+    }
+})
+time.Sleep(time.Second)
+executor.StopAndWaitForever()
+fmt.Println("executor stopped")
+```
+
+attach goroutine to executor instance, so that we can
+
+* cancel it by stop the executor with Stop/StopAndWait/StopAndWaitForever
+* handle panic by callback: the default behavior will no longer crash your application
\ No newline at end of file
diff --git a/vendor/github.com/modern-go/concurrent/executor.go b/vendor/github.com/modern-go/concurrent/executor.go
new file mode 100644
index 0000000..623dba1
--- /dev/null
+++ b/vendor/github.com/modern-go/concurrent/executor.go
@@ -0,0 +1,14 @@
+package concurrent
+
+import "context"
+
+// Executor replace go keyword to start a new goroutine
+// the goroutine should cancel itself if the context passed in has been cancelled
+// the goroutine started by the executor, is owned by the executor
+// we can cancel all executors owned by the executor just by stop the executor itself
+// however Executor interface does not Stop method, the one starting and owning executor
+// should use the concrete type of executor, instead of this interface.
+type Executor interface {
+	// Go starts a new goroutine controlled by the context
+	Go(handler func(ctx context.Context))
+}
diff --git a/vendor/github.com/modern-go/concurrent/go_above_19.go b/vendor/github.com/modern-go/concurrent/go_above_19.go
new file mode 100644
index 0000000..aeabf8c
--- /dev/null
+++ b/vendor/github.com/modern-go/concurrent/go_above_19.go
@@ -0,0 +1,15 @@
+//+build go1.9
+
+package concurrent
+
+import "sync"
+
+// Map is a wrapper for sync.Map introduced in go1.9
+type Map struct {
+	sync.Map
+}
+
+// NewMap creates a thread safe Map
+func NewMap() *Map {
+	return &Map{}
+}
diff --git a/vendor/github.com/modern-go/concurrent/go_below_19.go b/vendor/github.com/modern-go/concurrent/go_below_19.go
new file mode 100644
index 0000000..b9c8df7
--- /dev/null
+++ b/vendor/github.com/modern-go/concurrent/go_below_19.go
@@ -0,0 +1,33 @@
+//+build !go1.9
+
+package concurrent
+
+import "sync"
+
+// Map implements a thread safe map for go version below 1.9 using mutex
+type Map struct {
+	lock sync.RWMutex
+	data map[interface{}]interface{}
+}
+
+// NewMap creates a thread safe map
+func NewMap() *Map {
+	return &Map{
+		data: make(map[interface{}]interface{}, 32),
+	}
+}
+
+// Load is same as sync.Map Load
+func (m *Map) Load(key interface{}) (elem interface{}, found bool) {
+	m.lock.RLock()
+	elem, found = m.data[key]
+	m.lock.RUnlock()
+	return
+}
+
+// Load is same as sync.Map Store
+func (m *Map) Store(key interface{}, elem interface{}) {
+	m.lock.Lock()
+	m.data[key] = elem
+	m.lock.Unlock()
+}
diff --git a/vendor/github.com/modern-go/concurrent/log.go b/vendor/github.com/modern-go/concurrent/log.go
new file mode 100644
index 0000000..9756fcc
--- /dev/null
+++ b/vendor/github.com/modern-go/concurrent/log.go
@@ -0,0 +1,13 @@
+package concurrent
+
+import (
+	"os"
+	"log"
+	"io/ioutil"
+)
+
+// ErrorLogger is used to print out error, can be set to writer other than stderr
+var ErrorLogger = log.New(os.Stderr, "", 0)
+
+// InfoLogger is used to print informational message, default to off
+var InfoLogger = log.New(ioutil.Discard, "", 0)
\ No newline at end of file
diff --git a/vendor/github.com/modern-go/concurrent/test.sh b/vendor/github.com/modern-go/concurrent/test.sh
new file mode 100644
index 0000000..d1e6b2e
--- /dev/null
+++ b/vendor/github.com/modern-go/concurrent/test.sh
@@ -0,0 +1,12 @@
+#!/usr/bin/env bash
+
+set -e
+echo "" > coverage.txt
+
+for d in $(go list ./... | grep -v vendor); do
+    go test -coverprofile=profile.out -coverpkg=github.com/modern-go/concurrent $d
+    if [ -f profile.out ]; then
+        cat profile.out >> coverage.txt
+        rm profile.out
+    fi
+done
diff --git a/vendor/github.com/modern-go/concurrent/unbounded_executor.go b/vendor/github.com/modern-go/concurrent/unbounded_executor.go
new file mode 100644
index 0000000..05a77dc
--- /dev/null
+++ b/vendor/github.com/modern-go/concurrent/unbounded_executor.go
@@ -0,0 +1,119 @@
+package concurrent
+
+import (
+	"context"
+	"fmt"
+	"runtime"
+	"runtime/debug"
+	"sync"
+	"time"
+	"reflect"
+)
+
+// HandlePanic logs goroutine panic by default
+var HandlePanic = func(recovered interface{}, funcName string) {
+	ErrorLogger.Println(fmt.Sprintf("%s panic: %v", funcName, recovered))
+	ErrorLogger.Println(string(debug.Stack()))
+}
+
+// UnboundedExecutor is a executor without limits on counts of alive goroutines
+// it tracks the goroutine started by it, and can cancel them when shutdown
+type UnboundedExecutor struct {
+	ctx                   context.Context
+	cancel                context.CancelFunc
+	activeGoroutinesMutex *sync.Mutex
+	activeGoroutines      map[string]int
+	HandlePanic           func(recovered interface{}, funcName string)
+}
+
+// GlobalUnboundedExecutor has the life cycle of the program itself
+// any goroutine want to be shutdown before main exit can be started from this executor
+// GlobalUnboundedExecutor expects the main function to call stop
+// it does not magically knows the main function exits
+var GlobalUnboundedExecutor = NewUnboundedExecutor()
+
+// NewUnboundedExecutor creates a new UnboundedExecutor,
+// UnboundedExecutor can not be created by &UnboundedExecutor{}
+// HandlePanic can be set with a callback to override global HandlePanic
+func NewUnboundedExecutor() *UnboundedExecutor {
+	ctx, cancel := context.WithCancel(context.TODO())
+	return &UnboundedExecutor{
+		ctx:                   ctx,
+		cancel:                cancel,
+		activeGoroutinesMutex: &sync.Mutex{},
+		activeGoroutines:      map[string]int{},
+	}
+}
+
+// Go starts a new goroutine and tracks its lifecycle.
+// Panic will be recovered and logged automatically, except for StopSignal
+func (executor *UnboundedExecutor) Go(handler func(ctx context.Context)) {
+	pc := reflect.ValueOf(handler).Pointer()
+	f := runtime.FuncForPC(pc)
+	funcName := f.Name()
+	file, line := f.FileLine(pc)
+	executor.activeGoroutinesMutex.Lock()
+	defer executor.activeGoroutinesMutex.Unlock()
+	startFrom := fmt.Sprintf("%s:%d", file, line)
+	executor.activeGoroutines[startFrom] += 1
+	go func() {
+		defer func() {
+			recovered := recover()
+			// if you want to quit a goroutine without trigger HandlePanic
+			// use runtime.Goexit() to quit
+			if recovered != nil {
+				if executor.HandlePanic == nil {
+					HandlePanic(recovered, funcName)
+				} else {
+					executor.HandlePanic(recovered, funcName)
+				}
+			}
+			executor.activeGoroutinesMutex.Lock()
+			executor.activeGoroutines[startFrom] -= 1
+			executor.activeGoroutinesMutex.Unlock()
+		}()
+		handler(executor.ctx)
+	}()
+}
+
+// Stop cancel all goroutines started by this executor without wait
+func (executor *UnboundedExecutor) Stop() {
+	executor.cancel()
+}
+
+// StopAndWaitForever cancel all goroutines started by this executor and
+// wait until all goroutines exited
+func (executor *UnboundedExecutor) StopAndWaitForever() {
+	executor.StopAndWait(context.Background())
+}
+
+// StopAndWait cancel all goroutines started by this executor and wait.
+// Wait can be cancelled by the context passed in.
+func (executor *UnboundedExecutor) StopAndWait(ctx context.Context) {
+	executor.cancel()
+	for {
+		oneHundredMilliseconds := time.NewTimer(time.Millisecond * 100)
+		select {
+		case <-oneHundredMilliseconds.C:
+			if executor.checkNoActiveGoroutines() {
+				return
+			}
+		case <-ctx.Done():
+			return
+		}
+	}
+}
+
+func (executor *UnboundedExecutor) checkNoActiveGoroutines() bool {
+	executor.activeGoroutinesMutex.Lock()
+	defer executor.activeGoroutinesMutex.Unlock()
+	for startFrom, count := range executor.activeGoroutines {
+		if count > 0 {
+			InfoLogger.Println("UnboundedExecutor is still waiting goroutines to quit",
+				"startFrom", startFrom,
+				"count", count)
+			return false
+		}
+	}
+	return true
+}
diff --git a/vendor/github.com/modern-go/reflect2/.gitignore b/vendor/github.com/modern-go/reflect2/.gitignore
new file mode 100644
index 0000000..7b26c94
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/.gitignore
@@ -0,0 +1,2 @@
+/vendor
+/coverage.txt
diff --git a/vendor/github.com/modern-go/reflect2/.travis.yml b/vendor/github.com/modern-go/reflect2/.travis.yml
new file mode 100644
index 0000000..fbb4374
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/.travis.yml
@@ -0,0 +1,15 @@
+language: go
+
+go:
+  - 1.8.x
+  - 1.x
+
+before_install:
+  - go get -t -v ./...
+  - go get -t -v github.com/modern-go/reflect2-tests/...
+
+script:
+  - ./test.sh
+
+after_success:
+  - bash <(curl -s https://codecov.io/bash)
diff --git a/vendor/github.com/modern-go/reflect2/Gopkg.lock b/vendor/github.com/modern-go/reflect2/Gopkg.lock
new file mode 100644
index 0000000..2a3a698
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/Gopkg.lock
@@ -0,0 +1,15 @@
+# This file is autogenerated, do not edit; changes may be undone by the next 'dep ensure'.
+
+
+[[projects]]
+  name = "github.com/modern-go/concurrent"
+  packages = ["."]
+  revision = "e0a39a4cb4216ea8db28e22a69f4ec25610d513a"
+  version = "1.0.0"
+
+[solve-meta]
+  analyzer-name = "dep"
+  analyzer-version = 1
+  inputs-digest = "daee8a88b3498b61c5640056665b8b9eea062006f5e596bbb6a3ed9119a11ec7"
+  solver-name = "gps-cdcl"
+  solver-version = 1
diff --git a/vendor/github.com/modern-go/reflect2/Gopkg.toml b/vendor/github.com/modern-go/reflect2/Gopkg.toml
new file mode 100644
index 0000000..2f4f4db
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/Gopkg.toml
@@ -0,0 +1,35 @@
+# Gopkg.toml example
+#
+# Refer to https://golang.github.io/dep/docs/Gopkg.toml.html
+# for detailed Gopkg.toml documentation.
+#
+# required = ["github.com/user/thing/cmd/thing"]
+# ignored = ["github.com/user/project/pkgX", "bitbucket.org/user/project/pkgA/pkgY"]
+#
+# [[constraint]]
+#   name = "github.com/user/project"
+#   version = "1.0.0"
+#
+# [[constraint]]
+#   name = "github.com/user/project2"
+#   branch = "dev"
+#   source = "github.com/myfork/project2"
+#
+# [[override]]
+#   name = "github.com/x/y"
+#   version = "2.4.0"
+#
+# [prune]
+#   non-go = false
+#   go-tests = true
+#   unused-packages = true
+
+ignored = []
+
+[[constraint]]
+  name = "github.com/modern-go/concurrent"
+  version = "1.0.0"
+
+[prune]
+  go-tests = true
+  unused-packages = true
diff --git a/vendor/github.com/modern-go/reflect2/LICENSE b/vendor/github.com/modern-go/reflect2/LICENSE
new file mode 100644
index 0000000..261eeb9
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/LICENSE
@@ -0,0 +1,201 @@
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
diff --git a/vendor/github.com/modern-go/reflect2/README.md b/vendor/github.com/modern-go/reflect2/README.md
new file mode 100644
index 0000000..6f968aa
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/README.md
@@ -0,0 +1,71 @@
+# reflect2
+
+[![Sourcegraph](https://sourcegraph.com/github.com/modern-go/reflect2/-/badge.svg)](https://sourcegraph.com/github.com/modern-go/reflect2?badge)
+[![GoDoc](http://img.shields.io/badge/go-documentation-blue.svg?style=flat-square)](http://godoc.org/github.com/modern-go/reflect2)
+[![Build Status](https://travis-ci.org/modern-go/reflect2.svg?branch=master)](https://travis-ci.org/modern-go/reflect2)
+[![codecov](https://codecov.io/gh/modern-go/reflect2/branch/master/graph/badge.svg)](https://codecov.io/gh/modern-go/reflect2)
+[![rcard](https://goreportcard.com/badge/github.com/modern-go/reflect2)](https://goreportcard.com/report/github.com/modern-go/reflect2)
+[![License](https://img.shields.io/badge/License-Apache%202.0-blue.svg)](https://raw.githubusercontent.com/modern-go/reflect2/master/LICENSE)
+
+reflect api that avoids runtime reflect.Value cost
+
+* reflect get/set interface{}, with type checking
+* reflect get/set unsafe.Pointer, without type checking
+* `reflect2.TypeByName` works like `Class.forName` found in java
+
+[json-iterator](https://github.com/json-iterator/go) use this package to save runtime dispatching cost.
+This package is designed for low level libraries to optimize reflection performance.
+General application should still use reflect standard library.
+
+# reflect2.TypeByName
+
+```go
+// given package is github.com/your/awesome-package
+type MyStruct struct {
+	// ...
+}
+
+// will return the type
+reflect2.TypeByName("awesome-package.MyStruct")
+// however, if the type has not been used
+// it will be eliminated by compiler, so we can not get it in runtime
+```
+
+# reflect2 get/set interface{}
+
+```go
+valType := reflect2.TypeOf(1)
+i := 1
+j := 10
+valType.Set(&i, &j)
+// i will be 10
+```
+
+to get set `type`, always use its pointer `*type`
+
+# reflect2 get/set unsafe.Pointer
+
+```go
+valType := reflect2.TypeOf(1)
+i := 1
+j := 10
+valType.UnsafeSet(unsafe.Pointer(&i), unsafe.Pointer(&j))
+// i will be 10
+```
+
+to get set `type`, always use its pointer `*type`
+
+# benchmark
+
+Benchmark is not necessary for this package. It does nothing actually.
+As it is just a thin wrapper to make go runtime public. 
+Both `reflect2` and `reflect` call same function 
+provided by `runtime` package exposed by go language.
+
+# unsafe safety
+
+Instead of casting `[]byte` to `sliceHeader` in your application using unsafe.
+We can use reflect2 instead. This way, if `sliceHeader` changes in the future,
+only reflect2 need to be upgraded.
+
+reflect2 tries its best to keep the implementation same as reflect (by testing).
\ No newline at end of file
diff --git a/vendor/github.com/modern-go/reflect2/go_above_17.go b/vendor/github.com/modern-go/reflect2/go_above_17.go
new file mode 100644
index 0000000..5c1cea8
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/go_above_17.go
@@ -0,0 +1,8 @@
+//+build go1.7
+
+package reflect2
+
+import "unsafe"
+
+//go:linkname resolveTypeOff reflect.resolveTypeOff
+func resolveTypeOff(rtype unsafe.Pointer, off int32) unsafe.Pointer
diff --git a/vendor/github.com/modern-go/reflect2/go_above_19.go b/vendor/github.com/modern-go/reflect2/go_above_19.go
new file mode 100644
index 0000000..c7e3b78
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/go_above_19.go
@@ -0,0 +1,14 @@
+//+build go1.9
+
+package reflect2
+
+import (
+	"unsafe"
+)
+
+//go:linkname makemap reflect.makemap
+func makemap(rtype unsafe.Pointer, cap int) (m unsafe.Pointer)
+
+func makeMapWithSize(rtype unsafe.Pointer, cap int) unsafe.Pointer {
+	return makemap(rtype, cap)
+}
diff --git a/vendor/github.com/modern-go/reflect2/go_below_17.go b/vendor/github.com/modern-go/reflect2/go_below_17.go
new file mode 100644
index 0000000..65a93c8
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/go_below_17.go
@@ -0,0 +1,9 @@
+//+build !go1.7
+
+package reflect2
+
+import "unsafe"
+
+func resolveTypeOff(rtype unsafe.Pointer, off int32) unsafe.Pointer {
+	return nil
+}
diff --git a/vendor/github.com/modern-go/reflect2/go_below_19.go b/vendor/github.com/modern-go/reflect2/go_below_19.go
new file mode 100644
index 0000000..b050ef7
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/go_below_19.go
@@ -0,0 +1,14 @@
+//+build !go1.9
+
+package reflect2
+
+import (
+	"unsafe"
+)
+
+//go:linkname makemap reflect.makemap
+func makemap(rtype unsafe.Pointer) (m unsafe.Pointer)
+
+func makeMapWithSize(rtype unsafe.Pointer, cap int) unsafe.Pointer {
+	return makemap(rtype)
+}
diff --git a/vendor/github.com/modern-go/reflect2/reflect2.go b/vendor/github.com/modern-go/reflect2/reflect2.go
new file mode 100644
index 0000000..63b49c7
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/reflect2.go
@@ -0,0 +1,298 @@
+package reflect2
+
+import (
+	"github.com/modern-go/concurrent"
+	"reflect"
+	"unsafe"
+)
+
+type Type interface {
+	Kind() reflect.Kind
+	// New return pointer to data of this type
+	New() interface{}
+	// UnsafeNew return the allocated space pointed by unsafe.Pointer
+	UnsafeNew() unsafe.Pointer
+	// PackEFace cast a unsafe pointer to object represented pointer
+	PackEFace(ptr unsafe.Pointer) interface{}
+	// Indirect dereference object represented pointer to this type
+	Indirect(obj interface{}) interface{}
+	// UnsafeIndirect dereference pointer to this type
+	UnsafeIndirect(ptr unsafe.Pointer) interface{}
+	// Type1 returns reflect.Type
+	Type1() reflect.Type
+	Implements(thatType Type) bool
+	String() string
+	RType() uintptr
+	// interface{} of this type has pointer like behavior
+	LikePtr() bool
+	IsNullable() bool
+	IsNil(obj interface{}) bool
+	UnsafeIsNil(ptr unsafe.Pointer) bool
+	Set(obj interface{}, val interface{})
+	UnsafeSet(ptr unsafe.Pointer, val unsafe.Pointer)
+	AssignableTo(anotherType Type) bool
+}
+
+type ListType interface {
+	Type
+	Elem() Type
+	SetIndex(obj interface{}, index int, elem interface{})
+	UnsafeSetIndex(obj unsafe.Pointer, index int, elem unsafe.Pointer)
+	GetIndex(obj interface{}, index int) interface{}
+	UnsafeGetIndex(obj unsafe.Pointer, index int) unsafe.Pointer
+}
+
+type ArrayType interface {
+	ListType
+	Len() int
+}
+
+type SliceType interface {
+	ListType
+	MakeSlice(length int, cap int) interface{}
+	UnsafeMakeSlice(length int, cap int) unsafe.Pointer
+	Grow(obj interface{}, newLength int)
+	UnsafeGrow(ptr unsafe.Pointer, newLength int)
+	Append(obj interface{}, elem interface{})
+	UnsafeAppend(obj unsafe.Pointer, elem unsafe.Pointer)
+	LengthOf(obj interface{}) int
+	UnsafeLengthOf(ptr unsafe.Pointer) int
+	SetNil(obj interface{})
+	UnsafeSetNil(ptr unsafe.Pointer)
+	Cap(obj interface{}) int
+	UnsafeCap(ptr unsafe.Pointer) int
+}
+
+type StructType interface {
+	Type
+	NumField() int
+	Field(i int) StructField
+	FieldByName(name string) StructField
+	FieldByIndex(index []int) StructField
+	FieldByNameFunc(match func(string) bool) StructField
+}
+
+type StructField interface {
+	Offset() uintptr
+	Name() string
+	PkgPath() string
+	Type() Type
+	Tag() reflect.StructTag
+	Index() []int
+	Anonymous() bool
+	Set(obj interface{}, value interface{})
+	UnsafeSet(obj unsafe.Pointer, value unsafe.Pointer)
+	Get(obj interface{}) interface{}
+	UnsafeGet(obj unsafe.Pointer) unsafe.Pointer
+}
+
+type MapType interface {
+	Type
+	Key() Type
+	Elem() Type
+	MakeMap(cap int) interface{}
+	UnsafeMakeMap(cap int) unsafe.Pointer
+	SetIndex(obj interface{}, key interface{}, elem interface{})
+	UnsafeSetIndex(obj unsafe.Pointer, key unsafe.Pointer, elem unsafe.Pointer)
+	TryGetIndex(obj interface{}, key interface{}) (interface{}, bool)
+	GetIndex(obj interface{}, key interface{}) interface{}
+	UnsafeGetIndex(obj unsafe.Pointer, key unsafe.Pointer) unsafe.Pointer
+	Iterate(obj interface{}) MapIterator
+	UnsafeIterate(obj unsafe.Pointer) MapIterator
+}
+
+type MapIterator interface {
+	HasNext() bool
+	Next() (key interface{}, elem interface{})
+	UnsafeNext() (key unsafe.Pointer, elem unsafe.Pointer)
+}
+
+type PtrType interface {
+	Type
+	Elem() Type
+}
+
+type InterfaceType interface {
+	NumMethod() int
+}
+
+type Config struct {
+	UseSafeImplementation bool
+}
+
+type API interface {
+	TypeOf(obj interface{}) Type
+	Type2(type1 reflect.Type) Type
+}
+
+var ConfigUnsafe = Config{UseSafeImplementation: false}.Froze()
+var ConfigSafe = Config{UseSafeImplementation: true}.Froze()
+
+type frozenConfig struct {
+	useSafeImplementation bool
+	cache                 *concurrent.Map
+}
+
+func (cfg Config) Froze() *frozenConfig {
+	return &frozenConfig{
+		useSafeImplementation: cfg.UseSafeImplementation,
+		cache: concurrent.NewMap(),
+	}
+}
+
+func (cfg *frozenConfig) TypeOf(obj interface{}) Type {
+	cacheKey := uintptr(unpackEFace(obj).rtype)
+	typeObj, found := cfg.cache.Load(cacheKey)
+	if found {
+		return typeObj.(Type)
+	}
+	return cfg.Type2(reflect.TypeOf(obj))
+}
+
+func (cfg *frozenConfig) Type2(type1 reflect.Type) Type {
+	if type1 == nil {
+		return nil
+	}
+	cacheKey := uintptr(unpackEFace(type1).data)
+	typeObj, found := cfg.cache.Load(cacheKey)
+	if found {
+		return typeObj.(Type)
+	}
+	type2 := cfg.wrapType(type1)
+	cfg.cache.Store(cacheKey, type2)
+	return type2
+}
+
+func (cfg *frozenConfig) wrapType(type1 reflect.Type) Type {
+	safeType := safeType{Type: type1, cfg: cfg}
+	switch type1.Kind() {
+	case reflect.Struct:
+		if cfg.useSafeImplementation {
+			return &safeStructType{safeType}
+		}
+		return newUnsafeStructType(cfg, type1)
+	case reflect.Array:
+		if cfg.useSafeImplementation {
+			return &safeSliceType{safeType}
+		}
+		return newUnsafeArrayType(cfg, type1)
+	case reflect.Slice:
+		if cfg.useSafeImplementation {
+			return &safeSliceType{safeType}
+		}
+		return newUnsafeSliceType(cfg, type1)
+	case reflect.Map:
+		if cfg.useSafeImplementation {
+			return &safeMapType{safeType}
+		}
+		return newUnsafeMapType(cfg, type1)
+	case reflect.Ptr, reflect.Chan, reflect.Func:
+		if cfg.useSafeImplementation {
+			return &safeMapType{safeType}
+		}
+		return newUnsafePtrType(cfg, type1)
+	case reflect.Interface:
+		if cfg.useSafeImplementation {
+			return &safeMapType{safeType}
+		}
+		if type1.NumMethod() == 0 {
+			return newUnsafeEFaceType(cfg, type1)
+		}
+		return newUnsafeIFaceType(cfg, type1)
+	default:
+		if cfg.useSafeImplementation {
+			return &safeType
+		}
+		return newUnsafeType(cfg, type1)
+	}
+}
+
+func TypeOf(obj interface{}) Type {
+	return ConfigUnsafe.TypeOf(obj)
+}
+
+func TypeOfPtr(obj interface{}) PtrType {
+	return TypeOf(obj).(PtrType)
+}
+
+func Type2(type1 reflect.Type) Type {
+	if type1 == nil {
+		return nil
+	}
+	return ConfigUnsafe.Type2(type1)
+}
+
+func PtrTo(typ Type) Type {
+	return Type2(reflect.PtrTo(typ.Type1()))
+}
+
+func PtrOf(obj interface{}) unsafe.Pointer {
+	return unpackEFace(obj).data
+}
+
+func RTypeOf(obj interface{}) uintptr {
+	return uintptr(unpackEFace(obj).rtype)
+}
+
+func IsNil(obj interface{}) bool {
+	if obj == nil {
+		return true
+	}
+	return unpackEFace(obj).data == nil
+}
+
+func IsNullable(kind reflect.Kind) bool {
+	switch kind {
+	case reflect.Ptr, reflect.Map, reflect.Chan, reflect.Func, reflect.Slice, reflect.Interface:
+		return true
+	}
+	return false
+}
+
+func likePtrKind(kind reflect.Kind) bool {
+	switch kind {
+	case reflect.Ptr, reflect.Map, reflect.Chan, reflect.Func:
+		return true
+	}
+	return false
+}
+
+func likePtrType(typ reflect.Type) bool {
+	if likePtrKind(typ.Kind()) {
+		return true
+	}
+	if typ.Kind() == reflect.Struct {
+		if typ.NumField() != 1 {
+			return false
+		}
+		return likePtrType(typ.Field(0).Type)
+	}
+	if typ.Kind() == reflect.Array {
+		if typ.Len() != 1 {
+			return false
+		}
+		return likePtrType(typ.Elem())
+	}
+	return false
+}
+
+// NoEscape hides a pointer from escape analysis.  noescape is
+// the identity function but escape analysis doesn't think the
+// output depends on the input.  noescape is inlined and currently
+// compiles down to zero instructions.
+// USE CAREFULLY!
+//go:nosplit
+func NoEscape(p unsafe.Pointer) unsafe.Pointer {
+	x := uintptr(p)
+	return unsafe.Pointer(x ^ 0)
+}
+
+func UnsafeCastString(str string) []byte {
+	stringHeader := (*reflect.StringHeader)(unsafe.Pointer(&str))
+	sliceHeader := &reflect.SliceHeader{
+		Data: stringHeader.Data,
+		Cap: stringHeader.Len,
+		Len: stringHeader.Len,
+	}
+	return *(*[]byte)(unsafe.Pointer(sliceHeader))
+}
diff --git a/vendor/github.com/modern-go/reflect2/reflect2_amd64.s b/vendor/github.com/modern-go/reflect2/reflect2_amd64.s
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/reflect2_amd64.s
diff --git a/vendor/github.com/modern-go/reflect2/reflect2_kind.go b/vendor/github.com/modern-go/reflect2/reflect2_kind.go
new file mode 100644
index 0000000..62f299e
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/reflect2_kind.go
@@ -0,0 +1,30 @@
+package reflect2
+
+import (
+	"reflect"
+	"unsafe"
+)
+
+// DefaultTypeOfKind return the non aliased default type for the kind
+func DefaultTypeOfKind(kind reflect.Kind) Type {
+	return kindTypes[kind]
+}
+
+var kindTypes = map[reflect.Kind]Type{
+	reflect.Bool:          TypeOf(true),
+	reflect.Uint8:         TypeOf(uint8(0)),
+	reflect.Int8:          TypeOf(int8(0)),
+	reflect.Uint16:        TypeOf(uint16(0)),
+	reflect.Int16:         TypeOf(int16(0)),
+	reflect.Uint32:        TypeOf(uint32(0)),
+	reflect.Int32:         TypeOf(int32(0)),
+	reflect.Uint64:        TypeOf(uint64(0)),
+	reflect.Int64:         TypeOf(int64(0)),
+	reflect.Uint:          TypeOf(uint(0)),
+	reflect.Int:           TypeOf(int(0)),
+	reflect.Float32:       TypeOf(float32(0)),
+	reflect.Float64:       TypeOf(float64(0)),
+	reflect.Uintptr:       TypeOf(uintptr(0)),
+	reflect.String:        TypeOf(""),
+	reflect.UnsafePointer: TypeOf(unsafe.Pointer(nil)),
+}
diff --git a/vendor/github.com/modern-go/reflect2/relfect2_386.s b/vendor/github.com/modern-go/reflect2/relfect2_386.s
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/relfect2_386.s
diff --git a/vendor/github.com/modern-go/reflect2/relfect2_amd64p32.s b/vendor/github.com/modern-go/reflect2/relfect2_amd64p32.s
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/relfect2_amd64p32.s
diff --git a/vendor/github.com/modern-go/reflect2/relfect2_arm.s b/vendor/github.com/modern-go/reflect2/relfect2_arm.s
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/relfect2_arm.s
diff --git a/vendor/github.com/modern-go/reflect2/relfect2_arm64.s b/vendor/github.com/modern-go/reflect2/relfect2_arm64.s
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/relfect2_arm64.s
diff --git a/vendor/github.com/modern-go/reflect2/relfect2_mips64x.s b/vendor/github.com/modern-go/reflect2/relfect2_mips64x.s
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/relfect2_mips64x.s
diff --git a/vendor/github.com/modern-go/reflect2/relfect2_mipsx.s b/vendor/github.com/modern-go/reflect2/relfect2_mipsx.s
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/relfect2_mipsx.s
diff --git a/vendor/github.com/modern-go/reflect2/relfect2_ppc64x.s b/vendor/github.com/modern-go/reflect2/relfect2_ppc64x.s
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/relfect2_ppc64x.s
diff --git a/vendor/github.com/modern-go/reflect2/relfect2_s390x.s b/vendor/github.com/modern-go/reflect2/relfect2_s390x.s
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/relfect2_s390x.s
diff --git a/vendor/github.com/modern-go/reflect2/safe_field.go b/vendor/github.com/modern-go/reflect2/safe_field.go
new file mode 100644
index 0000000..d4ba1f4
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/safe_field.go
@@ -0,0 +1,58 @@
+package reflect2
+
+import (
+	"reflect"
+	"unsafe"
+)
+
+type safeField struct {
+	reflect.StructField
+}
+
+func (field *safeField) Offset() uintptr {
+	return field.StructField.Offset
+}
+
+func (field *safeField) Name() string {
+	return field.StructField.Name
+}
+
+func (field *safeField) PkgPath() string {
+	return field.StructField.PkgPath
+}
+
+func (field *safeField) Type() Type {
+	panic("not implemented")
+}
+
+func (field *safeField) Tag() reflect.StructTag {
+	return field.StructField.Tag
+}
+
+func (field *safeField) Index() []int {
+	return field.StructField.Index
+}
+
+func (field *safeField) Anonymous() bool {
+	return field.StructField.Anonymous
+}
+
+func (field *safeField) Set(obj interface{}, value interface{}) {
+	val := reflect.ValueOf(obj).Elem()
+	val.FieldByIndex(field.Index()).Set(reflect.ValueOf(value).Elem())
+}
+
+func (field *safeField) UnsafeSet(obj unsafe.Pointer, value unsafe.Pointer) {
+	panic("unsafe operation is not supported")
+}
+
+func (field *safeField) Get(obj interface{}) interface{} {
+	val := reflect.ValueOf(obj).Elem().FieldByIndex(field.Index())
+	ptr := reflect.New(val.Type())
+	ptr.Elem().Set(val)
+	return ptr.Interface()
+}
+
+func (field *safeField) UnsafeGet(obj unsafe.Pointer) unsafe.Pointer {
+	panic("does not support unsafe operation")
+}
diff --git a/vendor/github.com/modern-go/reflect2/safe_map.go b/vendor/github.com/modern-go/reflect2/safe_map.go
new file mode 100644
index 0000000..8836220
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/safe_map.go
@@ -0,0 +1,101 @@
+package reflect2
+
+import (
+	"reflect"
+	"unsafe"
+)
+
+type safeMapType struct {
+	safeType
+}
+
+func (type2 *safeMapType) Key() Type {
+	return type2.safeType.cfg.Type2(type2.Type.Key())
+}
+
+func (type2 *safeMapType) MakeMap(cap int) interface{} {
+	ptr := reflect.New(type2.Type)
+	ptr.Elem().Set(reflect.MakeMap(type2.Type))
+	return ptr.Interface()
+}
+
+func (type2 *safeMapType) UnsafeMakeMap(cap int) unsafe.Pointer {
+	panic("does not support unsafe operation")
+}
+
+func (type2 *safeMapType) SetIndex(obj interface{}, key interface{}, elem interface{}) {
+	keyVal := reflect.ValueOf(key)
+	elemVal := reflect.ValueOf(elem)
+	val := reflect.ValueOf(obj)
+	val.Elem().SetMapIndex(keyVal.Elem(), elemVal.Elem())
+}
+
+func (type2 *safeMapType) UnsafeSetIndex(obj unsafe.Pointer, key unsafe.Pointer, elem unsafe.Pointer) {
+	panic("does not support unsafe operation")
+}
+
+func (type2 *safeMapType) TryGetIndex(obj interface{}, key interface{}) (interface{}, bool) {
+	keyVal := reflect.ValueOf(key)
+	if key == nil {
+		keyVal = reflect.New(type2.Type.Key()).Elem()
+	}
+	val := reflect.ValueOf(obj).MapIndex(keyVal)
+	if !val.IsValid() {
+		return nil, false
+	}
+	return val.Interface(), true
+}
+
+func (type2 *safeMapType) GetIndex(obj interface{}, key interface{}) interface{} {
+	val := reflect.ValueOf(obj).Elem()
+	keyVal := reflect.ValueOf(key).Elem()
+	elemVal := val.MapIndex(keyVal)
+	if !elemVal.IsValid() {
+		ptr := reflect.New(reflect.PtrTo(val.Type().Elem()))
+		return ptr.Elem().Interface()
+	}
+	ptr := reflect.New(elemVal.Type())
+	ptr.Elem().Set(elemVal)
+	return ptr.Interface()
+}
+
+func (type2 *safeMapType) UnsafeGetIndex(obj unsafe.Pointer, key unsafe.Pointer) unsafe.Pointer {
+	panic("does not support unsafe operation")
+}
+
+func (type2 *safeMapType) Iterate(obj interface{}) MapIterator {
+	m := reflect.ValueOf(obj).Elem()
+	return &safeMapIterator{
+		m:    m,
+		keys: m.MapKeys(),
+	}
+}
+
+func (type2 *safeMapType) UnsafeIterate(obj unsafe.Pointer) MapIterator {
+	panic("does not support unsafe operation")
+}
+
+type safeMapIterator struct {
+	i    int
+	m    reflect.Value
+	keys []reflect.Value
+}
+
+func (iter *safeMapIterator) HasNext() bool {
+	return iter.i != len(iter.keys)
+}
+
+func (iter *safeMapIterator) Next() (interface{}, interface{}) {
+	key := iter.keys[iter.i]
+	elem := iter.m.MapIndex(key)
+	iter.i += 1
+	keyPtr := reflect.New(key.Type())
+	keyPtr.Elem().Set(key)
+	elemPtr := reflect.New(elem.Type())
+	elemPtr.Elem().Set(elem)
+	return keyPtr.Interface(), elemPtr.Interface()
+}
+
+func (iter *safeMapIterator) UnsafeNext() (unsafe.Pointer, unsafe.Pointer) {
+	panic("does not support unsafe operation")
+}
diff --git a/vendor/github.com/modern-go/reflect2/safe_slice.go b/vendor/github.com/modern-go/reflect2/safe_slice.go
new file mode 100644
index 0000000..bcce6fd
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/safe_slice.go
@@ -0,0 +1,92 @@
+package reflect2
+
+import (
+	"reflect"
+	"unsafe"
+)
+
+type safeSliceType struct {
+	safeType
+}
+
+func (type2 *safeSliceType) SetIndex(obj interface{}, index int, value interface{}) {
+	val := reflect.ValueOf(obj).Elem()
+	elem := reflect.ValueOf(value).Elem()
+	val.Index(index).Set(elem)
+}
+
+func (type2 *safeSliceType) UnsafeSetIndex(obj unsafe.Pointer, index int, value unsafe.Pointer) {
+	panic("does not support unsafe operation")
+}
+
+func (type2 *safeSliceType) GetIndex(obj interface{}, index int) interface{} {
+	val := reflect.ValueOf(obj).Elem()
+	elem := val.Index(index)
+	ptr := reflect.New(elem.Type())
+	ptr.Elem().Set(elem)
+	return ptr.Interface()
+}
+
+func (type2 *safeSliceType) UnsafeGetIndex(obj unsafe.Pointer, index int) unsafe.Pointer {
+	panic("does not support unsafe operation")
+}
+
+func (type2 *safeSliceType) MakeSlice(length int, cap int) interface{} {
+	val := reflect.MakeSlice(type2.Type, length, cap)
+	ptr := reflect.New(val.Type())
+	ptr.Elem().Set(val)
+	return ptr.Interface()
+}
+
+func (type2 *safeSliceType) UnsafeMakeSlice(length int, cap int) unsafe.Pointer {
+	panic("does not support unsafe operation")
+}
+
+func (type2 *safeSliceType) Grow(obj interface{}, newLength int) {
+	oldCap := type2.Cap(obj)
+	oldSlice := reflect.ValueOf(obj).Elem()
+	delta := newLength - oldCap
+	deltaVals := make([]reflect.Value, delta)
+	newSlice := reflect.Append(oldSlice, deltaVals...)
+	oldSlice.Set(newSlice)
+}
+
+func (type2 *safeSliceType) UnsafeGrow(ptr unsafe.Pointer, newLength int) {
+	panic("does not support unsafe operation")
+}
+
+func (type2 *safeSliceType) Append(obj interface{}, elem interface{}) {
+	val := reflect.ValueOf(obj).Elem()
+	elemVal := reflect.ValueOf(elem).Elem()
+	newVal := reflect.Append(val, elemVal)
+	val.Set(newVal)
+}
+
+func (type2 *safeSliceType) UnsafeAppend(obj unsafe.Pointer, elem unsafe.Pointer) {
+	panic("does not support unsafe operation")
+}
+
+func (type2 *safeSliceType) SetNil(obj interface{}) {
+	val := reflect.ValueOf(obj).Elem()
+	val.Set(reflect.Zero(val.Type()))
+}
+
+func (type2 *safeSliceType) UnsafeSetNil(ptr unsafe.Pointer) {
+	panic("does not support unsafe operation")
+}
+
+func (type2 *safeSliceType) LengthOf(obj interface{}) int {
+	return reflect.ValueOf(obj).Elem().Len()
+}
+
+func (type2 *safeSliceType) UnsafeLengthOf(ptr unsafe.Pointer) int {
+	panic("does not support unsafe operation")
+}
+
+func (type2 *safeSliceType) Cap(obj interface{}) int {
+	return reflect.ValueOf(obj).Elem().Cap()
+}
+
+func (type2 *safeSliceType) UnsafeCap(ptr unsafe.Pointer) int {
+	panic("does not support unsafe operation")
+}
diff --git a/vendor/github.com/modern-go/reflect2/safe_struct.go b/vendor/github.com/modern-go/reflect2/safe_struct.go
new file mode 100644
index 0000000..e5fb9b3
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/safe_struct.go
@@ -0,0 +1,29 @@
+package reflect2
+
+type safeStructType struct {
+	safeType
+}
+
+func (type2 *safeStructType) FieldByName(name string) StructField {
+	field, found := type2.Type.FieldByName(name)
+	if !found {
+		panic("field " + name + " not found")
+	}
+	return &safeField{StructField: field}
+}
+
+func (type2 *safeStructType) Field(i int) StructField {
+	return &safeField{StructField: type2.Type.Field(i)}
+}
+
+func (type2 *safeStructType) FieldByIndex(index []int) StructField {
+	return &safeField{StructField: type2.Type.FieldByIndex(index)}
+}
+
+func (type2 *safeStructType) FieldByNameFunc(match func(string) bool) StructField {
+	field, found := type2.Type.FieldByNameFunc(match)
+	if !found {
+		panic("field match condition not found in " + type2.Type.String())
+	}
+	return &safeField{StructField: field}
+}
diff --git a/vendor/github.com/modern-go/reflect2/safe_type.go b/vendor/github.com/modern-go/reflect2/safe_type.go
new file mode 100644
index 0000000..ee4e7bb
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/safe_type.go
@@ -0,0 +1,78 @@
+package reflect2
+
+import (
+	"reflect"
+	"unsafe"
+)
+
+type safeType struct {
+	reflect.Type
+	cfg *frozenConfig
+}
+
+func (type2 *safeType) New() interface{} {
+	return reflect.New(type2.Type).Interface()
+}
+
+func (type2 *safeType) UnsafeNew() unsafe.Pointer {
+	panic("does not support unsafe operation")
+}
+
+func (type2 *safeType) Elem() Type {
+	return type2.cfg.Type2(type2.Type.Elem())
+}
+
+func (type2 *safeType) Type1() reflect.Type {
+	return type2.Type
+}
+
+func (type2 *safeType) PackEFace(ptr unsafe.Pointer) interface{} {
+	panic("does not support unsafe operation")
+}
+
+func (type2 *safeType) Implements(thatType Type) bool {
+	return type2.Type.Implements(thatType.Type1())
+}
+
+func (type2 *safeType) RType() uintptr {
+	panic("does not support unsafe operation")
+}
+
+func (type2 *safeType) Indirect(obj interface{}) interface{} {
+	return reflect.Indirect(reflect.ValueOf(obj)).Interface()
+}
+
+func (type2 *safeType) UnsafeIndirect(ptr unsafe.Pointer) interface{} {
+	panic("does not support unsafe operation")
+}
+
+func (type2 *safeType) LikePtr() bool {
+	panic("does not support unsafe operation")
+}
+
+func (type2 *safeType) IsNullable() bool {
+	return IsNullable(type2.Kind())
+}
+
+func (type2 *safeType) IsNil(obj interface{}) bool {
+	if obj == nil {
+		return true
+	}
+	return reflect.ValueOf(obj).Elem().IsNil()
+}
+
+func (type2 *safeType) UnsafeIsNil(ptr unsafe.Pointer) bool {
+	panic("does not support unsafe operation")
+}
+
+func (type2 *safeType) Set(obj interface{}, val interface{}) {
+	reflect.ValueOf(obj).Elem().Set(reflect.ValueOf(val).Elem())
+}
+
+func (type2 *safeType) UnsafeSet(ptr unsafe.Pointer, val unsafe.Pointer) {
+	panic("does not support unsafe operation")
+}
+
+func (type2 *safeType) AssignableTo(anotherType Type) bool {
+	return type2.Type1().AssignableTo(anotherType.Type1())
+}
diff --git a/vendor/github.com/modern-go/reflect2/test.sh b/vendor/github.com/modern-go/reflect2/test.sh
new file mode 100644
index 0000000..3d2b976
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/test.sh
@@ -0,0 +1,12 @@
+#!/usr/bin/env bash
+
+set -e
+echo "" > coverage.txt
+
+for d in $(go list github.com/modern-go/reflect2-tests/... | grep -v vendor); do
+    go test -coverprofile=profile.out -coverpkg=github.com/modern-go/reflect2 $d
+    if [ -f profile.out ]; then
+        cat profile.out >> coverage.txt
+        rm profile.out
+    fi
+done
diff --git a/vendor/github.com/modern-go/reflect2/type_map.go b/vendor/github.com/modern-go/reflect2/type_map.go
new file mode 100644
index 0000000..3acfb55
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/type_map.go
@@ -0,0 +1,113 @@
+package reflect2
+
+import (
+	"reflect"
+	"runtime"
+	"strings"
+	"sync"
+	"unsafe"
+)
+
+// typelinks1 for 1.5 ~ 1.6
+//go:linkname typelinks1 reflect.typelinks
+func typelinks1() [][]unsafe.Pointer
+
+// typelinks2 for 1.7 ~
+//go:linkname typelinks2 reflect.typelinks
+func typelinks2() (sections []unsafe.Pointer, offset [][]int32)
+
+// initOnce guards initialization of types and packages
+var initOnce sync.Once
+
+var types map[string]reflect.Type
+var packages map[string]map[string]reflect.Type
+
+// discoverTypes initializes types and packages
+func discoverTypes() {
+	types = make(map[string]reflect.Type)
+	packages = make(map[string]map[string]reflect.Type)
+
+	ver := runtime.Version()
+	if ver == "go1.5" || strings.HasPrefix(ver, "go1.5.") {
+		loadGo15Types()
+	} else if ver == "go1.6" || strings.HasPrefix(ver, "go1.6.") {
+		loadGo15Types()
+	} else {
+		loadGo17Types()
+	}
+}
+
+func loadGo15Types() {
+	var obj interface{} = reflect.TypeOf(0)
+	typePtrss := typelinks1()
+	for _, typePtrs := range typePtrss {
+		for _, typePtr := range typePtrs {
+			(*emptyInterface)(unsafe.Pointer(&obj)).word = typePtr
+			typ := obj.(reflect.Type)
+			if typ.Kind() == reflect.Ptr && typ.Elem().Kind() == reflect.Struct {
+				loadedType := typ.Elem()
+				pkgTypes := packages[loadedType.PkgPath()]
+				if pkgTypes == nil {
+					pkgTypes = map[string]reflect.Type{}
+					packages[loadedType.PkgPath()] = pkgTypes
+				}
+				types[loadedType.String()] = loadedType
+				pkgTypes[loadedType.Name()] = loadedType
+			}
+			if typ.Kind() == reflect.Slice && typ.Elem().Kind() == reflect.Ptr &&
+				typ.Elem().Elem().Kind() == reflect.Struct {
+				loadedType := typ.Elem().Elem()
+				pkgTypes := packages[loadedType.PkgPath()]
+				if pkgTypes == nil {
+					pkgTypes = map[string]reflect.Type{}
+					packages[loadedType.PkgPath()] = pkgTypes
+				}
+				types[loadedType.String()] = loadedType
+				pkgTypes[loadedType.Name()] = loadedType
+			}
+		}
+	}
+}
+
+func loadGo17Types() {
+	var obj interface{} = reflect.TypeOf(0)
+	sections, offset := typelinks2()
+	for i, offs := range offset {
+		rodata := sections[i]
+		for _, off := range offs {
+			(*emptyInterface)(unsafe.Pointer(&obj)).word = resolveTypeOff(unsafe.Pointer(rodata), off)
+			typ := obj.(reflect.Type)
+			if typ.Kind() == reflect.Ptr && typ.Elem().Kind() == reflect.Struct {
+				loadedType := typ.Elem()
+				pkgTypes := packages[loadedType.PkgPath()]
+				if pkgTypes == nil {
+					pkgTypes = map[string]reflect.Type{}
+					packages[loadedType.PkgPath()] = pkgTypes
+				}
+				types[loadedType.String()] = loadedType
+				pkgTypes[loadedType.Name()] = loadedType
+			}
+		}
+	}
+}
+
+type emptyInterface struct {
+	typ  unsafe.Pointer
+	word unsafe.Pointer
+}
+
+// TypeByName return the type by its name, just like Class.forName in java
+func TypeByName(typeName string) Type {
+	initOnce.Do(discoverTypes)
+	return Type2(types[typeName])
+}
+
+// TypeByPackageName return the type by its package and name
+func TypeByPackageName(pkgPath string, name string) Type {
+	initOnce.Do(discoverTypes)
+	pkgTypes := packages[pkgPath]
+	if pkgTypes == nil {
+		return nil
+	}
+	return Type2(pkgTypes[name])
+}
diff --git a/vendor/github.com/modern-go/reflect2/unsafe_array.go b/vendor/github.com/modern-go/reflect2/unsafe_array.go
new file mode 100644
index 0000000..76cbdba
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/unsafe_array.go
@@ -0,0 +1,65 @@
+package reflect2
+
+import (
+	"reflect"
+	"unsafe"
+)
+
+type UnsafeArrayType struct {
+	unsafeType
+	elemRType  unsafe.Pointer
+	pElemRType unsafe.Pointer
+	elemSize   uintptr
+	likePtr    bool
+}
+
+func newUnsafeArrayType(cfg *frozenConfig, type1 reflect.Type) *UnsafeArrayType {
+	return &UnsafeArrayType{
+		unsafeType: *newUnsafeType(cfg, type1),
+		elemRType:  unpackEFace(type1.Elem()).data,
+		pElemRType: unpackEFace(reflect.PtrTo(type1.Elem())).data,
+		elemSize:   type1.Elem().Size(),
+		likePtr:    likePtrType(type1),
+	}
+}
+
+func (type2 *UnsafeArrayType) LikePtr() bool {
+	return type2.likePtr
+}
+
+func (type2 *UnsafeArrayType) Indirect(obj interface{}) interface{} {
+	objEFace := unpackEFace(obj)
+	assertType("Type.Indirect argument 1", type2.ptrRType, objEFace.rtype)
+	return type2.UnsafeIndirect(objEFace.data)
+}
+
+func (type2 *UnsafeArrayType) UnsafeIndirect(ptr unsafe.Pointer) interface{} {
+	if type2.likePtr {
+		return packEFace(type2.rtype, *(*unsafe.Pointer)(ptr))
+	}
+	return packEFace(type2.rtype, ptr)
+}
+
+func (type2 *UnsafeArrayType) SetIndex(obj interface{}, index int, elem interface{}) {
+	objEFace := unpackEFace(obj)
+	assertType("ArrayType.SetIndex argument 1", type2.ptrRType, objEFace.rtype)
+	elemEFace := unpackEFace(elem)
+	assertType("ArrayType.SetIndex argument 3", type2.pElemRType, elemEFace.rtype)
+	type2.UnsafeSetIndex(objEFace.data, index, elemEFace.data)
+}
+
+func (type2 *UnsafeArrayType) UnsafeSetIndex(obj unsafe.Pointer, index int, elem unsafe.Pointer) {
+	elemPtr := arrayAt(obj, index, type2.elemSize, "i < s.Len")
+	typedmemmove(type2.elemRType, elemPtr, elem)
+}
+
+func (type2 *UnsafeArrayType) GetIndex(obj interface{}, index int) interface{} {
+	objEFace := unpackEFace(obj)
+	assertType("ArrayType.GetIndex argument 1", type2.ptrRType, objEFace.rtype)
+	elemPtr := type2.UnsafeGetIndex(objEFace.data, index)
+	return packEFace(type2.pElemRType, elemPtr)
+}
+
+func (type2 *UnsafeArrayType) UnsafeGetIndex(obj unsafe.Pointer, index int) unsafe.Pointer {
+	return arrayAt(obj, index, type2.elemSize, "i < s.Len")
+}
diff --git a/vendor/github.com/modern-go/reflect2/unsafe_eface.go b/vendor/github.com/modern-go/reflect2/unsafe_eface.go
new file mode 100644
index 0000000..805010f
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/unsafe_eface.go
@@ -0,0 +1,59 @@
+package reflect2
+
+import (
+	"reflect"
+	"unsafe"
+)
+
+type eface struct {
+	rtype unsafe.Pointer
+	data  unsafe.Pointer
+}
+
+func unpackEFace(obj interface{}) *eface {
+	return (*eface)(unsafe.Pointer(&obj))
+}
+
+func packEFace(rtype unsafe.Pointer, data unsafe.Pointer) interface{} {
+	var i interface{}
+	e := (*eface)(unsafe.Pointer(&i))
+	e.rtype = rtype
+	e.data = data
+	return i
+}
+
+type UnsafeEFaceType struct {
+	unsafeType
+}
+
+func newUnsafeEFaceType(cfg *frozenConfig, type1 reflect.Type) *UnsafeEFaceType {
+	return &UnsafeEFaceType{
+		unsafeType: *newUnsafeType(cfg, type1),
+	}
+}
+
+func (type2 *UnsafeEFaceType) IsNil(obj interface{}) bool {
+	if obj == nil {
+		return true
+	}
+	objEFace := unpackEFace(obj)
+	assertType("Type.IsNil argument 1", type2.ptrRType, objEFace.rtype)
+	return type2.UnsafeIsNil(objEFace.data)
+}
+
+func (type2 *UnsafeEFaceType) UnsafeIsNil(ptr unsafe.Pointer) bool {
+	if ptr == nil {
+		return true
+	}
+	return unpackEFace(*(*interface{})(ptr)).data == nil
+}
+
+func (type2 *UnsafeEFaceType) Indirect(obj interface{}) interface{} {
+	objEFace := unpackEFace(obj)
+	assertType("Type.Indirect argument 1", type2.ptrRType, objEFace.rtype)
+	return type2.UnsafeIndirect(objEFace.data)
+}
+
+func (type2 *UnsafeEFaceType) UnsafeIndirect(ptr unsafe.Pointer) interface{} {
+	return *(*interface{})(ptr)
+}
diff --git a/vendor/github.com/modern-go/reflect2/unsafe_field.go b/vendor/github.com/modern-go/reflect2/unsafe_field.go
new file mode 100644
index 0000000..5eb5313
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/unsafe_field.go
@@ -0,0 +1,74 @@
+package reflect2
+
+import (
+	"reflect"
+	"unsafe"
+)
+
+type UnsafeStructField struct {
+	reflect.StructField
+	structType *UnsafeStructType
+	rtype      unsafe.Pointer
+	ptrRType   unsafe.Pointer
+}
+
+func newUnsafeStructField(structType *UnsafeStructType, structField reflect.StructField) *UnsafeStructField {
+	return &UnsafeStructField{
+		StructField: structField,
+		rtype:       unpackEFace(structField.Type).data,
+		ptrRType:    unpackEFace(reflect.PtrTo(structField.Type)).data,
+		structType:  structType,
+	}
+}
+
+func (field *UnsafeStructField) Offset() uintptr {
+	return field.StructField.Offset
+}
+
+func (field *UnsafeStructField) Name() string {
+	return field.StructField.Name
+}
+
+func (field *UnsafeStructField) PkgPath() string {
+	return field.StructField.PkgPath
+}
+
+func (field *UnsafeStructField) Type() Type {
+	return field.structType.cfg.Type2(field.StructField.Type)
+}
+
+func (field *UnsafeStructField) Tag() reflect.StructTag {
+	return field.StructField.Tag
+}
+
+func (field *UnsafeStructField) Index() []int {
+	return field.StructField.Index
+}
+
+func (field *UnsafeStructField) Anonymous() bool {
+	return field.StructField.Anonymous
+}
+
+func (field *UnsafeStructField) Set(obj interface{}, value interface{}) {
+	objEFace := unpackEFace(obj)
+	assertType("StructField.SetIndex argument 1", field.structType.ptrRType, objEFace.rtype)
+	valueEFace := unpackEFace(value)
+	assertType("StructField.SetIndex argument 2", field.ptrRType, valueEFace.rtype)
+	field.UnsafeSet(objEFace.data, valueEFace.data)
+}
+
+func (field *UnsafeStructField) UnsafeSet(obj unsafe.Pointer, value unsafe.Pointer) {
+	fieldPtr := add(obj, field.StructField.Offset, "same as non-reflect &v.field")
+	typedmemmove(field.rtype, fieldPtr, value)
+}
+
+func (field *UnsafeStructField) Get(obj interface{}) interface{} {
+	objEFace := unpackEFace(obj)
+	assertType("StructField.GetIndex argument 1", field.structType.ptrRType, objEFace.rtype)
+	value := field.UnsafeGet(objEFace.data)
+	return packEFace(field.ptrRType, value)
+}
+
+func (field *UnsafeStructField) UnsafeGet(obj unsafe.Pointer) unsafe.Pointer {
+	return add(obj, field.StructField.Offset, "same as non-reflect &v.field")
+}
diff --git a/vendor/github.com/modern-go/reflect2/unsafe_iface.go b/vendor/github.com/modern-go/reflect2/unsafe_iface.go
new file mode 100644
index 0000000..b601955
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/unsafe_iface.go
@@ -0,0 +1,64 @@
+package reflect2
+
+import (
+	"reflect"
+	"unsafe"
+)
+
+type iface struct {
+	itab *itab
+	data unsafe.Pointer
+}
+
+type itab struct {
+	ignore unsafe.Pointer
+	rtype  unsafe.Pointer
+}
+
+func IFaceToEFace(ptr unsafe.Pointer) interface{} {
+	iface := (*iface)(ptr)
+	if iface.itab == nil {
+		return nil
+	}
+	return packEFace(iface.itab.rtype, iface.data)
+}
+
+type UnsafeIFaceType struct {
+	unsafeType
+}
+
+func newUnsafeIFaceType(cfg *frozenConfig, type1 reflect.Type) *UnsafeIFaceType {
+	return &UnsafeIFaceType{
+		unsafeType: *newUnsafeType(cfg, type1),
+	}
+}
+
+func (type2 *UnsafeIFaceType) Indirect(obj interface{}) interface{} {
+	objEFace := unpackEFace(obj)
+	assertType("Type.Indirect argument 1", type2.ptrRType, objEFace.rtype)
+	return type2.UnsafeIndirect(objEFace.data)
+}
+
+func (type2 *UnsafeIFaceType) UnsafeIndirect(ptr unsafe.Pointer) interface{} {
+	return IFaceToEFace(ptr)
+}
+
+func (type2 *UnsafeIFaceType) IsNil(obj interface{}) bool {
+	if obj == nil {
+		return true
+	}
+	objEFace := unpackEFace(obj)
+	assertType("Type.IsNil argument 1", type2.ptrRType, objEFace.rtype)
+	return type2.UnsafeIsNil(objEFace.data)
+}
+
+func (type2 *UnsafeIFaceType) UnsafeIsNil(ptr unsafe.Pointer) bool {
+	if ptr == nil {
+		return true
+	}
+	iface := (*iface)(ptr)
+	if iface.itab == nil {
+		return true
+	}
+	return false
+}
diff --git a/vendor/github.com/modern-go/reflect2/unsafe_link.go b/vendor/github.com/modern-go/reflect2/unsafe_link.go
new file mode 100644
index 0000000..57229c8
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/unsafe_link.go
@@ -0,0 +1,70 @@
+package reflect2
+
+import "unsafe"
+
+//go:linkname unsafe_New reflect.unsafe_New
+func unsafe_New(rtype unsafe.Pointer) unsafe.Pointer
+
+//go:linkname typedmemmove reflect.typedmemmove
+func typedmemmove(rtype unsafe.Pointer, dst, src unsafe.Pointer)
+
+//go:linkname unsafe_NewArray reflect.unsafe_NewArray
+func unsafe_NewArray(rtype unsafe.Pointer, length int) unsafe.Pointer
+
+// typedslicecopy copies a slice of elemType values from src to dst,
+// returning the number of elements copied.
+//go:linkname typedslicecopy reflect.typedslicecopy
+//go:noescape
+func typedslicecopy(elemType unsafe.Pointer, dst, src sliceHeader) int
+
+//go:linkname mapassign reflect.mapassign
+//go:noescape
+func mapassign(rtype unsafe.Pointer, m unsafe.Pointer, key, val unsafe.Pointer)
+
+//go:linkname mapaccess reflect.mapaccess
+//go:noescape
+func mapaccess(rtype unsafe.Pointer, m unsafe.Pointer, key unsafe.Pointer) (val unsafe.Pointer)
+
+// m escapes into the return value, but the caller of mapiterinit
+// doesn't let the return value escape.
+//go:noescape
+//go:linkname mapiterinit reflect.mapiterinit
+func mapiterinit(rtype unsafe.Pointer, m unsafe.Pointer) *hiter
+
+//go:noescape
+//go:linkname mapiternext reflect.mapiternext
+func mapiternext(it *hiter)
+
+//go:linkname ifaceE2I reflect.ifaceE2I
+func ifaceE2I(rtype unsafe.Pointer, src interface{}, dst unsafe.Pointer)
+
+// A hash iteration structure.
+// If you modify hiter, also change cmd/internal/gc/reflect.go to indicate
+// the layout of this structure.
+type hiter struct {
+	key   unsafe.Pointer // Must be in first position.  Write nil to indicate iteration end (see cmd/internal/gc/range.go).
+	value unsafe.Pointer // Must be in second position (see cmd/internal/gc/range.go).
+	// rest fields are ignored
+}
+
+// add returns p+x.
+//
+// The whySafe string is ignored, so that the function still inlines
+// as efficiently as p+x, but all call sites should use the string to
+// record why the addition is safe, which is to say why the addition
+// does not cause x to advance to the very end of p's allocation
+// and therefore point incorrectly at the next block in memory.
+func add(p unsafe.Pointer, x uintptr, whySafe string) unsafe.Pointer {
+	return unsafe.Pointer(uintptr(p) + x)
+}
+
+// arrayAt returns the i-th element of p,
+// an array whose elements are eltSize bytes wide.
+// The array pointed at by p must have at least i+1 elements:
+// it is invalid (but impossible to check here) to pass i >= len,
+// because then the result will point outside the array.
+// whySafe must explain why i < len. (Passing "i < len" is fine;
+// the benefit is to surface this assumption at the call site.)
+func arrayAt(p unsafe.Pointer, i int, eltSize uintptr, whySafe string) unsafe.Pointer {
+	return add(p, uintptr(i)*eltSize, "i < len")
+}
diff --git a/vendor/github.com/modern-go/reflect2/unsafe_map.go b/vendor/github.com/modern-go/reflect2/unsafe_map.go
new file mode 100644
index 0000000..f2e76e6
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/unsafe_map.go
@@ -0,0 +1,138 @@
+package reflect2
+
+import (
+	"reflect"
+	"unsafe"
+)
+
+type UnsafeMapType struct {
+	unsafeType
+	pKeyRType  unsafe.Pointer
+	pElemRType unsafe.Pointer
+}
+
+func newUnsafeMapType(cfg *frozenConfig, type1 reflect.Type) MapType {
+	return &UnsafeMapType{
+		unsafeType: *newUnsafeType(cfg, type1),
+		pKeyRType:  unpackEFace(reflect.PtrTo(type1.Key())).data,
+		pElemRType: unpackEFace(reflect.PtrTo(type1.Elem())).data,
+	}
+}
+
+func (type2 *UnsafeMapType) IsNil(obj interface{}) bool {
+	if obj == nil {
+		return true
+	}
+	objEFace := unpackEFace(obj)
+	assertType("Type.IsNil argument 1", type2.ptrRType, objEFace.rtype)
+	return type2.UnsafeIsNil(objEFace.data)
+}
+
+func (type2 *UnsafeMapType) UnsafeIsNil(ptr unsafe.Pointer) bool {
+	if ptr == nil {
+		return true
+	}
+	return *(*unsafe.Pointer)(ptr) == nil
+}
+
+func (type2 *UnsafeMapType) LikePtr() bool {
+	return true
+}
+
+func (type2 *UnsafeMapType) Indirect(obj interface{}) interface{} {
+	objEFace := unpackEFace(obj)
+	assertType("MapType.Indirect argument 1", type2.ptrRType, objEFace.rtype)
+	return type2.UnsafeIndirect(objEFace.data)
+}
+
+func (type2 *UnsafeMapType) UnsafeIndirect(ptr unsafe.Pointer) interface{} {
+	return packEFace(type2.rtype, *(*unsafe.Pointer)(ptr))
+}
+
+func (type2 *UnsafeMapType) Key() Type {
+	return type2.cfg.Type2(type2.Type.Key())
+}
+
+func (type2 *UnsafeMapType) MakeMap(cap int) interface{} {
+	return packEFace(type2.ptrRType, type2.UnsafeMakeMap(cap))
+}
+
+func (type2 *UnsafeMapType) UnsafeMakeMap(cap int) unsafe.Pointer {
+	m := makeMapWithSize(type2.rtype, cap)
+	return unsafe.Pointer(&m)
+}
+
+func (type2 *UnsafeMapType) SetIndex(obj interface{}, key interface{}, elem interface{}) {
+	objEFace := unpackEFace(obj)
+	assertType("MapType.SetIndex argument 1", type2.ptrRType, objEFace.rtype)
+	keyEFace := unpackEFace(key)
+	assertType("MapType.SetIndex argument 2", type2.pKeyRType, keyEFace.rtype)
+	elemEFace := unpackEFace(elem)
+	assertType("MapType.SetIndex argument 3", type2.pElemRType, elemEFace.rtype)
+	type2.UnsafeSetIndex(objEFace.data, keyEFace.data, elemEFace.data)
+}
+
+func (type2 *UnsafeMapType) UnsafeSetIndex(obj unsafe.Pointer, key unsafe.Pointer, elem unsafe.Pointer) {
+	mapassign(type2.rtype, *(*unsafe.Pointer)(obj), key, elem)
+}
+
+func (type2 *UnsafeMapType) TryGetIndex(obj interface{}, key interface{}) (interface{}, bool) {
+	objEFace := unpackEFace(obj)
+	assertType("MapType.TryGetIndex argument 1", type2.ptrRType, objEFace.rtype)
+	keyEFace := unpackEFace(key)
+	assertType("MapType.TryGetIndex argument 2", type2.pKeyRType, keyEFace.rtype)
+	elemPtr := type2.UnsafeGetIndex(objEFace.data, keyEFace.data)
+	if elemPtr == nil {
+		return nil, false
+	}
+	return packEFace(type2.pElemRType, elemPtr), true
+}
+
+func (type2 *UnsafeMapType) GetIndex(obj interface{}, key interface{}) interface{} {
+	objEFace := unpackEFace(obj)
+	assertType("MapType.GetIndex argument 1", type2.ptrRType, objEFace.rtype)
+	keyEFace := unpackEFace(key)
+	assertType("MapType.GetIndex argument 2", type2.pKeyRType, keyEFace.rtype)
+	elemPtr := type2.UnsafeGetIndex(objEFace.data, keyEFace.data)
+	return packEFace(type2.pElemRType, elemPtr)
+}
+
+func (type2 *UnsafeMapType) UnsafeGetIndex(obj unsafe.Pointer, key unsafe.Pointer) unsafe.Pointer {
+	return mapaccess(type2.rtype, *(*unsafe.Pointer)(obj), key)
+}
+
+func (type2 *UnsafeMapType) Iterate(obj interface{}) MapIterator {
+	objEFace := unpackEFace(obj)
+	assertType("MapType.Iterate argument 1", type2.ptrRType, objEFace.rtype)
+	return type2.UnsafeIterate(objEFace.data)
+}
+
+func (type2 *UnsafeMapType) UnsafeIterate(obj unsafe.Pointer) MapIterator {
+	return &UnsafeMapIterator{
+		hiter:      mapiterinit(type2.rtype, *(*unsafe.Pointer)(obj)),
+		pKeyRType:  type2.pKeyRType,
+		pElemRType: type2.pElemRType,
+	}
+}
+
+type UnsafeMapIterator struct {
+	*hiter
+	pKeyRType  unsafe.Pointer
+	pElemRType unsafe.Pointer
+}
+
+func (iter *UnsafeMapIterator) HasNext() bool {
+	return iter.key != nil
+}
+
+func (iter *UnsafeMapIterator) Next() (interface{}, interface{}) {
+	key, elem := iter.UnsafeNext()
+	return packEFace(iter.pKeyRType, key), packEFace(iter.pElemRType, elem)
+}
+
+func (iter *UnsafeMapIterator) UnsafeNext() (unsafe.Pointer, unsafe.Pointer) {
+	key := iter.key
+	elem := iter.value
+	mapiternext(iter.hiter)
+	return key, elem
+}
diff --git a/vendor/github.com/modern-go/reflect2/unsafe_ptr.go b/vendor/github.com/modern-go/reflect2/unsafe_ptr.go
new file mode 100644
index 0000000..8e5ec9c
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/unsafe_ptr.go
@@ -0,0 +1,46 @@
+package reflect2
+
+import (
+	"reflect"
+	"unsafe"
+)
+
+type UnsafePtrType struct {
+	unsafeType
+}
+
+func newUnsafePtrType(cfg *frozenConfig, type1 reflect.Type) *UnsafePtrType {
+	return &UnsafePtrType{
+		unsafeType: *newUnsafeType(cfg, type1),
+	}
+}
+
+func (type2 *UnsafePtrType) IsNil(obj interface{}) bool {
+	if obj == nil {
+		return true
+	}
+	objEFace := unpackEFace(obj)
+	assertType("Type.IsNil argument 1", type2.ptrRType, objEFace.rtype)
+	return type2.UnsafeIsNil(objEFace.data)
+}
+
+func (type2 *UnsafePtrType) UnsafeIsNil(ptr unsafe.Pointer) bool {
+	if ptr == nil {
+		return true
+	}
+	return *(*unsafe.Pointer)(ptr) == nil
+}
+
+func (type2 *UnsafePtrType) LikePtr() bool {
+	return true
+}
+
+func (type2 *UnsafePtrType) Indirect(obj interface{}) interface{} {
+	objEFace := unpackEFace(obj)
+	assertType("Type.Indirect argument 1", type2.ptrRType, objEFace.rtype)
+	return type2.UnsafeIndirect(objEFace.data)
+}
+
+func (type2 *UnsafePtrType) UnsafeIndirect(ptr unsafe.Pointer) interface{} {
+	return packEFace(type2.rtype, *(*unsafe.Pointer)(ptr))
+}
diff --git a/vendor/github.com/modern-go/reflect2/unsafe_slice.go b/vendor/github.com/modern-go/reflect2/unsafe_slice.go
new file mode 100644
index 0000000..1c6d876
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/unsafe_slice.go
@@ -0,0 +1,177 @@
+package reflect2
+
+import (
+	"reflect"
+	"unsafe"
+)
+
+// sliceHeader is a safe version of SliceHeader used within this package.
+type sliceHeader struct {
+	Data unsafe.Pointer
+	Len  int
+	Cap  int
+}
+
+type UnsafeSliceType struct {
+	unsafeType
+	elemRType  unsafe.Pointer
+	pElemRType unsafe.Pointer
+	elemSize   uintptr
+}
+
+func newUnsafeSliceType(cfg *frozenConfig, type1 reflect.Type) SliceType {
+	elemType := type1.Elem()
+	return &UnsafeSliceType{
+		unsafeType: *newUnsafeType(cfg, type1),
+		pElemRType: unpackEFace(reflect.PtrTo(elemType)).data,
+		elemRType:  unpackEFace(elemType).data,
+		elemSize:   elemType.Size(),
+	}
+}
+
+func (type2 *UnsafeSliceType) Set(obj interface{}, val interface{}) {
+	objEFace := unpackEFace(obj)
+	assertType("Type.Set argument 1", type2.ptrRType, objEFace.rtype)
+	valEFace := unpackEFace(val)
+	assertType("Type.Set argument 2", type2.ptrRType, valEFace.rtype)
+	type2.UnsafeSet(objEFace.data, valEFace.data)
+}
+
+func (type2 *UnsafeSliceType) UnsafeSet(ptr unsafe.Pointer, val unsafe.Pointer) {
+	*(*sliceHeader)(ptr) = *(*sliceHeader)(val)
+}
+
+func (type2 *UnsafeSliceType) IsNil(obj interface{}) bool {
+	if obj == nil {
+		return true
+	}
+	objEFace := unpackEFace(obj)
+	assertType("Type.IsNil argument 1", type2.ptrRType, objEFace.rtype)
+	return type2.UnsafeIsNil(objEFace.data)
+}
+
+func (type2 *UnsafeSliceType) UnsafeIsNil(ptr unsafe.Pointer) bool {
+	if ptr == nil {
+		return true
+	}
+	return (*sliceHeader)(ptr).Data == nil
+}
+
+func (type2 *UnsafeSliceType) SetNil(obj interface{}) {
+	objEFace := unpackEFace(obj)
+	assertType("SliceType.SetNil argument 1", type2.ptrRType, objEFace.rtype)
+	type2.UnsafeSetNil(objEFace.data)
+}
+
+func (type2 *UnsafeSliceType) UnsafeSetNil(ptr unsafe.Pointer) {
+	header := (*sliceHeader)(ptr)
+	header.Len = 0
+	header.Cap = 0
+	header.Data = nil
+}
+
+func (type2 *UnsafeSliceType) MakeSlice(length int, cap int) interface{} {
+	return packEFace(type2.ptrRType, type2.UnsafeMakeSlice(length, cap))
+}
+
+func (type2 *UnsafeSliceType) UnsafeMakeSlice(length int, cap int) unsafe.Pointer {
+	header := &sliceHeader{unsafe_NewArray(type2.elemRType, cap), length, cap}
+	return unsafe.Pointer(header)
+}
+
+func (type2 *UnsafeSliceType) LengthOf(obj interface{}) int {
+	objEFace := unpackEFace(obj)
+	assertType("SliceType.Len argument 1", type2.ptrRType, objEFace.rtype)
+	return type2.UnsafeLengthOf(objEFace.data)
+}
+
+func (type2 *UnsafeSliceType) UnsafeLengthOf(obj unsafe.Pointer) int {
+	header := (*sliceHeader)(obj)
+	return header.Len
+}
+
+func (type2 *UnsafeSliceType) SetIndex(obj interface{}, index int, elem interface{}) {
+	objEFace := unpackEFace(obj)
+	assertType("SliceType.SetIndex argument 1", type2.ptrRType, objEFace.rtype)
+	elemEFace := unpackEFace(elem)
+	assertType("SliceType.SetIndex argument 3", type2.pElemRType, elemEFace.rtype)
+	type2.UnsafeSetIndex(objEFace.data, index, elemEFace.data)
+}
+
+func (type2 *UnsafeSliceType) UnsafeSetIndex(obj unsafe.Pointer, index int, elem unsafe.Pointer) {
+	header := (*sliceHeader)(obj)
+	elemPtr := arrayAt(header.Data, index, type2.elemSize, "i < s.Len")
+	typedmemmove(type2.elemRType, elemPtr, elem)
+}
+
+func (type2 *UnsafeSliceType) GetIndex(obj interface{}, index int) interface{} {
+	objEFace := unpackEFace(obj)
+	assertType("SliceType.GetIndex argument 1", type2.ptrRType, objEFace.rtype)
+	elemPtr := type2.UnsafeGetIndex(objEFace.data, index)
+	return packEFace(type2.pElemRType, elemPtr)
+}
+
+func (type2 *UnsafeSliceType) UnsafeGetIndex(obj unsafe.Pointer, index int) unsafe.Pointer {
+	header := (*sliceHeader)(obj)
+	return arrayAt(header.Data, index, type2.elemSize, "i < s.Len")
+}
+
+func (type2 *UnsafeSliceType) Append(obj interface{}, elem interface{}) {
+	objEFace := unpackEFace(obj)
+	assertType("SliceType.Append argument 1", type2.ptrRType, objEFace.rtype)
+	elemEFace := unpackEFace(elem)
+	assertType("SliceType.Append argument 2", type2.pElemRType, elemEFace.rtype)
+	type2.UnsafeAppend(objEFace.data, elemEFace.data)
+}
+
+func (type2 *UnsafeSliceType) UnsafeAppend(obj unsafe.Pointer, elem unsafe.Pointer) {
+	header := (*sliceHeader)(obj)
+	oldLen := header.Len
+	type2.UnsafeGrow(obj, oldLen+1)
+	type2.UnsafeSetIndex(obj, oldLen, elem)
+}
+
+func (type2 *UnsafeSliceType) Cap(obj interface{}) int {
+	objEFace := unpackEFace(obj)
+	assertType("SliceType.Cap argument 1", type2.ptrRType, objEFace.rtype)
+	return type2.UnsafeCap(objEFace.data)
+}
+
+func (type2 *UnsafeSliceType) UnsafeCap(ptr unsafe.Pointer) int {
+	return (*sliceHeader)(ptr).Cap
+}
+
+func (type2 *UnsafeSliceType) Grow(obj interface{}, newLength int) {
+	objEFace := unpackEFace(obj)
+	assertType("SliceType.Grow argument 1", type2.ptrRType, objEFace.rtype)
+	type2.UnsafeGrow(objEFace.data, newLength)
+}
+
+func (type2 *UnsafeSliceType) UnsafeGrow(obj unsafe.Pointer, newLength int) {
+	header := (*sliceHeader)(obj)
+	if newLength <= header.Cap {
+		header.Len = newLength
+		return
+	}
+	newCap := calcNewCap(header.Cap, newLength)
+	newHeader := (*sliceHeader)(type2.UnsafeMakeSlice(header.Len, newCap))
+	typedslicecopy(type2.elemRType, *newHeader, *header)
+	header.Data = newHeader.Data
+	header.Cap = newHeader.Cap
+	header.Len = newLength
+}
+
+func calcNewCap(cap int, expectedCap int) int {
+	if cap == 0 {
+		cap = expectedCap
+	} else {
+		for cap < expectedCap {
+			if cap < 1024 {
+				cap += cap
+			} else {
+				cap += cap / 4
+			}
+		}
+	}
+	return cap
+}
diff --git a/vendor/github.com/modern-go/reflect2/unsafe_struct.go b/vendor/github.com/modern-go/reflect2/unsafe_struct.go
new file mode 100644
index 0000000..804d916
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/unsafe_struct.go
@@ -0,0 +1,59 @@
+package reflect2
+
+import (
+	"reflect"
+	"unsafe"
+)
+
+type UnsafeStructType struct {
+	unsafeType
+	likePtr bool
+}
+
+func newUnsafeStructType(cfg *frozenConfig, type1 reflect.Type) *UnsafeStructType {
+	return &UnsafeStructType{
+		unsafeType: *newUnsafeType(cfg, type1),
+		likePtr:    likePtrType(type1),
+	}
+}
+
+func (type2 *UnsafeStructType) LikePtr() bool {
+	return type2.likePtr
+}
+
+func (type2 *UnsafeStructType) Indirect(obj interface{}) interface{} {
+	objEFace := unpackEFace(obj)
+	assertType("Type.Indirect argument 1", type2.ptrRType, objEFace.rtype)
+	return type2.UnsafeIndirect(objEFace.data)
+}
+
+func (type2 *UnsafeStructType) UnsafeIndirect(ptr unsafe.Pointer) interface{} {
+	if type2.likePtr {
+		return packEFace(type2.rtype, *(*unsafe.Pointer)(ptr))
+	}
+	return packEFace(type2.rtype, ptr)
+}
+
+func (type2 *UnsafeStructType) FieldByName(name string) StructField {
+	structField, found := type2.Type.FieldByName(name)
+	if !found {
+		return nil
+	}
+	return newUnsafeStructField(type2, structField)
+}
+
+func (type2 *UnsafeStructType) Field(i int) StructField {
+	return newUnsafeStructField(type2, type2.Type.Field(i))
+}
+
+func (type2 *UnsafeStructType) FieldByIndex(index []int) StructField {
+	return newUnsafeStructField(type2, type2.Type.FieldByIndex(index))
+}
+
+func (type2 *UnsafeStructType) FieldByNameFunc(match func(string) bool) StructField {
+	structField, found := type2.Type.FieldByNameFunc(match)
+	if !found {
+		panic("field match condition not found in " + type2.Type.String())
+	}
+	return newUnsafeStructField(type2, structField)
+}
diff --git a/vendor/github.com/modern-go/reflect2/unsafe_type.go b/vendor/github.com/modern-go/reflect2/unsafe_type.go
new file mode 100644
index 0000000..1394171
--- /dev/null
+++ b/vendor/github.com/modern-go/reflect2/unsafe_type.go
@@ -0,0 +1,85 @@
+package reflect2
+
+import (
+	"reflect"
+	"unsafe"
+)
+
+type unsafeType struct {
+	safeType
+	rtype    unsafe.Pointer
+	ptrRType unsafe.Pointer
+}
+
+func newUnsafeType(cfg *frozenConfig, type1 reflect.Type) *unsafeType {
+	return &unsafeType{
+		safeType: safeType{
+			Type: type1,
+			cfg:  cfg,
+		},
+		rtype:    unpackEFace(type1).data,
+		ptrRType: unpackEFace(reflect.PtrTo(type1)).data,
+	}
+}
+
+func (type2 *unsafeType) Set(obj interface{}, val interface{}) {
+	objEFace := unpackEFace(obj)
+	assertType("Type.Set argument 1", type2.ptrRType, objEFace.rtype)
+	valEFace := unpackEFace(val)
+	assertType("Type.Set argument 2", type2.ptrRType, valEFace.rtype)
+	type2.UnsafeSet(objEFace.data, valEFace.data)
+}
+
+func (type2 *unsafeType) UnsafeSet(ptr unsafe.Pointer, val unsafe.Pointer) {
+	typedmemmove(type2.rtype, ptr, val)
+}
+
+func (type2 *unsafeType) IsNil(obj interface{}) bool {
+	objEFace := unpackEFace(obj)
+	assertType("Type.IsNil argument 1", type2.ptrRType, objEFace.rtype)
+	return type2.UnsafeIsNil(objEFace.data)
+}
+
+func (type2 *unsafeType) UnsafeIsNil(ptr unsafe.Pointer) bool {
+	return ptr == nil
+}
+
+func (type2 *unsafeType) UnsafeNew() unsafe.Pointer {
+	return unsafe_New(type2.rtype)
+}
+
+func (type2 *unsafeType) New() interface{} {
+	return packEFace(type2.ptrRType, type2.UnsafeNew())
+}
+
+func (type2 *unsafeType) PackEFace(ptr unsafe.Pointer) interface{} {
+	return packEFace(type2.ptrRType, ptr)
+}
+
+func (type2 *unsafeType) RType() uintptr {
+	return uintptr(type2.rtype)
+}
+
+func (type2 *unsafeType) Indirect(obj interface{}) interface{} {
+	objEFace := unpackEFace(obj)
+	assertType("Type.Indirect argument 1", type2.ptrRType, objEFace.rtype)
+	return type2.UnsafeIndirect(objEFace.data)
+}
+
+func (type2 *unsafeType) UnsafeIndirect(obj unsafe.Pointer) interface{} {
+	return packEFace(type2.rtype, obj)
+}
+
+func (type2 *unsafeType) LikePtr() bool {
+	return false
+}
+
+func assertType(where string, expectRType unsafe.Pointer, actualRType unsafe.Pointer) {
+	if expectRType != actualRType {
+		expectType := reflect.TypeOf(0)
+		(*iface)(unsafe.Pointer(&expectType)).data = expectRType
+		actualType := reflect.TypeOf(0)
+		(*iface)(unsafe.Pointer(&actualType)).data = actualRType
+		panic(where + ": expect " + expectType.String() + ", actual " + actualType.String())
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v2/pkg/mocks/etcd_server.go b/vendor/github.com/opencord/voltha-lib-go/v2/pkg/mocks/etcd_server.go
new file mode 100644
index 0000000..9b9dec4
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v2/pkg/mocks/etcd_server.go
@@ -0,0 +1,84 @@
+/*
+ * Copyright 2019-present Open Networking Foundation
+
+ * 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 mocks
+
+import (
+	"go.etcd.io/etcd/embed"
+	"log"
+	"os"
+	"time"
+)
+
+const (
+	serverStartUpTimeout   = 10 * time.Second // Maximum time allowed to wait for the Etcd server to be ready
+	localPersistentStorage = "voltha.embed.etcd"
+)
+
+//EtcdServer represents an embedded Etcd server.  It is used for testing only.
+type EtcdServer struct {
+	server *embed.Etcd
+}
+
+//getDefaultCfg specifies the default config
+func getDefaultCfg() *embed.Config {
+	cfg := embed.NewConfig()
+	cfg.Dir = localPersistentStorage
+	cfg.Logger = "zap"
+	cfg.LogLevel = "error"
+	return cfg
+}
+
+//StartEtcdServer creates and starts an embedded Etcd server.  A local directory to store data is created for the
+//embedded server lifetime (for the duration of a unit test.  The server runs at localhost:2379.
+func StartEtcdServer(cfg *embed.Config) *EtcdServer {
+	// If the server is already running, just return
+	if cfg == nil {
+		cfg = getDefaultCfg()
+	}
+	// Remove the local directory as
+	// a safeguard for the case where a prior test failed
+	if err := os.RemoveAll(localPersistentStorage); err != nil {
+		log.Fatalf("Failure removing local directory %s", localPersistentStorage)
+	}
+	e, err := embed.StartEtcd(cfg)
+	if err != nil {
+		log.Fatal(err)
+	}
+	select {
+	case <-e.Server.ReadyNotify():
+		log.Printf("Embedded Etcd server is ready!")
+	case <-time.After(serverStartUpTimeout):
+		e.Server.HardStop() // trigger a shutdown
+		e.Close()
+		log.Fatal("Embedded Etcd server took too long to start!")
+	case err := <-e.Err():
+		e.Server.HardStop() // trigger a shutdown
+		e.Close()
+		log.Fatalf("Embedded Etcd server errored out - %s", err)
+	}
+	return &EtcdServer{server: e}
+}
+
+//Stop closes the embedded Etcd server and removes the local data directory as well
+func (es *EtcdServer) Stop() {
+	if es != nil {
+		es.server.Server.HardStop()
+		es.server.Close()
+		if err := os.RemoveAll(localPersistentStorage); err != nil {
+			log.Fatalf("Failure removing local directory %s", es.server.Config().Dir)
+		}
+	}
+}
diff --git a/vendor/github.com/opencord/voltha-lib-go/v2/pkg/mocks/kafka_client.go b/vendor/github.com/opencord/voltha-lib-go/v2/pkg/mocks/kafka_client.go
new file mode 100644
index 0000000..79cdfbc
--- /dev/null
+++ b/vendor/github.com/opencord/voltha-lib-go/v2/pkg/mocks/kafka_client.go
@@ -0,0 +1,133 @@
+/*
+ * Copyright 2019-present Open Networking Foundation
+
+ * 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 mocks
+
+import (
+	"fmt"
+	"github.com/opencord/voltha-lib-go/v2/pkg/kafka"
+	"github.com/opencord/voltha-lib-go/v2/pkg/log"
+	ic "github.com/opencord/voltha-protos/v2/go/inter_container"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+	"sync"
+)
+
+type KafkaClient struct {
+	topicsChannelMap map[string][]chan *ic.InterContainerMessage
+	lock             sync.RWMutex
+}
+
+func NewKafkaClient() *KafkaClient {
+	return &KafkaClient{
+		topicsChannelMap: make(map[string][]chan *ic.InterContainerMessage),
+		lock:             sync.RWMutex{},
+	}
+}
+
+func (kc *KafkaClient) Start() error {
+	log.Debug("kafka-client-started")
+	return nil
+}
+
+func (kc *KafkaClient) Stop() {
+	kc.lock.Lock()
+	defer kc.lock.Unlock()
+	for topic, chnls := range kc.topicsChannelMap {
+		for _, c := range chnls {
+			close(c)
+		}
+		delete(kc.topicsChannelMap, topic)
+	}
+	log.Debug("kafka-client-stopped")
+}
+
+func (kc *KafkaClient) CreateTopic(topic *kafka.Topic, numPartition int, repFactor int) error {
+	log.Debugw("CreatingTopic", log.Fields{"topic": topic.Name, "numPartition": numPartition, "replicationFactor": repFactor})
+	kc.lock.Lock()
+	defer kc.lock.Unlock()
+	if _, ok := kc.topicsChannelMap[topic.Name]; ok {
+		return fmt.Errorf("Topic %s already exist", topic.Name)
+	}
+	ch := make(chan *ic.InterContainerMessage)
+	kc.topicsChannelMap[topic.Name] = append(kc.topicsChannelMap[topic.Name], ch)
+	return nil
+}
+
+func (kc *KafkaClient) DeleteTopic(topic *kafka.Topic) error {
+	log.Debugw("DeleteTopic", log.Fields{"topic": topic.Name})
+	kc.lock.Lock()
+	defer kc.lock.Unlock()
+	delete(kc.topicsChannelMap, topic.Name)
+	return nil
+}
+
+func (kc *KafkaClient) Subscribe(topic *kafka.Topic, kvArgs ...*kafka.KVArg) (<-chan *ic.InterContainerMessage, error) {
+	log.Debugw("Subscribe", log.Fields{"topic": topic.Name, "args": kvArgs})
+	kc.lock.Lock()
+	defer kc.lock.Unlock()
+	ch := make(chan *ic.InterContainerMessage)
+	kc.topicsChannelMap[topic.Name] = append(kc.topicsChannelMap[topic.Name], ch)
+	return ch, nil
+}
+
+func removeChannel(s []chan *ic.InterContainerMessage, i int) []chan *ic.InterContainerMessage {
+	s[i] = s[len(s)-1]
+	return s[:len(s)-1]
+}
+
+func (kc *KafkaClient) UnSubscribe(topic *kafka.Topic, ch <-chan *ic.InterContainerMessage) error {
+	log.Debugw("UnSubscribe", log.Fields{"topic": topic.Name})
+	kc.lock.Lock()
+	defer kc.lock.Unlock()
+	if chnls, ok := kc.topicsChannelMap[topic.Name]; ok {
+		idx := -1
+		for i, c := range chnls {
+			if c == ch {
+				close(c)
+				idx = i
+			}
+		}
+		if idx >= 0 {
+			kc.topicsChannelMap[topic.Name] = removeChannel(kc.topicsChannelMap[topic.Name], idx)
+		}
+	}
+	return nil
+}
+
+func (kc *KafkaClient) Send(msg interface{}, topic *kafka.Topic, keys ...string) error {
+	req, ok := msg.(*ic.InterContainerMessage)
+	if !ok {
+		return status.Error(codes.InvalidArgument, "msg-not-InterContainerMessage-type")
+	}
+	if req == nil {
+		return status.Error(codes.InvalidArgument, "msg-nil")
+	}
+	kc.lock.RLock()
+	defer kc.lock.RUnlock()
+	for _, ch := range kc.topicsChannelMap[topic.Name] {
+		log.Debugw("Publishing", log.Fields{"fromTopic": req.Header.FromTopic, "toTopic": topic.Name, "id": req.Header.Id})
+		ch <- req
+	}
+	return nil
+}
+
+func (kc *KafkaClient) SendLiveness() error {
+	return status.Error(codes.Unimplemented, "SendLiveness")
+}
+
+func (kc *KafkaClient) EnableLivenessChannel(enable bool) chan bool {
+	return nil
+}
diff --git a/vendor/github.com/prometheus/client_golang/LICENSE b/vendor/github.com/prometheus/client_golang/LICENSE
new file mode 100644
index 0000000..261eeb9
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/LICENSE
@@ -0,0 +1,201 @@
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
diff --git a/vendor/github.com/prometheus/client_golang/NOTICE b/vendor/github.com/prometheus/client_golang/NOTICE
new file mode 100644
index 0000000..dd878a3
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/NOTICE
@@ -0,0 +1,23 @@
+Prometheus instrumentation library for Go applications
+Copyright 2012-2015 The Prometheus Authors
+
+This product includes software developed at
+SoundCloud Ltd. (http://soundcloud.com/).
+
+
+The following components are included in this product:
+
+perks - a fork of https://github.com/bmizerany/perks
+https://github.com/beorn7/perks
+Copyright 2013-2015 Blake Mizerany, Björn Rabenstein
+See https://github.com/beorn7/perks/blob/master/README.md for license details.
+
+Go support for Protocol Buffers - Google's data interchange format
+http://github.com/golang/protobuf/
+Copyright 2010 The Go Authors
+See source code for license details.
+
+Support for streaming Protocol Buffer messages for the Go language (golang).
+https://github.com/matttproud/golang_protobuf_extensions
+Copyright 2013 Matt T. Proud
+Licensed under the Apache License, Version 2.0
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/.gitignore b/vendor/github.com/prometheus/client_golang/prometheus/.gitignore
new file mode 100644
index 0000000..3460f03
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/.gitignore
@@ -0,0 +1 @@
+command-line-arguments.test
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/README.md b/vendor/github.com/prometheus/client_golang/prometheus/README.md
new file mode 100644
index 0000000..44986bf
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/README.md
@@ -0,0 +1 @@
+See [![go-doc](https://godoc.org/github.com/prometheus/client_golang/prometheus?status.svg)](https://godoc.org/github.com/prometheus/client_golang/prometheus).
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/build_info.go b/vendor/github.com/prometheus/client_golang/prometheus/build_info.go
new file mode 100644
index 0000000..288f0e8
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/build_info.go
@@ -0,0 +1,29 @@
+// Copyright 2019 The Prometheus 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.
+
+// +build go1.12
+
+package prometheus
+
+import "runtime/debug"
+
+// readBuildInfo is a wrapper around debug.ReadBuildInfo for Go 1.12+.
+func readBuildInfo() (path, version, sum string) {
+	path, version, sum = "unknown", "unknown", "unknown"
+	if bi, ok := debug.ReadBuildInfo(); ok {
+		path = bi.Main.Path
+		version = bi.Main.Version
+		sum = bi.Main.Sum
+	}
+	return
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/build_info_pre_1.12.go b/vendor/github.com/prometheus/client_golang/prometheus/build_info_pre_1.12.go
new file mode 100644
index 0000000..6609e28
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/build_info_pre_1.12.go
@@ -0,0 +1,22 @@
+// Copyright 2019 The Prometheus 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.
+
+// +build !go1.12
+
+package prometheus
+
+// readBuildInfo is a wrapper around debug.ReadBuildInfo for Go versions before
+// 1.12. Remove this whole file once the minimum supported Go version is 1.12.
+func readBuildInfo() (path, version, sum string) {
+	return "unknown", "unknown", "unknown"
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/collector.go b/vendor/github.com/prometheus/client_golang/prometheus/collector.go
new file mode 100644
index 0000000..1e83965
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/collector.go
@@ -0,0 +1,120 @@
+// Copyright 2014 The Prometheus 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 prometheus
+
+// Collector is the interface implemented by anything that can be used by
+// Prometheus to collect metrics. A Collector has to be registered for
+// collection. See Registerer.Register.
+//
+// The stock metrics provided by this package (Gauge, Counter, Summary,
+// Histogram, Untyped) are also Collectors (which only ever collect one metric,
+// namely itself). An implementer of Collector may, however, collect multiple
+// metrics in a coordinated fashion and/or create metrics on the fly. Examples
+// for collectors already implemented in this library are the metric vectors
+// (i.e. collection of multiple instances of the same Metric but with different
+// label values) like GaugeVec or SummaryVec, and the ExpvarCollector.
+type Collector interface {
+	// Describe sends the super-set of all possible descriptors of metrics
+	// collected by this Collector to the provided channel and returns once
+	// the last descriptor has been sent. The sent descriptors fulfill the
+	// consistency and uniqueness requirements described in the Desc
+	// documentation.
+	//
+	// It is valid if one and the same Collector sends duplicate
+	// descriptors. Those duplicates are simply ignored. However, two
+	// different Collectors must not send duplicate descriptors.
+	//
+	// Sending no descriptor at all marks the Collector as “unchecked”,
+	// i.e. no checks will be performed at registration time, and the
+	// Collector may yield any Metric it sees fit in its Collect method.
+	//
+	// This method idempotently sends the same descriptors throughout the
+	// lifetime of the Collector. It may be called concurrently and
+	// therefore must be implemented in a concurrency safe way.
+	//
+	// If a Collector encounters an error while executing this method, it
+	// must send an invalid descriptor (created with NewInvalidDesc) to
+	// signal the error to the registry.
+	Describe(chan<- *Desc)
+	// Collect is called by the Prometheus registry when collecting
+	// metrics. The implementation sends each collected metric via the
+	// provided channel and returns once the last metric has been sent. The
+	// descriptor of each sent metric is one of those returned by Describe
+	// (unless the Collector is unchecked, see above). Returned metrics that
+	// share the same descriptor must differ in their variable label
+	// values.
+	//
+	// This method may be called concurrently and must therefore be
+	// implemented in a concurrency safe way. Blocking occurs at the expense
+	// of total performance of rendering all registered metrics. Ideally,
+	// Collector implementations support concurrent readers.
+	Collect(chan<- Metric)
+}
+
+// DescribeByCollect is a helper to implement the Describe method of a custom
+// Collector. It collects the metrics from the provided Collector and sends
+// their descriptors to the provided channel.
+//
+// If a Collector collects the same metrics throughout its lifetime, its
+// Describe method can simply be implemented as:
+//
+//   func (c customCollector) Describe(ch chan<- *Desc) {
+//   	DescribeByCollect(c, ch)
+//   }
+//
+// However, this will not work if the metrics collected change dynamically over
+// the lifetime of the Collector in a way that their combined set of descriptors
+// changes as well. The shortcut implementation will then violate the contract
+// of the Describe method. If a Collector sometimes collects no metrics at all
+// (for example vectors like CounterVec, GaugeVec, etc., which only collect
+// metrics after a metric with a fully specified label set has been accessed),
+// it might even get registered as an unchecked Collector (cf. the Register
+// method of the Registerer interface). Hence, only use this shortcut
+// implementation of Describe if you are certain to fulfill the contract.
+//
+// The Collector example demonstrates a use of DescribeByCollect.
+func DescribeByCollect(c Collector, descs chan<- *Desc) {
+	metrics := make(chan Metric)
+	go func() {
+		c.Collect(metrics)
+		close(metrics)
+	}()
+	for m := range metrics {
+		descs <- m.Desc()
+	}
+}
+
+// selfCollector implements Collector for a single Metric so that the Metric
+// collects itself. Add it as an anonymous field to a struct that implements
+// Metric, and call init with the Metric itself as an argument.
+type selfCollector struct {
+	self Metric
+}
+
+// init provides the selfCollector with a reference to the metric it is supposed
+// to collect. It is usually called within the factory function to create a
+// metric. See example.
+func (c *selfCollector) init(self Metric) {
+	c.self = self
+}
+
+// Describe implements Collector.
+func (c *selfCollector) Describe(ch chan<- *Desc) {
+	ch <- c.self.Desc()
+}
+
+// Collect implements Collector.
+func (c *selfCollector) Collect(ch chan<- Metric) {
+	ch <- c.self
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/counter.go b/vendor/github.com/prometheus/client_golang/prometheus/counter.go
new file mode 100644
index 0000000..d463e36
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/counter.go
@@ -0,0 +1,277 @@
+// Copyright 2014 The Prometheus 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 prometheus
+
+import (
+	"errors"
+	"math"
+	"sync/atomic"
+
+	dto "github.com/prometheus/client_model/go"
+)
+
+// Counter is a Metric that represents a single numerical value that only ever
+// goes up. That implies that it cannot be used to count items whose number can
+// also go down, e.g. the number of currently running goroutines. Those
+// "counters" are represented by Gauges.
+//
+// A Counter is typically used to count requests served, tasks completed, errors
+// occurred, etc.
+//
+// To create Counter instances, use NewCounter.
+type Counter interface {
+	Metric
+	Collector
+
+	// Inc increments the counter by 1. Use Add to increment it by arbitrary
+	// non-negative values.
+	Inc()
+	// Add adds the given value to the counter. It panics if the value is <
+	// 0.
+	Add(float64)
+}
+
+// CounterOpts is an alias for Opts. See there for doc comments.
+type CounterOpts Opts
+
+// NewCounter creates a new Counter based on the provided CounterOpts.
+//
+// The returned implementation tracks the counter value in two separate
+// variables, a float64 and a uint64. The latter is used to track calls of the
+// Inc method and calls of the Add method with a value that can be represented
+// as a uint64. This allows atomic increments of the counter with optimal
+// performance. (It is common to have an Inc call in very hot execution paths.)
+// Both internal tracking values are added up in the Write method. This has to
+// be taken into account when it comes to precision and overflow behavior.
+func NewCounter(opts CounterOpts) Counter {
+	desc := NewDesc(
+		BuildFQName(opts.Namespace, opts.Subsystem, opts.Name),
+		opts.Help,
+		nil,
+		opts.ConstLabels,
+	)
+	result := &counter{desc: desc, labelPairs: desc.constLabelPairs}
+	result.init(result) // Init self-collection.
+	return result
+}
+
+type counter struct {
+	// valBits contains the bits of the represented float64 value, while
+	// valInt stores values that are exact integers. Both have to go first
+	// in the struct to guarantee alignment for atomic operations.
+	// http://golang.org/pkg/sync/atomic/#pkg-note-BUG
+	valBits uint64
+	valInt  uint64
+
+	selfCollector
+	desc *Desc
+
+	labelPairs []*dto.LabelPair
+}
+
+func (c *counter) Desc() *Desc {
+	return c.desc
+}
+
+func (c *counter) Add(v float64) {
+	if v < 0 {
+		panic(errors.New("counter cannot decrease in value"))
+	}
+	ival := uint64(v)
+	if float64(ival) == v {
+		atomic.AddUint64(&c.valInt, ival)
+		return
+	}
+
+	for {
+		oldBits := atomic.LoadUint64(&c.valBits)
+		newBits := math.Float64bits(math.Float64frombits(oldBits) + v)
+		if atomic.CompareAndSwapUint64(&c.valBits, oldBits, newBits) {
+			return
+		}
+	}
+}
+
+func (c *counter) Inc() {
+	atomic.AddUint64(&c.valInt, 1)
+}
+
+func (c *counter) Write(out *dto.Metric) error {
+	fval := math.Float64frombits(atomic.LoadUint64(&c.valBits))
+	ival := atomic.LoadUint64(&c.valInt)
+	val := fval + float64(ival)
+
+	return populateMetric(CounterValue, val, c.labelPairs, out)
+}
+
+// CounterVec is a Collector that bundles a set of Counters that all share the
+// same Desc, but have different values for their variable labels. This is used
+// if you want to count the same thing partitioned by various dimensions
+// (e.g. number of HTTP requests, partitioned by response code and
+// method). Create instances with NewCounterVec.
+type CounterVec struct {
+	*metricVec
+}
+
+// NewCounterVec creates a new CounterVec based on the provided CounterOpts and
+// partitioned by the given label names.
+func NewCounterVec(opts CounterOpts, labelNames []string) *CounterVec {
+	desc := NewDesc(
+		BuildFQName(opts.Namespace, opts.Subsystem, opts.Name),
+		opts.Help,
+		labelNames,
+		opts.ConstLabels,
+	)
+	return &CounterVec{
+		metricVec: newMetricVec(desc, func(lvs ...string) Metric {
+			if len(lvs) != len(desc.variableLabels) {
+				panic(makeInconsistentCardinalityError(desc.fqName, desc.variableLabels, lvs))
+			}
+			result := &counter{desc: desc, labelPairs: makeLabelPairs(desc, lvs)}
+			result.init(result) // Init self-collection.
+			return result
+		}),
+	}
+}
+
+// GetMetricWithLabelValues returns the Counter for the given slice of label
+// values (same order as the VariableLabels in Desc). If that combination of
+// label values is accessed for the first time, a new Counter is created.
+//
+// It is possible to call this method without using the returned Counter to only
+// create the new Counter but leave it at its starting value 0. See also the
+// SummaryVec example.
+//
+// Keeping the Counter for later use is possible (and should be considered if
+// performance is critical), but keep in mind that Reset, DeleteLabelValues and
+// Delete can be used to delete the Counter from the CounterVec. In that case,
+// the Counter will still exist, but it will not be exported anymore, even if a
+// Counter with the same label values is created later.
+//
+// An error is returned if the number of label values is not the same as the
+// number of VariableLabels in Desc (minus any curried labels).
+//
+// Note that for more than one label value, this method is prone to mistakes
+// caused by an incorrect order of arguments. Consider GetMetricWith(Labels) as
+// an alternative to avoid that type of mistake. For higher label numbers, the
+// latter has a much more readable (albeit more verbose) syntax, but it comes
+// with a performance overhead (for creating and processing the Labels map).
+// See also the GaugeVec example.
+func (v *CounterVec) GetMetricWithLabelValues(lvs ...string) (Counter, error) {
+	metric, err := v.metricVec.getMetricWithLabelValues(lvs...)
+	if metric != nil {
+		return metric.(Counter), err
+	}
+	return nil, err
+}
+
+// GetMetricWith returns the Counter for the given Labels map (the label names
+// must match those of the VariableLabels in Desc). If that label map is
+// accessed for the first time, a new Counter is created. Implications of
+// creating a Counter without using it and keeping the Counter for later use are
+// the same as for GetMetricWithLabelValues.
+//
+// An error is returned if the number and names of the Labels are inconsistent
+// with those of the VariableLabels in Desc (minus any curried labels).
+//
+// This method is used for the same purpose as
+// GetMetricWithLabelValues(...string). See there for pros and cons of the two
+// methods.
+func (v *CounterVec) GetMetricWith(labels Labels) (Counter, error) {
+	metric, err := v.metricVec.getMetricWith(labels)
+	if metric != nil {
+		return metric.(Counter), err
+	}
+	return nil, err
+}
+
+// WithLabelValues works as GetMetricWithLabelValues, but panics where
+// GetMetricWithLabelValues would have returned an error. Not returning an
+// error allows shortcuts like
+//     myVec.WithLabelValues("404", "GET").Add(42)
+func (v *CounterVec) WithLabelValues(lvs ...string) Counter {
+	c, err := v.GetMetricWithLabelValues(lvs...)
+	if err != nil {
+		panic(err)
+	}
+	return c
+}
+
+// With works as GetMetricWith, but panics where GetMetricWithLabels would have
+// returned an error. Not returning an error allows shortcuts like
+//     myVec.With(prometheus.Labels{"code": "404", "method": "GET"}).Add(42)
+func (v *CounterVec) With(labels Labels) Counter {
+	c, err := v.GetMetricWith(labels)
+	if err != nil {
+		panic(err)
+	}
+	return c
+}
+
+// CurryWith returns a vector curried with the provided labels, i.e. the
+// returned vector has those labels pre-set for all labeled operations performed
+// on it. The cardinality of the curried vector is reduced accordingly. The
+// order of the remaining labels stays the same (just with the curried labels
+// taken out of the sequence – which is relevant for the
+// (GetMetric)WithLabelValues methods). It is possible to curry a curried
+// vector, but only with labels not yet used for currying before.
+//
+// The metrics contained in the CounterVec are shared between the curried and
+// uncurried vectors. They are just accessed differently. Curried and uncurried
+// vectors behave identically in terms of collection. Only one must be
+// registered with a given registry (usually the uncurried version). The Reset
+// method deletes all metrics, even if called on a curried vector.
+func (v *CounterVec) CurryWith(labels Labels) (*CounterVec, error) {
+	vec, err := v.curryWith(labels)
+	if vec != nil {
+		return &CounterVec{vec}, err
+	}
+	return nil, err
+}
+
+// MustCurryWith works as CurryWith but panics where CurryWith would have
+// returned an error.
+func (v *CounterVec) MustCurryWith(labels Labels) *CounterVec {
+	vec, err := v.CurryWith(labels)
+	if err != nil {
+		panic(err)
+	}
+	return vec
+}
+
+// CounterFunc is a Counter whose value is determined at collect time by calling a
+// provided function.
+//
+// To create CounterFunc instances, use NewCounterFunc.
+type CounterFunc interface {
+	Metric
+	Collector
+}
+
+// NewCounterFunc creates a new CounterFunc based on the provided
+// CounterOpts. The value reported is determined by calling the given function
+// from within the Write method. Take into account that metric collection may
+// happen concurrently. If that results in concurrent calls to Write, like in
+// the case where a CounterFunc is directly registered with Prometheus, the
+// provided function must be concurrency-safe. The function should also honor
+// the contract for a Counter (values only go up, not down), but compliance will
+// not be checked.
+func NewCounterFunc(opts CounterOpts, function func() float64) CounterFunc {
+	return newValueFunc(NewDesc(
+		BuildFQName(opts.Namespace, opts.Subsystem, opts.Name),
+		opts.Help,
+		nil,
+		opts.ConstLabels,
+	), CounterValue, function)
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/desc.go b/vendor/github.com/prometheus/client_golang/prometheus/desc.go
new file mode 100644
index 0000000..1d034f8
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/desc.go
@@ -0,0 +1,184 @@
+// Copyright 2016 The Prometheus 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 prometheus
+
+import (
+	"errors"
+	"fmt"
+	"sort"
+	"strings"
+
+	"github.com/golang/protobuf/proto"
+	"github.com/prometheus/common/model"
+
+	dto "github.com/prometheus/client_model/go"
+)
+
+// Desc is the descriptor used by every Prometheus Metric. It is essentially
+// the immutable meta-data of a Metric. The normal Metric implementations
+// included in this package manage their Desc under the hood. Users only have to
+// deal with Desc if they use advanced features like the ExpvarCollector or
+// custom Collectors and Metrics.
+//
+// Descriptors registered with the same registry have to fulfill certain
+// consistency and uniqueness criteria if they share the same fully-qualified
+// name: They must have the same help string and the same label names (aka label
+// dimensions) in each, constLabels and variableLabels, but they must differ in
+// the values of the constLabels.
+//
+// Descriptors that share the same fully-qualified names and the same label
+// values of their constLabels are considered equal.
+//
+// Use NewDesc to create new Desc instances.
+type Desc struct {
+	// fqName has been built from Namespace, Subsystem, and Name.
+	fqName string
+	// help provides some helpful information about this metric.
+	help string
+	// constLabelPairs contains precalculated DTO label pairs based on
+	// the constant labels.
+	constLabelPairs []*dto.LabelPair
+	// VariableLabels contains names of labels for which the metric
+	// maintains variable values.
+	variableLabels []string
+	// id is a hash of the values of the ConstLabels and fqName. This
+	// must be unique among all registered descriptors and can therefore be
+	// used as an identifier of the descriptor.
+	id uint64
+	// dimHash is a hash of the label names (preset and variable) and the
+	// Help string. Each Desc with the same fqName must have the same
+	// dimHash.
+	dimHash uint64
+	// err is an error that occurred during construction. It is reported on
+	// registration time.
+	err error
+}
+
+// NewDesc allocates and initializes a new Desc. Errors are recorded in the Desc
+// and will be reported on registration time. variableLabels and constLabels can
+// be nil if no such labels should be set. fqName must not be empty.
+//
+// variableLabels only contain the label names. Their label values are variable
+// and therefore not part of the Desc. (They are managed within the Metric.)
+//
+// For constLabels, the label values are constant. Therefore, they are fully
+// specified in the Desc. See the Collector example for a usage pattern.
+func NewDesc(fqName, help string, variableLabels []string, constLabels Labels) *Desc {
+	d := &Desc{
+		fqName:         fqName,
+		help:           help,
+		variableLabels: variableLabels,
+	}
+	if !model.IsValidMetricName(model.LabelValue(fqName)) {
+		d.err = fmt.Errorf("%q is not a valid metric name", fqName)
+		return d
+	}
+	// labelValues contains the label values of const labels (in order of
+	// their sorted label names) plus the fqName (at position 0).
+	labelValues := make([]string, 1, len(constLabels)+1)
+	labelValues[0] = fqName
+	labelNames := make([]string, 0, len(constLabels)+len(variableLabels))
+	labelNameSet := map[string]struct{}{}
+	// First add only the const label names and sort them...
+	for labelName := range constLabels {
+		if !checkLabelName(labelName) {
+			d.err = fmt.Errorf("%q is not a valid label name for metric %q", labelName, fqName)
+			return d
+		}
+		labelNames = append(labelNames, labelName)
+		labelNameSet[labelName] = struct{}{}
+	}
+	sort.Strings(labelNames)
+	// ... so that we can now add const label values in the order of their names.
+	for _, labelName := range labelNames {
+		labelValues = append(labelValues, constLabels[labelName])
+	}
+	// Validate the const label values. They can't have a wrong cardinality, so
+	// use in len(labelValues) as expectedNumberOfValues.
+	if err := validateLabelValues(labelValues, len(labelValues)); err != nil {
+		d.err = err
+		return d
+	}
+	// Now add the variable label names, but prefix them with something that
+	// cannot be in a regular label name. That prevents matching the label
+	// dimension with a different mix between preset and variable labels.
+	for _, labelName := range variableLabels {
+		if !checkLabelName(labelName) {
+			d.err = fmt.Errorf("%q is not a valid label name for metric %q", labelName, fqName)
+			return d
+		}
+		labelNames = append(labelNames, "$"+labelName)
+		labelNameSet[labelName] = struct{}{}
+	}
+	if len(labelNames) != len(labelNameSet) {
+		d.err = errors.New("duplicate label names")
+		return d
+	}
+
+	vh := hashNew()
+	for _, val := range labelValues {
+		vh = hashAdd(vh, val)
+		vh = hashAddByte(vh, separatorByte)
+	}
+	d.id = vh
+	// Sort labelNames so that order doesn't matter for the hash.
+	sort.Strings(labelNames)
+	// Now hash together (in this order) the help string and the sorted
+	// label names.
+	lh := hashNew()
+	lh = hashAdd(lh, help)
+	lh = hashAddByte(lh, separatorByte)
+	for _, labelName := range labelNames {
+		lh = hashAdd(lh, labelName)
+		lh = hashAddByte(lh, separatorByte)
+	}
+	d.dimHash = lh
+
+	d.constLabelPairs = make([]*dto.LabelPair, 0, len(constLabels))
+	for n, v := range constLabels {
+		d.constLabelPairs = append(d.constLabelPairs, &dto.LabelPair{
+			Name:  proto.String(n),
+			Value: proto.String(v),
+		})
+	}
+	sort.Sort(labelPairSorter(d.constLabelPairs))
+	return d
+}
+
+// NewInvalidDesc returns an invalid descriptor, i.e. a descriptor with the
+// provided error set. If a collector returning such a descriptor is registered,
+// registration will fail with the provided error. NewInvalidDesc can be used by
+// a Collector to signal inability to describe itself.
+func NewInvalidDesc(err error) *Desc {
+	return &Desc{
+		err: err,
+	}
+}
+
+func (d *Desc) String() string {
+	lpStrings := make([]string, 0, len(d.constLabelPairs))
+	for _, lp := range d.constLabelPairs {
+		lpStrings = append(
+			lpStrings,
+			fmt.Sprintf("%s=%q", lp.GetName(), lp.GetValue()),
+		)
+	}
+	return fmt.Sprintf(
+		"Desc{fqName: %q, help: %q, constLabels: {%s}, variableLabels: %v}",
+		d.fqName,
+		d.help,
+		strings.Join(lpStrings, ","),
+		d.variableLabels,
+	)
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/doc.go b/vendor/github.com/prometheus/client_golang/prometheus/doc.go
new file mode 100644
index 0000000..01977de
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/doc.go
@@ -0,0 +1,200 @@
+// Copyright 2014 The Prometheus 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 prometheus is the core instrumentation package. It provides metrics
+// primitives to instrument code for monitoring. It also offers a registry for
+// metrics. Sub-packages allow to expose the registered metrics via HTTP
+// (package promhttp) or push them to a Pushgateway (package push). There is
+// also a sub-package promauto, which provides metrics constructors with
+// automatic registration.
+//
+// All exported functions and methods are safe to be used concurrently unless
+// specified otherwise.
+//
+// A Basic Example
+//
+// As a starting point, a very basic usage example:
+//
+//    package main
+//
+//    import (
+//    	"log"
+//    	"net/http"
+//
+//    	"github.com/prometheus/client_golang/prometheus"
+//    	"github.com/prometheus/client_golang/prometheus/promhttp"
+//    )
+//
+//    var (
+//    	cpuTemp = prometheus.NewGauge(prometheus.GaugeOpts{
+//    		Name: "cpu_temperature_celsius",
+//    		Help: "Current temperature of the CPU.",
+//    	})
+//    	hdFailures = prometheus.NewCounterVec(
+//    		prometheus.CounterOpts{
+//    			Name: "hd_errors_total",
+//    			Help: "Number of hard-disk errors.",
+//    		},
+//    		[]string{"device"},
+//    	)
+//    )
+//
+//    func init() {
+//    	// Metrics have to be registered to be exposed:
+//    	prometheus.MustRegister(cpuTemp)
+//    	prometheus.MustRegister(hdFailures)
+//    }
+//
+//    func main() {
+//    	cpuTemp.Set(65.3)
+//    	hdFailures.With(prometheus.Labels{"device":"/dev/sda"}).Inc()
+//
+//    	// The Handler function provides a default handler to expose metrics
+//    	// via an HTTP server. "/metrics" is the usual endpoint for that.
+//    	http.Handle("/metrics", promhttp.Handler())
+//    	log.Fatal(http.ListenAndServe(":8080", nil))
+//    }
+//
+//
+// This is a complete program that exports two metrics, a Gauge and a Counter,
+// the latter with a label attached to turn it into a (one-dimensional) vector.
+//
+// Metrics
+//
+// The number of exported identifiers in this package might appear a bit
+// overwhelming. However, in addition to the basic plumbing shown in the example
+// above, you only need to understand the different metric types and their
+// vector versions for basic usage. Furthermore, if you are not concerned with
+// fine-grained control of when and how to register metrics with the registry,
+// have a look at the promauto package, which will effectively allow you to
+// ignore registration altogether in simple cases.
+//
+// Above, you have already touched the Counter and the Gauge. There are two more
+// advanced metric types: the Summary and Histogram. A more thorough description
+// of those four metric types can be found in the Prometheus docs:
+// https://prometheus.io/docs/concepts/metric_types/
+//
+// A fifth "type" of metric is Untyped. It behaves like a Gauge, but signals the
+// Prometheus server not to assume anything about its type.
+//
+// In addition to the fundamental metric types Gauge, Counter, Summary,
+// Histogram, and Untyped, a very important part of the Prometheus data model is
+// the partitioning of samples along dimensions called labels, which results in
+// metric vectors. The fundamental types are GaugeVec, CounterVec, SummaryVec,
+// HistogramVec, and UntypedVec.
+//
+// While only the fundamental metric types implement the Metric interface, both
+// the metrics and their vector versions implement the Collector interface. A
+// Collector manages the collection of a number of Metrics, but for convenience,
+// a Metric can also “collect itself”. Note that Gauge, Counter, Summary,
+// Histogram, and Untyped are interfaces themselves while GaugeVec, CounterVec,
+// SummaryVec, HistogramVec, and UntypedVec are not.
+//
+// To create instances of Metrics and their vector versions, you need a suitable
+// …Opts struct, i.e. GaugeOpts, CounterOpts, SummaryOpts, HistogramOpts, or
+// UntypedOpts.
+//
+// Custom Collectors and constant Metrics
+//
+// While you could create your own implementations of Metric, most likely you
+// will only ever implement the Collector interface on your own. At a first
+// glance, a custom Collector seems handy to bundle Metrics for common
+// registration (with the prime example of the different metric vectors above,
+// which bundle all the metrics of the same name but with different labels).
+//
+// There is a more involved use case, too: If you already have metrics
+// available, created outside of the Prometheus context, you don't need the
+// interface of the various Metric types. You essentially want to mirror the
+// existing numbers into Prometheus Metrics during collection. An own
+// implementation of the Collector interface is perfect for that. You can create
+// Metric instances “on the fly” using NewConstMetric, NewConstHistogram, and
+// NewConstSummary (and their respective Must… versions). That will happen in
+// the Collect method. The Describe method has to return separate Desc
+// instances, representative of the “throw-away” metrics to be created later.
+// NewDesc comes in handy to create those Desc instances. Alternatively, you
+// could return no Desc at all, which will mark the Collector “unchecked”.  No
+// checks are performed at registration time, but metric consistency will still
+// be ensured at scrape time, i.e. any inconsistencies will lead to scrape
+// errors. Thus, with unchecked Collectors, the responsibility to not collect
+// metrics that lead to inconsistencies in the total scrape result lies with the
+// implementer of the Collector. While this is not a desirable state, it is
+// sometimes necessary. The typical use case is a situation where the exact
+// metrics to be returned by a Collector cannot be predicted at registration
+// time, but the implementer has sufficient knowledge of the whole system to
+// guarantee metric consistency.
+//
+// The Collector example illustrates the use case. You can also look at the
+// source code of the processCollector (mirroring process metrics), the
+// goCollector (mirroring Go metrics), or the expvarCollector (mirroring expvar
+// metrics) as examples that are used in this package itself.
+//
+// If you just need to call a function to get a single float value to collect as
+// a metric, GaugeFunc, CounterFunc, or UntypedFunc might be interesting
+// shortcuts.
+//
+// Advanced Uses of the Registry
+//
+// While MustRegister is the by far most common way of registering a Collector,
+// sometimes you might want to handle the errors the registration might cause.
+// As suggested by the name, MustRegister panics if an error occurs. With the
+// Register function, the error is returned and can be handled.
+//
+// An error is returned if the registered Collector is incompatible or
+// inconsistent with already registered metrics. The registry aims for
+// consistency of the collected metrics according to the Prometheus data model.
+// Inconsistencies are ideally detected at registration time, not at collect
+// time. The former will usually be detected at start-up time of a program,
+// while the latter will only happen at scrape time, possibly not even on the
+// first scrape if the inconsistency only becomes relevant later. That is the
+// main reason why a Collector and a Metric have to describe themselves to the
+// registry.
+//
+// So far, everything we did operated on the so-called default registry, as it
+// can be found in the global DefaultRegisterer variable. With NewRegistry, you
+// can create a custom registry, or you can even implement the Registerer or
+// Gatherer interfaces yourself. The methods Register and Unregister work in the
+// same way on a custom registry as the global functions Register and Unregister
+// on the default registry.
+//
+// There are a number of uses for custom registries: You can use registries with
+// special properties, see NewPedanticRegistry. You can avoid global state, as
+// it is imposed by the DefaultRegisterer. You can use multiple registries at
+// the same time to expose different metrics in different ways.  You can use
+// separate registries for testing purposes.
+//
+// Also note that the DefaultRegisterer comes registered with a Collector for Go
+// runtime metrics (via NewGoCollector) and a Collector for process metrics (via
+// NewProcessCollector). With a custom registry, you are in control and decide
+// yourself about the Collectors to register.
+//
+// HTTP Exposition
+//
+// The Registry implements the Gatherer interface. The caller of the Gather
+// method can then expose the gathered metrics in some way. Usually, the metrics
+// are served via HTTP on the /metrics endpoint. That's happening in the example
+// above. The tools to expose metrics via HTTP are in the promhttp sub-package.
+//
+// Pushing to the Pushgateway
+//
+// Function for pushing to the Pushgateway can be found in the push sub-package.
+//
+// Graphite Bridge
+//
+// Functions and examples to push metrics from a Gatherer to Graphite can be
+// found in the graphite sub-package.
+//
+// Other Means of Exposition
+//
+// More ways of exposing metrics can easily be added by following the approaches
+// of the existing implementations.
+package prometheus
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/expvar_collector.go b/vendor/github.com/prometheus/client_golang/prometheus/expvar_collector.go
new file mode 100644
index 0000000..18a99d5
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/expvar_collector.go
@@ -0,0 +1,119 @@
+// Copyright 2014 The Prometheus 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 prometheus
+
+import (
+	"encoding/json"
+	"expvar"
+)
+
+type expvarCollector struct {
+	exports map[string]*Desc
+}
+
+// NewExpvarCollector returns a newly allocated expvar Collector that still has
+// to be registered with a Prometheus registry.
+//
+// An expvar Collector collects metrics from the expvar interface. It provides a
+// quick way to expose numeric values that are already exported via expvar as
+// Prometheus metrics. Note that the data models of expvar and Prometheus are
+// fundamentally different, and that the expvar Collector is inherently slower
+// than native Prometheus metrics. Thus, the expvar Collector is probably great
+// for experiments and prototying, but you should seriously consider a more
+// direct implementation of Prometheus metrics for monitoring production
+// systems.
+//
+// The exports map has the following meaning:
+//
+// The keys in the map correspond to expvar keys, i.e. for every expvar key you
+// want to export as Prometheus metric, you need an entry in the exports
+// map. The descriptor mapped to each key describes how to export the expvar
+// value. It defines the name and the help string of the Prometheus metric
+// proxying the expvar value. The type will always be Untyped.
+//
+// For descriptors without variable labels, the expvar value must be a number or
+// a bool. The number is then directly exported as the Prometheus sample
+// value. (For a bool, 'false' translates to 0 and 'true' to 1). Expvar values
+// that are not numbers or bools are silently ignored.
+//
+// If the descriptor has one variable label, the expvar value must be an expvar
+// map. The keys in the expvar map become the various values of the one
+// Prometheus label. The values in the expvar map must be numbers or bools again
+// as above.
+//
+// For descriptors with more than one variable label, the expvar must be a
+// nested expvar map, i.e. where the values of the topmost map are maps again
+// etc. until a depth is reached that corresponds to the number of labels. The
+// leaves of that structure must be numbers or bools as above to serve as the
+// sample values.
+//
+// Anything that does not fit into the scheme above is silently ignored.
+func NewExpvarCollector(exports map[string]*Desc) Collector {
+	return &expvarCollector{
+		exports: exports,
+	}
+}
+
+// Describe implements Collector.
+func (e *expvarCollector) Describe(ch chan<- *Desc) {
+	for _, desc := range e.exports {
+		ch <- desc
+	}
+}
+
+// Collect implements Collector.
+func (e *expvarCollector) Collect(ch chan<- Metric) {
+	for name, desc := range e.exports {
+		var m Metric
+		expVar := expvar.Get(name)
+		if expVar == nil {
+			continue
+		}
+		var v interface{}
+		labels := make([]string, len(desc.variableLabels))
+		if err := json.Unmarshal([]byte(expVar.String()), &v); err != nil {
+			ch <- NewInvalidMetric(desc, err)
+			continue
+		}
+		var processValue func(v interface{}, i int)
+		processValue = func(v interface{}, i int) {
+			if i >= len(labels) {
+				copiedLabels := append(make([]string, 0, len(labels)), labels...)
+				switch v := v.(type) {
+				case float64:
+					m = MustNewConstMetric(desc, UntypedValue, v, copiedLabels...)
+				case bool:
+					if v {
+						m = MustNewConstMetric(desc, UntypedValue, 1, copiedLabels...)
+					} else {
+						m = MustNewConstMetric(desc, UntypedValue, 0, copiedLabels...)
+					}
+				default:
+					return
+				}
+				ch <- m
+				return
+			}
+			vm, ok := v.(map[string]interface{})
+			if !ok {
+				return
+			}
+			for lv, val := range vm {
+				labels[i] = lv
+				processValue(val, i+1)
+			}
+		}
+		processValue(v, 0)
+	}
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/fnv.go b/vendor/github.com/prometheus/client_golang/prometheus/fnv.go
new file mode 100644
index 0000000..3d383a7
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/fnv.go
@@ -0,0 +1,42 @@
+// Copyright 2018 The Prometheus 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 prometheus
+
+// Inline and byte-free variant of hash/fnv's fnv64a.
+
+const (
+	offset64 = 14695981039346656037
+	prime64  = 1099511628211
+)
+
+// hashNew initializies a new fnv64a hash value.
+func hashNew() uint64 {
+	return offset64
+}
+
+// hashAdd adds a string to a fnv64a hash value, returning the updated hash.
+func hashAdd(h uint64, s string) uint64 {
+	for i := 0; i < len(s); i++ {
+		h ^= uint64(s[i])
+		h *= prime64
+	}
+	return h
+}
+
+// hashAddByte adds a byte to a fnv64a hash value, returning the updated hash.
+func hashAddByte(h uint64, b byte) uint64 {
+	h ^= uint64(b)
+	h *= prime64
+	return h
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/gauge.go b/vendor/github.com/prometheus/client_golang/prometheus/gauge.go
new file mode 100644
index 0000000..71d406b
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/gauge.go
@@ -0,0 +1,286 @@
+// Copyright 2014 The Prometheus 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 prometheus
+
+import (
+	"math"
+	"sync/atomic"
+	"time"
+
+	dto "github.com/prometheus/client_model/go"
+)
+
+// Gauge is a Metric that represents a single numerical value that can
+// arbitrarily go up and down.
+//
+// A Gauge is typically used for measured values like temperatures or current
+// memory usage, but also "counts" that can go up and down, like the number of
+// running goroutines.
+//
+// To create Gauge instances, use NewGauge.
+type Gauge interface {
+	Metric
+	Collector
+
+	// Set sets the Gauge to an arbitrary value.
+	Set(float64)
+	// Inc increments the Gauge by 1. Use Add to increment it by arbitrary
+	// values.
+	Inc()
+	// Dec decrements the Gauge by 1. Use Sub to decrement it by arbitrary
+	// values.
+	Dec()
+	// Add adds the given value to the Gauge. (The value can be negative,
+	// resulting in a decrease of the Gauge.)
+	Add(float64)
+	// Sub subtracts the given value from the Gauge. (The value can be
+	// negative, resulting in an increase of the Gauge.)
+	Sub(float64)
+
+	// SetToCurrentTime sets the Gauge to the current Unix time in seconds.
+	SetToCurrentTime()
+}
+
+// GaugeOpts is an alias for Opts. See there for doc comments.
+type GaugeOpts Opts
+
+// NewGauge creates a new Gauge based on the provided GaugeOpts.
+//
+// The returned implementation is optimized for a fast Set method. If you have a
+// choice for managing the value of a Gauge via Set vs. Inc/Dec/Add/Sub, pick
+// the former. For example, the Inc method of the returned Gauge is slower than
+// the Inc method of a Counter returned by NewCounter. This matches the typical
+// scenarios for Gauges and Counters, where the former tends to be Set-heavy and
+// the latter Inc-heavy.
+func NewGauge(opts GaugeOpts) Gauge {
+	desc := NewDesc(
+		BuildFQName(opts.Namespace, opts.Subsystem, opts.Name),
+		opts.Help,
+		nil,
+		opts.ConstLabels,
+	)
+	result := &gauge{desc: desc, labelPairs: desc.constLabelPairs}
+	result.init(result) // Init self-collection.
+	return result
+}
+
+type gauge struct {
+	// valBits contains the bits of the represented float64 value. It has
+	// to go first in the struct to guarantee alignment for atomic
+	// operations.  http://golang.org/pkg/sync/atomic/#pkg-note-BUG
+	valBits uint64
+
+	selfCollector
+
+	desc       *Desc
+	labelPairs []*dto.LabelPair
+}
+
+func (g *gauge) Desc() *Desc {
+	return g.desc
+}
+
+func (g *gauge) Set(val float64) {
+	atomic.StoreUint64(&g.valBits, math.Float64bits(val))
+}
+
+func (g *gauge) SetToCurrentTime() {
+	g.Set(float64(time.Now().UnixNano()) / 1e9)
+}
+
+func (g *gauge) Inc() {
+	g.Add(1)
+}
+
+func (g *gauge) Dec() {
+	g.Add(-1)
+}
+
+func (g *gauge) Add(val float64) {
+	for {
+		oldBits := atomic.LoadUint64(&g.valBits)
+		newBits := math.Float64bits(math.Float64frombits(oldBits) + val)
+		if atomic.CompareAndSwapUint64(&g.valBits, oldBits, newBits) {
+			return
+		}
+	}
+}
+
+func (g *gauge) Sub(val float64) {
+	g.Add(val * -1)
+}
+
+func (g *gauge) Write(out *dto.Metric) error {
+	val := math.Float64frombits(atomic.LoadUint64(&g.valBits))
+	return populateMetric(GaugeValue, val, g.labelPairs, out)
+}
+
+// GaugeVec is a Collector that bundles a set of Gauges that all share the same
+// Desc, but have different values for their variable labels. This is used if
+// you want to count the same thing partitioned by various dimensions
+// (e.g. number of operations queued, partitioned by user and operation
+// type). Create instances with NewGaugeVec.
+type GaugeVec struct {
+	*metricVec
+}
+
+// NewGaugeVec creates a new GaugeVec based on the provided GaugeOpts and
+// partitioned by the given label names.
+func NewGaugeVec(opts GaugeOpts, labelNames []string) *GaugeVec {
+	desc := NewDesc(
+		BuildFQName(opts.Namespace, opts.Subsystem, opts.Name),
+		opts.Help,
+		labelNames,
+		opts.ConstLabels,
+	)
+	return &GaugeVec{
+		metricVec: newMetricVec(desc, func(lvs ...string) Metric {
+			if len(lvs) != len(desc.variableLabels) {
+				panic(makeInconsistentCardinalityError(desc.fqName, desc.variableLabels, lvs))
+			}
+			result := &gauge{desc: desc, labelPairs: makeLabelPairs(desc, lvs)}
+			result.init(result) // Init self-collection.
+			return result
+		}),
+	}
+}
+
+// GetMetricWithLabelValues returns the Gauge for the given slice of label
+// values (same order as the VariableLabels in Desc). If that combination of
+// label values is accessed for the first time, a new Gauge is created.
+//
+// It is possible to call this method without using the returned Gauge to only
+// create the new Gauge but leave it at its starting value 0. See also the
+// SummaryVec example.
+//
+// Keeping the Gauge for later use is possible (and should be considered if
+// performance is critical), but keep in mind that Reset, DeleteLabelValues and
+// Delete can be used to delete the Gauge from the GaugeVec. In that case, the
+// Gauge will still exist, but it will not be exported anymore, even if a
+// Gauge with the same label values is created later. See also the CounterVec
+// example.
+//
+// An error is returned if the number of label values is not the same as the
+// number of VariableLabels in Desc (minus any curried labels).
+//
+// Note that for more than one label value, this method is prone to mistakes
+// caused by an incorrect order of arguments. Consider GetMetricWith(Labels) as
+// an alternative to avoid that type of mistake. For higher label numbers, the
+// latter has a much more readable (albeit more verbose) syntax, but it comes
+// with a performance overhead (for creating and processing the Labels map).
+func (v *GaugeVec) GetMetricWithLabelValues(lvs ...string) (Gauge, error) {
+	metric, err := v.metricVec.getMetricWithLabelValues(lvs...)
+	if metric != nil {
+		return metric.(Gauge), err
+	}
+	return nil, err
+}
+
+// GetMetricWith returns the Gauge for the given Labels map (the label names
+// must match those of the VariableLabels in Desc). If that label map is
+// accessed for the first time, a new Gauge is created. Implications of
+// creating a Gauge without using it and keeping the Gauge for later use are
+// the same as for GetMetricWithLabelValues.
+//
+// An error is returned if the number and names of the Labels are inconsistent
+// with those of the VariableLabels in Desc (minus any curried labels).
+//
+// This method is used for the same purpose as
+// GetMetricWithLabelValues(...string). See there for pros and cons of the two
+// methods.
+func (v *GaugeVec) GetMetricWith(labels Labels) (Gauge, error) {
+	metric, err := v.metricVec.getMetricWith(labels)
+	if metric != nil {
+		return metric.(Gauge), err
+	}
+	return nil, err
+}
+
+// WithLabelValues works as GetMetricWithLabelValues, but panics where
+// GetMetricWithLabelValues would have returned an error. Not returning an
+// error allows shortcuts like
+//     myVec.WithLabelValues("404", "GET").Add(42)
+func (v *GaugeVec) WithLabelValues(lvs ...string) Gauge {
+	g, err := v.GetMetricWithLabelValues(lvs...)
+	if err != nil {
+		panic(err)
+	}
+	return g
+}
+
+// With works as GetMetricWith, but panics where GetMetricWithLabels would have
+// returned an error. Not returning an error allows shortcuts like
+//     myVec.With(prometheus.Labels{"code": "404", "method": "GET"}).Add(42)
+func (v *GaugeVec) With(labels Labels) Gauge {
+	g, err := v.GetMetricWith(labels)
+	if err != nil {
+		panic(err)
+	}
+	return g
+}
+
+// CurryWith returns a vector curried with the provided labels, i.e. the
+// returned vector has those labels pre-set for all labeled operations performed
+// on it. The cardinality of the curried vector is reduced accordingly. The
+// order of the remaining labels stays the same (just with the curried labels
+// taken out of the sequence – which is relevant for the
+// (GetMetric)WithLabelValues methods). It is possible to curry a curried
+// vector, but only with labels not yet used for currying before.
+//
+// The metrics contained in the GaugeVec are shared between the curried and
+// uncurried vectors. They are just accessed differently. Curried and uncurried
+// vectors behave identically in terms of collection. Only one must be
+// registered with a given registry (usually the uncurried version). The Reset
+// method deletes all metrics, even if called on a curried vector.
+func (v *GaugeVec) CurryWith(labels Labels) (*GaugeVec, error) {
+	vec, err := v.curryWith(labels)
+	if vec != nil {
+		return &GaugeVec{vec}, err
+	}
+	return nil, err
+}
+
+// MustCurryWith works as CurryWith but panics where CurryWith would have
+// returned an error.
+func (v *GaugeVec) MustCurryWith(labels Labels) *GaugeVec {
+	vec, err := v.CurryWith(labels)
+	if err != nil {
+		panic(err)
+	}
+	return vec
+}
+
+// GaugeFunc is a Gauge whose value is determined at collect time by calling a
+// provided function.
+//
+// To create GaugeFunc instances, use NewGaugeFunc.
+type GaugeFunc interface {
+	Metric
+	Collector
+}
+
+// NewGaugeFunc creates a new GaugeFunc based on the provided GaugeOpts. The
+// value reported is determined by calling the given function from within the
+// Write method. Take into account that metric collection may happen
+// concurrently. If that results in concurrent calls to Write, like in the case
+// where a GaugeFunc is directly registered with Prometheus, the provided
+// function must be concurrency-safe.
+func NewGaugeFunc(opts GaugeOpts, function func() float64) GaugeFunc {
+	return newValueFunc(NewDesc(
+		BuildFQName(opts.Namespace, opts.Subsystem, opts.Name),
+		opts.Help,
+		nil,
+		opts.ConstLabels,
+	), GaugeValue, function)
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/go_collector.go b/vendor/github.com/prometheus/client_golang/prometheus/go_collector.go
new file mode 100644
index 0000000..dc9247f
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/go_collector.go
@@ -0,0 +1,396 @@
+// Copyright 2018 The Prometheus 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 prometheus
+
+import (
+	"runtime"
+	"runtime/debug"
+	"sync"
+	"time"
+)
+
+type goCollector struct {
+	goroutinesDesc *Desc
+	threadsDesc    *Desc
+	gcDesc         *Desc
+	goInfoDesc     *Desc
+
+	// ms... are memstats related.
+	msLast          *runtime.MemStats // Previously collected memstats.
+	msLastTimestamp time.Time
+	msMtx           sync.Mutex // Protects msLast and msLastTimestamp.
+	msMetrics       memStatsMetrics
+	msRead          func(*runtime.MemStats) // For mocking in tests.
+	msMaxWait       time.Duration           // Wait time for fresh memstats.
+	msMaxAge        time.Duration           // Maximum allowed age of old memstats.
+}
+
+// NewGoCollector returns a collector that exports metrics about the current Go
+// process. This includes memory stats. To collect those, runtime.ReadMemStats
+// is called. This requires to “stop the world”, which usually only happens for
+// garbage collection (GC). Take the following implications into account when
+// deciding whether to use the Go collector:
+//
+// 1. The performance impact of stopping the world is the more relevant the more
+// frequently metrics are collected. However, with Go1.9 or later the
+// stop-the-world time per metrics collection is very short (~25µs) so that the
+// performance impact will only matter in rare cases. However, with older Go
+// versions, the stop-the-world duration depends on the heap size and can be
+// quite significant (~1.7 ms/GiB as per
+// https://go-review.googlesource.com/c/go/+/34937).
+//
+// 2. During an ongoing GC, nothing else can stop the world. Therefore, if the
+// metrics collection happens to coincide with GC, it will only complete after
+// GC has finished. Usually, GC is fast enough to not cause problems. However,
+// with a very large heap, GC might take multiple seconds, which is enough to
+// cause scrape timeouts in common setups. To avoid this problem, the Go
+// collector will use the memstats from a previous collection if
+// runtime.ReadMemStats takes more than 1s. However, if there are no previously
+// collected memstats, or their collection is more than 5m ago, the collection
+// will block until runtime.ReadMemStats succeeds. (The problem might be solved
+// in Go1.13, see https://github.com/golang/go/issues/19812 for the related Go
+// issue.)
+func NewGoCollector() Collector {
+	return &goCollector{
+		goroutinesDesc: NewDesc(
+			"go_goroutines",
+			"Number of goroutines that currently exist.",
+			nil, nil),
+		threadsDesc: NewDesc(
+			"go_threads",
+			"Number of OS threads created.",
+			nil, nil),
+		gcDesc: NewDesc(
+			"go_gc_duration_seconds",
+			"A summary of the GC invocation durations.",
+			nil, nil),
+		goInfoDesc: NewDesc(
+			"go_info",
+			"Information about the Go environment.",
+			nil, Labels{"version": runtime.Version()}),
+		msLast:    &runtime.MemStats{},
+		msRead:    runtime.ReadMemStats,
+		msMaxWait: time.Second,
+		msMaxAge:  5 * time.Minute,
+		msMetrics: memStatsMetrics{
+			{
+				desc: NewDesc(
+					memstatNamespace("alloc_bytes"),
+					"Number of bytes allocated and still in use.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.Alloc) },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("alloc_bytes_total"),
+					"Total number of bytes allocated, even if freed.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.TotalAlloc) },
+				valType: CounterValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("sys_bytes"),
+					"Number of bytes obtained from system.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.Sys) },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("lookups_total"),
+					"Total number of pointer lookups.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.Lookups) },
+				valType: CounterValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("mallocs_total"),
+					"Total number of mallocs.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.Mallocs) },
+				valType: CounterValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("frees_total"),
+					"Total number of frees.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.Frees) },
+				valType: CounterValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("heap_alloc_bytes"),
+					"Number of heap bytes allocated and still in use.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.HeapAlloc) },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("heap_sys_bytes"),
+					"Number of heap bytes obtained from system.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.HeapSys) },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("heap_idle_bytes"),
+					"Number of heap bytes waiting to be used.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.HeapIdle) },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("heap_inuse_bytes"),
+					"Number of heap bytes that are in use.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.HeapInuse) },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("heap_released_bytes"),
+					"Number of heap bytes released to OS.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.HeapReleased) },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("heap_objects"),
+					"Number of allocated objects.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.HeapObjects) },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("stack_inuse_bytes"),
+					"Number of bytes in use by the stack allocator.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.StackInuse) },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("stack_sys_bytes"),
+					"Number of bytes obtained from system for stack allocator.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.StackSys) },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("mspan_inuse_bytes"),
+					"Number of bytes in use by mspan structures.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.MSpanInuse) },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("mspan_sys_bytes"),
+					"Number of bytes used for mspan structures obtained from system.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.MSpanSys) },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("mcache_inuse_bytes"),
+					"Number of bytes in use by mcache structures.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.MCacheInuse) },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("mcache_sys_bytes"),
+					"Number of bytes used for mcache structures obtained from system.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.MCacheSys) },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("buck_hash_sys_bytes"),
+					"Number of bytes used by the profiling bucket hash table.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.BuckHashSys) },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("gc_sys_bytes"),
+					"Number of bytes used for garbage collection system metadata.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.GCSys) },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("other_sys_bytes"),
+					"Number of bytes used for other system allocations.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.OtherSys) },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("next_gc_bytes"),
+					"Number of heap bytes when next garbage collection will take place.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.NextGC) },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("last_gc_time_seconds"),
+					"Number of seconds since 1970 of last garbage collection.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return float64(ms.LastGC) / 1e9 },
+				valType: GaugeValue,
+			}, {
+				desc: NewDesc(
+					memstatNamespace("gc_cpu_fraction"),
+					"The fraction of this program's available CPU time used by the GC since the program started.",
+					nil, nil,
+				),
+				eval:    func(ms *runtime.MemStats) float64 { return ms.GCCPUFraction },
+				valType: GaugeValue,
+			},
+		},
+	}
+}
+
+func memstatNamespace(s string) string {
+	return "go_memstats_" + s
+}
+
+// Describe returns all descriptions of the collector.
+func (c *goCollector) Describe(ch chan<- *Desc) {
+	ch <- c.goroutinesDesc
+	ch <- c.threadsDesc
+	ch <- c.gcDesc
+	ch <- c.goInfoDesc
+	for _, i := range c.msMetrics {
+		ch <- i.desc
+	}
+}
+
+// Collect returns the current state of all metrics of the collector.
+func (c *goCollector) Collect(ch chan<- Metric) {
+	var (
+		ms   = &runtime.MemStats{}
+		done = make(chan struct{})
+	)
+	// Start reading memstats first as it might take a while.
+	go func() {
+		c.msRead(ms)
+		c.msMtx.Lock()
+		c.msLast = ms
+		c.msLastTimestamp = time.Now()
+		c.msMtx.Unlock()
+		close(done)
+	}()
+
+	ch <- MustNewConstMetric(c.goroutinesDesc, GaugeValue, float64(runtime.NumGoroutine()))
+	n, _ := runtime.ThreadCreateProfile(nil)
+	ch <- MustNewConstMetric(c.threadsDesc, GaugeValue, float64(n))
+
+	var stats debug.GCStats
+	stats.PauseQuantiles = make([]time.Duration, 5)
+	debug.ReadGCStats(&stats)
+
+	quantiles := make(map[float64]float64)
+	for idx, pq := range stats.PauseQuantiles[1:] {
+		quantiles[float64(idx+1)/float64(len(stats.PauseQuantiles)-1)] = pq.Seconds()
+	}
+	quantiles[0.0] = stats.PauseQuantiles[0].Seconds()
+	ch <- MustNewConstSummary(c.gcDesc, uint64(stats.NumGC), stats.PauseTotal.Seconds(), quantiles)
+
+	ch <- MustNewConstMetric(c.goInfoDesc, GaugeValue, 1)
+
+	timer := time.NewTimer(c.msMaxWait)
+	select {
+	case <-done: // Our own ReadMemStats succeeded in time. Use it.
+		timer.Stop() // Important for high collection frequencies to not pile up timers.
+		c.msCollect(ch, ms)
+		return
+	case <-timer.C: // Time out, use last memstats if possible. Continue below.
+	}
+	c.msMtx.Lock()
+	if time.Since(c.msLastTimestamp) < c.msMaxAge {
+		// Last memstats are recent enough. Collect from them under the lock.
+		c.msCollect(ch, c.msLast)
+		c.msMtx.Unlock()
+		return
+	}
+	// If we are here, the last memstats are too old or don't exist. We have
+	// to wait until our own ReadMemStats finally completes. For that to
+	// happen, we have to release the lock.
+	c.msMtx.Unlock()
+	<-done
+	c.msCollect(ch, ms)
+}
+
+func (c *goCollector) msCollect(ch chan<- Metric, ms *runtime.MemStats) {
+	for _, i := range c.msMetrics {
+		ch <- MustNewConstMetric(i.desc, i.valType, i.eval(ms))
+	}
+}
+
+// memStatsMetrics provide description, value, and value type for memstat metrics.
+type memStatsMetrics []struct {
+	desc    *Desc
+	eval    func(*runtime.MemStats) float64
+	valType ValueType
+}
+
+// NewBuildInfoCollector returns a collector collecting a single metric
+// "go_build_info" with the constant value 1 and three labels "path", "version",
+// and "checksum". Their label values contain the main module path, version, and
+// checksum, respectively. The labels will only have meaningful values if the
+// binary is built with Go module support and from source code retrieved from
+// the source repository (rather than the local file system). This is usually
+// accomplished by building from outside of GOPATH, specifying the full address
+// of the main package, e.g. "GO111MODULE=on go run
+// github.com/prometheus/client_golang/examples/random". If built without Go
+// module support, all label values will be "unknown". If built with Go module
+// support but using the source code from the local file system, the "path" will
+// be set appropriately, but "checksum" will be empty and "version" will be
+// "(devel)".
+//
+// This collector uses only the build information for the main module. See
+// https://github.com/povilasv/prommod for an example of a collector for the
+// module dependencies.
+func NewBuildInfoCollector() Collector {
+	path, version, sum := readBuildInfo()
+	c := &selfCollector{MustNewConstMetric(
+		NewDesc(
+			"go_build_info",
+			"Build information about the main Go module.",
+			nil, Labels{"path": path, "version": version, "checksum": sum},
+		),
+		GaugeValue, 1)}
+	c.init(c.self)
+	return c
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/histogram.go b/vendor/github.com/prometheus/client_golang/prometheus/histogram.go
new file mode 100644
index 0000000..d7ea67b
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/histogram.go
@@ -0,0 +1,586 @@
+// Copyright 2015 The Prometheus 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 prometheus
+
+import (
+	"fmt"
+	"math"
+	"runtime"
+	"sort"
+	"sync"
+	"sync/atomic"
+
+	"github.com/golang/protobuf/proto"
+
+	dto "github.com/prometheus/client_model/go"
+)
+
+// A Histogram counts individual observations from an event or sample stream in
+// configurable buckets. Similar to a summary, it also provides a sum of
+// observations and an observation count.
+//
+// On the Prometheus server, quantiles can be calculated from a Histogram using
+// the histogram_quantile function in the query language.
+//
+// Note that Histograms, in contrast to Summaries, can be aggregated with the
+// Prometheus query language (see the documentation for detailed
+// procedures). However, Histograms require the user to pre-define suitable
+// buckets, and they are in general less accurate. The Observe method of a
+// Histogram has a very low performance overhead in comparison with the Observe
+// method of a Summary.
+//
+// To create Histogram instances, use NewHistogram.
+type Histogram interface {
+	Metric
+	Collector
+
+	// Observe adds a single observation to the histogram.
+	Observe(float64)
+}
+
+// bucketLabel is used for the label that defines the upper bound of a
+// bucket of a histogram ("le" -> "less or equal").
+const bucketLabel = "le"
+
+// DefBuckets are the default Histogram buckets. The default buckets are
+// tailored to broadly measure the response time (in seconds) of a network
+// service. Most likely, however, you will be required to define buckets
+// customized to your use case.
+var (
+	DefBuckets = []float64{.005, .01, .025, .05, .1, .25, .5, 1, 2.5, 5, 10}
+
+	errBucketLabelNotAllowed = fmt.Errorf(
+		"%q is not allowed as label name in histograms", bucketLabel,
+	)
+)
+
+// LinearBuckets creates 'count' buckets, each 'width' wide, where the lowest
+// bucket has an upper bound of 'start'. The final +Inf bucket is not counted
+// and not included in the returned slice. The returned slice is meant to be
+// used for the Buckets field of HistogramOpts.
+//
+// The function panics if 'count' is zero or negative.
+func LinearBuckets(start, width float64, count int) []float64 {
+	if count < 1 {
+		panic("LinearBuckets needs a positive count")
+	}
+	buckets := make([]float64, count)
+	for i := range buckets {
+		buckets[i] = start
+		start += width
+	}
+	return buckets
+}
+
+// ExponentialBuckets creates 'count' buckets, where the lowest bucket has an
+// upper bound of 'start' and each following bucket's upper bound is 'factor'
+// times the previous bucket's upper bound. The final +Inf bucket is not counted
+// and not included in the returned slice. The returned slice is meant to be
+// used for the Buckets field of HistogramOpts.
+//
+// The function panics if 'count' is 0 or negative, if 'start' is 0 or negative,
+// or if 'factor' is less than or equal 1.
+func ExponentialBuckets(start, factor float64, count int) []float64 {
+	if count < 1 {
+		panic("ExponentialBuckets needs a positive count")
+	}
+	if start <= 0 {
+		panic("ExponentialBuckets needs a positive start value")
+	}
+	if factor <= 1 {
+		panic("ExponentialBuckets needs a factor greater than 1")
+	}
+	buckets := make([]float64, count)
+	for i := range buckets {
+		buckets[i] = start
+		start *= factor
+	}
+	return buckets
+}
+
+// HistogramOpts bundles the options for creating a Histogram metric. It is
+// mandatory to set Name to a non-empty string. All other fields are optional
+// and can safely be left at their zero value, although it is strongly
+// encouraged to set a Help string.
+type HistogramOpts struct {
+	// Namespace, Subsystem, and Name are components of the fully-qualified
+	// name of the Histogram (created by joining these components with
+	// "_"). Only Name is mandatory, the others merely help structuring the
+	// name. Note that the fully-qualified name of the Histogram must be a
+	// valid Prometheus metric name.
+	Namespace string
+	Subsystem string
+	Name      string
+
+	// Help provides information about this Histogram.
+	//
+	// Metrics with the same fully-qualified name must have the same Help
+	// string.
+	Help string
+
+	// ConstLabels are used to attach fixed labels to this metric. Metrics
+	// with the same fully-qualified name must have the same label names in
+	// their ConstLabels.
+	//
+	// ConstLabels are only used rarely. In particular, do not use them to
+	// attach the same labels to all your metrics. Those use cases are
+	// better covered by target labels set by the scraping Prometheus
+	// server, or by one specific metric (e.g. a build_info or a
+	// machine_role metric). See also
+	// https://prometheus.io/docs/instrumenting/writing_exporters/#target-labels,-not-static-scraped-labels
+	ConstLabels Labels
+
+	// Buckets defines the buckets into which observations are counted. Each
+	// element in the slice is the upper inclusive bound of a bucket. The
+	// values must be sorted in strictly increasing order. There is no need
+	// to add a highest bucket with +Inf bound, it will be added
+	// implicitly. The default value is DefBuckets.
+	Buckets []float64
+}
+
+// NewHistogram creates a new Histogram based on the provided HistogramOpts. It
+// panics if the buckets in HistogramOpts are not in strictly increasing order.
+func NewHistogram(opts HistogramOpts) Histogram {
+	return newHistogram(
+		NewDesc(
+			BuildFQName(opts.Namespace, opts.Subsystem, opts.Name),
+			opts.Help,
+			nil,
+			opts.ConstLabels,
+		),
+		opts,
+	)
+}
+
+func newHistogram(desc *Desc, opts HistogramOpts, labelValues ...string) Histogram {
+	if len(desc.variableLabels) != len(labelValues) {
+		panic(makeInconsistentCardinalityError(desc.fqName, desc.variableLabels, labelValues))
+	}
+
+	for _, n := range desc.variableLabels {
+		if n == bucketLabel {
+			panic(errBucketLabelNotAllowed)
+		}
+	}
+	for _, lp := range desc.constLabelPairs {
+		if lp.GetName() == bucketLabel {
+			panic(errBucketLabelNotAllowed)
+		}
+	}
+
+	if len(opts.Buckets) == 0 {
+		opts.Buckets = DefBuckets
+	}
+
+	h := &histogram{
+		desc:        desc,
+		upperBounds: opts.Buckets,
+		labelPairs:  makeLabelPairs(desc, labelValues),
+		counts:      [2]*histogramCounts{&histogramCounts{}, &histogramCounts{}},
+	}
+	for i, upperBound := range h.upperBounds {
+		if i < len(h.upperBounds)-1 {
+			if upperBound >= h.upperBounds[i+1] {
+				panic(fmt.Errorf(
+					"histogram buckets must be in increasing order: %f >= %f",
+					upperBound, h.upperBounds[i+1],
+				))
+			}
+		} else {
+			if math.IsInf(upperBound, +1) {
+				// The +Inf bucket is implicit. Remove it here.
+				h.upperBounds = h.upperBounds[:i]
+			}
+		}
+	}
+	// Finally we know the final length of h.upperBounds and can make buckets
+	// for both counts:
+	h.counts[0].buckets = make([]uint64, len(h.upperBounds))
+	h.counts[1].buckets = make([]uint64, len(h.upperBounds))
+
+	h.init(h) // Init self-collection.
+	return h
+}
+
+type histogramCounts struct {
+	// sumBits contains the bits of the float64 representing the sum of all
+	// observations. sumBits and count have to go first in the struct to
+	// guarantee alignment for atomic operations.
+	// http://golang.org/pkg/sync/atomic/#pkg-note-BUG
+	sumBits uint64
+	count   uint64
+	buckets []uint64
+}
+
+type histogram struct {
+	// countAndHotIdx enables lock-free writes with use of atomic updates.
+	// The most significant bit is the hot index [0 or 1] of the count field
+	// below. Observe calls update the hot one. All remaining bits count the
+	// number of Observe calls. Observe starts by incrementing this counter,
+	// and finish by incrementing the count field in the respective
+	// histogramCounts, as a marker for completion.
+	//
+	// Calls of the Write method (which are non-mutating reads from the
+	// perspective of the histogram) swap the hot–cold under the writeMtx
+	// lock. A cooldown is awaited (while locked) by comparing the number of
+	// observations with the initiation count. Once they match, then the
+	// last observation on the now cool one has completed. All cool fields must
+	// be merged into the new hot before releasing writeMtx.
+	//
+	// Fields with atomic access first! See alignment constraint:
+	// http://golang.org/pkg/sync/atomic/#pkg-note-BUG
+	countAndHotIdx uint64
+
+	selfCollector
+	desc     *Desc
+	writeMtx sync.Mutex // Only used in the Write method.
+
+	// Two counts, one is "hot" for lock-free observations, the other is
+	// "cold" for writing out a dto.Metric. It has to be an array of
+	// pointers to guarantee 64bit alignment of the histogramCounts, see
+	// http://golang.org/pkg/sync/atomic/#pkg-note-BUG.
+	counts [2]*histogramCounts
+
+	upperBounds []float64
+	labelPairs  []*dto.LabelPair
+}
+
+func (h *histogram) Desc() *Desc {
+	return h.desc
+}
+
+func (h *histogram) Observe(v float64) {
+	// TODO(beorn7): For small numbers of buckets (<30), a linear search is
+	// slightly faster than the binary search. If we really care, we could
+	// switch from one search strategy to the other depending on the number
+	// of buckets.
+	//
+	// Microbenchmarks (BenchmarkHistogramNoLabels):
+	// 11 buckets: 38.3 ns/op linear - binary 48.7 ns/op
+	// 100 buckets: 78.1 ns/op linear - binary 54.9 ns/op
+	// 300 buckets: 154 ns/op linear - binary 61.6 ns/op
+	i := sort.SearchFloat64s(h.upperBounds, v)
+
+	// We increment h.countAndHotIdx so that the counter in the lower
+	// 63 bits gets incremented. At the same time, we get the new value
+	// back, which we can use to find the currently-hot counts.
+	n := atomic.AddUint64(&h.countAndHotIdx, 1)
+	hotCounts := h.counts[n>>63]
+
+	if i < len(h.upperBounds) {
+		atomic.AddUint64(&hotCounts.buckets[i], 1)
+	}
+	for {
+		oldBits := atomic.LoadUint64(&hotCounts.sumBits)
+		newBits := math.Float64bits(math.Float64frombits(oldBits) + v)
+		if atomic.CompareAndSwapUint64(&hotCounts.sumBits, oldBits, newBits) {
+			break
+		}
+	}
+	// Increment count last as we take it as a signal that the observation
+	// is complete.
+	atomic.AddUint64(&hotCounts.count, 1)
+}
+
+func (h *histogram) Write(out *dto.Metric) error {
+	// For simplicity, we protect this whole method by a mutex. It is not in
+	// the hot path, i.e. Observe is called much more often than Write. The
+	// complication of making Write lock-free isn't worth it, if possible at
+	// all.
+	h.writeMtx.Lock()
+	defer h.writeMtx.Unlock()
+
+	// Adding 1<<63 switches the hot index (from 0 to 1 or from 1 to 0)
+	// without touching the count bits. See the struct comments for a full
+	// description of the algorithm.
+	n := atomic.AddUint64(&h.countAndHotIdx, 1<<63)
+	// count is contained unchanged in the lower 63 bits.
+	count := n & ((1 << 63) - 1)
+	// The most significant bit tells us which counts is hot. The complement
+	// is thus the cold one.
+	hotCounts := h.counts[n>>63]
+	coldCounts := h.counts[(^n)>>63]
+
+	// Await cooldown.
+	for count != atomic.LoadUint64(&coldCounts.count) {
+		runtime.Gosched() // Let observations get work done.
+	}
+
+	his := &dto.Histogram{
+		Bucket:      make([]*dto.Bucket, len(h.upperBounds)),
+		SampleCount: proto.Uint64(count),
+		SampleSum:   proto.Float64(math.Float64frombits(atomic.LoadUint64(&coldCounts.sumBits))),
+	}
+	var cumCount uint64
+	for i, upperBound := range h.upperBounds {
+		cumCount += atomic.LoadUint64(&coldCounts.buckets[i])
+		his.Bucket[i] = &dto.Bucket{
+			CumulativeCount: proto.Uint64(cumCount),
+			UpperBound:      proto.Float64(upperBound),
+		}
+	}
+
+	out.Histogram = his
+	out.Label = h.labelPairs
+
+	// Finally add all the cold counts to the new hot counts and reset the cold counts.
+	atomic.AddUint64(&hotCounts.count, count)
+	atomic.StoreUint64(&coldCounts.count, 0)
+	for {
+		oldBits := atomic.LoadUint64(&hotCounts.sumBits)
+		newBits := math.Float64bits(math.Float64frombits(oldBits) + his.GetSampleSum())
+		if atomic.CompareAndSwapUint64(&hotCounts.sumBits, oldBits, newBits) {
+			atomic.StoreUint64(&coldCounts.sumBits, 0)
+			break
+		}
+	}
+	for i := range h.upperBounds {
+		atomic.AddUint64(&hotCounts.buckets[i], atomic.LoadUint64(&coldCounts.buckets[i]))
+		atomic.StoreUint64(&coldCounts.buckets[i], 0)
+	}
+	return nil
+}
+
+// HistogramVec is a Collector that bundles a set of Histograms that all share the
+// same Desc, but have different values for their variable labels. This is used
+// if you want to count the same thing partitioned by various dimensions
+// (e.g. HTTP request latencies, partitioned by status code and method). Create
+// instances with NewHistogramVec.
+type HistogramVec struct {
+	*metricVec
+}
+
+// NewHistogramVec creates a new HistogramVec based on the provided HistogramOpts and
+// partitioned by the given label names.
+func NewHistogramVec(opts HistogramOpts, labelNames []string) *HistogramVec {
+	desc := NewDesc(
+		BuildFQName(opts.Namespace, opts.Subsystem, opts.Name),
+		opts.Help,
+		labelNames,
+		opts.ConstLabels,
+	)
+	return &HistogramVec{
+		metricVec: newMetricVec(desc, func(lvs ...string) Metric {
+			return newHistogram(desc, opts, lvs...)
+		}),
+	}
+}
+
+// GetMetricWithLabelValues returns the Histogram for the given slice of label
+// values (same order as the VariableLabels in Desc). If that combination of
+// label values is accessed for the first time, a new Histogram is created.
+//
+// It is possible to call this method without using the returned Histogram to only
+// create the new Histogram but leave it at its starting value, a Histogram without
+// any observations.
+//
+// Keeping the Histogram for later use is possible (and should be considered if
+// performance is critical), but keep in mind that Reset, DeleteLabelValues and
+// Delete can be used to delete the Histogram from the HistogramVec. In that case, the
+// Histogram will still exist, but it will not be exported anymore, even if a
+// Histogram with the same label values is created later. See also the CounterVec
+// example.
+//
+// An error is returned if the number of label values is not the same as the
+// number of VariableLabels in Desc (minus any curried labels).
+//
+// Note that for more than one label value, this method is prone to mistakes
+// caused by an incorrect order of arguments. Consider GetMetricWith(Labels) as
+// an alternative to avoid that type of mistake. For higher label numbers, the
+// latter has a much more readable (albeit more verbose) syntax, but it comes
+// with a performance overhead (for creating and processing the Labels map).
+// See also the GaugeVec example.
+func (v *HistogramVec) GetMetricWithLabelValues(lvs ...string) (Observer, error) {
+	metric, err := v.metricVec.getMetricWithLabelValues(lvs...)
+	if metric != nil {
+		return metric.(Observer), err
+	}
+	return nil, err
+}
+
+// GetMetricWith returns the Histogram for the given Labels map (the label names
+// must match those of the VariableLabels in Desc). If that label map is
+// accessed for the first time, a new Histogram is created. Implications of
+// creating a Histogram without using it and keeping the Histogram for later use
+// are the same as for GetMetricWithLabelValues.
+//
+// An error is returned if the number and names of the Labels are inconsistent
+// with those of the VariableLabels in Desc (minus any curried labels).
+//
+// This method is used for the same purpose as
+// GetMetricWithLabelValues(...string). See there for pros and cons of the two
+// methods.
+func (v *HistogramVec) GetMetricWith(labels Labels) (Observer, error) {
+	metric, err := v.metricVec.getMetricWith(labels)
+	if metric != nil {
+		return metric.(Observer), err
+	}
+	return nil, err
+}
+
+// WithLabelValues works as GetMetricWithLabelValues, but panics where
+// GetMetricWithLabelValues would have returned an error. Not returning an
+// error allows shortcuts like
+//     myVec.WithLabelValues("404", "GET").Observe(42.21)
+func (v *HistogramVec) WithLabelValues(lvs ...string) Observer {
+	h, err := v.GetMetricWithLabelValues(lvs...)
+	if err != nil {
+		panic(err)
+	}
+	return h
+}
+
+// With works as GetMetricWith but panics where GetMetricWithLabels would have
+// returned an error. Not returning an error allows shortcuts like
+//     myVec.With(prometheus.Labels{"code": "404", "method": "GET"}).Observe(42.21)
+func (v *HistogramVec) With(labels Labels) Observer {
+	h, err := v.GetMetricWith(labels)
+	if err != nil {
+		panic(err)
+	}
+	return h
+}
+
+// CurryWith returns a vector curried with the provided labels, i.e. the
+// returned vector has those labels pre-set for all labeled operations performed
+// on it. The cardinality of the curried vector is reduced accordingly. The
+// order of the remaining labels stays the same (just with the curried labels
+// taken out of the sequence – which is relevant for the
+// (GetMetric)WithLabelValues methods). It is possible to curry a curried
+// vector, but only with labels not yet used for currying before.
+//
+// The metrics contained in the HistogramVec are shared between the curried and
+// uncurried vectors. They are just accessed differently. Curried and uncurried
+// vectors behave identically in terms of collection. Only one must be
+// registered with a given registry (usually the uncurried version). The Reset
+// method deletes all metrics, even if called on a curried vector.
+func (v *HistogramVec) CurryWith(labels Labels) (ObserverVec, error) {
+	vec, err := v.curryWith(labels)
+	if vec != nil {
+		return &HistogramVec{vec}, err
+	}
+	return nil, err
+}
+
+// MustCurryWith works as CurryWith but panics where CurryWith would have
+// returned an error.
+func (v *HistogramVec) MustCurryWith(labels Labels) ObserverVec {
+	vec, err := v.CurryWith(labels)
+	if err != nil {
+		panic(err)
+	}
+	return vec
+}
+
+type constHistogram struct {
+	desc       *Desc
+	count      uint64
+	sum        float64
+	buckets    map[float64]uint64
+	labelPairs []*dto.LabelPair
+}
+
+func (h *constHistogram) Desc() *Desc {
+	return h.desc
+}
+
+func (h *constHistogram) Write(out *dto.Metric) error {
+	his := &dto.Histogram{}
+	buckets := make([]*dto.Bucket, 0, len(h.buckets))
+
+	his.SampleCount = proto.Uint64(h.count)
+	his.SampleSum = proto.Float64(h.sum)
+
+	for upperBound, count := range h.buckets {
+		buckets = append(buckets, &dto.Bucket{
+			CumulativeCount: proto.Uint64(count),
+			UpperBound:      proto.Float64(upperBound),
+		})
+	}
+
+	if len(buckets) > 0 {
+		sort.Sort(buckSort(buckets))
+	}
+	his.Bucket = buckets
+
+	out.Histogram = his
+	out.Label = h.labelPairs
+
+	return nil
+}
+
+// NewConstHistogram returns a metric representing a Prometheus histogram with
+// fixed values for the count, sum, and bucket counts. As those parameters
+// cannot be changed, the returned value does not implement the Histogram
+// interface (but only the Metric interface). Users of this package will not
+// have much use for it in regular operations. However, when implementing custom
+// Collectors, it is useful as a throw-away metric that is generated on the fly
+// to send it to Prometheus in the Collect method.
+//
+// buckets is a map of upper bounds to cumulative counts, excluding the +Inf
+// bucket.
+//
+// NewConstHistogram returns an error if the length of labelValues is not
+// consistent with the variable labels in Desc or if Desc is invalid.
+func NewConstHistogram(
+	desc *Desc,
+	count uint64,
+	sum float64,
+	buckets map[float64]uint64,
+	labelValues ...string,
+) (Metric, error) {
+	if desc.err != nil {
+		return nil, desc.err
+	}
+	if err := validateLabelValues(labelValues, len(desc.variableLabels)); err != nil {
+		return nil, err
+	}
+	return &constHistogram{
+		desc:       desc,
+		count:      count,
+		sum:        sum,
+		buckets:    buckets,
+		labelPairs: makeLabelPairs(desc, labelValues),
+	}, nil
+}
+
+// MustNewConstHistogram is a version of NewConstHistogram that panics where
+// NewConstMetric would have returned an error.
+func MustNewConstHistogram(
+	desc *Desc,
+	count uint64,
+	sum float64,
+	buckets map[float64]uint64,
+	labelValues ...string,
+) Metric {
+	m, err := NewConstHistogram(desc, count, sum, buckets, labelValues...)
+	if err != nil {
+		panic(err)
+	}
+	return m
+}
+
+type buckSort []*dto.Bucket
+
+func (s buckSort) Len() int {
+	return len(s)
+}
+
+func (s buckSort) Swap(i, j int) {
+	s[i], s[j] = s[j], s[i]
+}
+
+func (s buckSort) Less(i, j int) bool {
+	return s[i].GetUpperBound() < s[j].GetUpperBound()
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/internal/metric.go b/vendor/github.com/prometheus/client_golang/prometheus/internal/metric.go
new file mode 100644
index 0000000..351c26e
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/internal/metric.go
@@ -0,0 +1,85 @@
+// Copyright 2018 The Prometheus 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 internal
+
+import (
+	"sort"
+
+	dto "github.com/prometheus/client_model/go"
+)
+
+// metricSorter is a sortable slice of *dto.Metric.
+type metricSorter []*dto.Metric
+
+func (s metricSorter) Len() int {
+	return len(s)
+}
+
+func (s metricSorter) Swap(i, j int) {
+	s[i], s[j] = s[j], s[i]
+}
+
+func (s metricSorter) Less(i, j int) bool {
+	if len(s[i].Label) != len(s[j].Label) {
+		// This should not happen. The metrics are
+		// inconsistent. However, we have to deal with the fact, as
+		// people might use custom collectors or metric family injection
+		// to create inconsistent metrics. So let's simply compare the
+		// number of labels in this case. That will still yield
+		// reproducible sorting.
+		return len(s[i].Label) < len(s[j].Label)
+	}
+	for n, lp := range s[i].Label {
+		vi := lp.GetValue()
+		vj := s[j].Label[n].GetValue()
+		if vi != vj {
+			return vi < vj
+		}
+	}
+
+	// We should never arrive here. Multiple metrics with the same
+	// label set in the same scrape will lead to undefined ingestion
+	// behavior. However, as above, we have to provide stable sorting
+	// here, even for inconsistent metrics. So sort equal metrics
+	// by their timestamp, with missing timestamps (implying "now")
+	// coming last.
+	if s[i].TimestampMs == nil {
+		return false
+	}
+	if s[j].TimestampMs == nil {
+		return true
+	}
+	return s[i].GetTimestampMs() < s[j].GetTimestampMs()
+}
+
+// NormalizeMetricFamilies returns a MetricFamily slice with empty
+// MetricFamilies pruned and the remaining MetricFamilies sorted by name within
+// the slice, with the contained Metrics sorted within each MetricFamily.
+func NormalizeMetricFamilies(metricFamiliesByName map[string]*dto.MetricFamily) []*dto.MetricFamily {
+	for _, mf := range metricFamiliesByName {
+		sort.Sort(metricSorter(mf.Metric))
+	}
+	names := make([]string, 0, len(metricFamiliesByName))
+	for name, mf := range metricFamiliesByName {
+		if len(mf.Metric) > 0 {
+			names = append(names, name)
+		}
+	}
+	sort.Strings(names)
+	result := make([]*dto.MetricFamily, 0, len(names))
+	for _, name := range names {
+		result = append(result, metricFamiliesByName[name])
+	}
+	return result
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/labels.go b/vendor/github.com/prometheus/client_golang/prometheus/labels.go
new file mode 100644
index 0000000..2744443
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/labels.go
@@ -0,0 +1,87 @@
+// Copyright 2018 The Prometheus 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 prometheus
+
+import (
+	"errors"
+	"fmt"
+	"strings"
+	"unicode/utf8"
+
+	"github.com/prometheus/common/model"
+)
+
+// Labels represents a collection of label name -> value mappings. This type is
+// commonly used with the With(Labels) and GetMetricWith(Labels) methods of
+// metric vector Collectors, e.g.:
+//     myVec.With(Labels{"code": "404", "method": "GET"}).Add(42)
+//
+// The other use-case is the specification of constant label pairs in Opts or to
+// create a Desc.
+type Labels map[string]string
+
+// reservedLabelPrefix is a prefix which is not legal in user-supplied
+// label names.
+const reservedLabelPrefix = "__"
+
+var errInconsistentCardinality = errors.New("inconsistent label cardinality")
+
+func makeInconsistentCardinalityError(fqName string, labels, labelValues []string) error {
+	return fmt.Errorf(
+		"%s: %q has %d variable labels named %q but %d values %q were provided",
+		errInconsistentCardinality, fqName,
+		len(labels), labels,
+		len(labelValues), labelValues,
+	)
+}
+
+func validateValuesInLabels(labels Labels, expectedNumberOfValues int) error {
+	if len(labels) != expectedNumberOfValues {
+		return fmt.Errorf(
+			"%s: expected %d label values but got %d in %#v",
+			errInconsistentCardinality, expectedNumberOfValues,
+			len(labels), labels,
+		)
+	}
+
+	for name, val := range labels {
+		if !utf8.ValidString(val) {
+			return fmt.Errorf("label %s: value %q is not valid UTF-8", name, val)
+		}
+	}
+
+	return nil
+}
+
+func validateLabelValues(vals []string, expectedNumberOfValues int) error {
+	if len(vals) != expectedNumberOfValues {
+		return fmt.Errorf(
+			"%s: expected %d label values but got %d in %#v",
+			errInconsistentCardinality, expectedNumberOfValues,
+			len(vals), vals,
+		)
+	}
+
+	for _, val := range vals {
+		if !utf8.ValidString(val) {
+			return fmt.Errorf("label value %q is not valid UTF-8", val)
+		}
+	}
+
+	return nil
+}
+
+func checkLabelName(l string) bool {
+	return model.LabelName(l).IsValid() && !strings.HasPrefix(l, reservedLabelPrefix)
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/metric.go b/vendor/github.com/prometheus/client_golang/prometheus/metric.go
new file mode 100644
index 0000000..55e6d86
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/metric.go
@@ -0,0 +1,174 @@
+// Copyright 2014 The Prometheus 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 prometheus
+
+import (
+	"strings"
+	"time"
+
+	"github.com/golang/protobuf/proto"
+
+	dto "github.com/prometheus/client_model/go"
+)
+
+const separatorByte byte = 255
+
+// A Metric models a single sample value with its meta data being exported to
+// Prometheus. Implementations of Metric in this package are Gauge, Counter,
+// Histogram, Summary, and Untyped.
+type Metric interface {
+	// Desc returns the descriptor for the Metric. This method idempotently
+	// returns the same descriptor throughout the lifetime of the
+	// Metric. The returned descriptor is immutable by contract. A Metric
+	// unable to describe itself must return an invalid descriptor (created
+	// with NewInvalidDesc).
+	Desc() *Desc
+	// Write encodes the Metric into a "Metric" Protocol Buffer data
+	// transmission object.
+	//
+	// Metric implementations must observe concurrency safety as reads of
+	// this metric may occur at any time, and any blocking occurs at the
+	// expense of total performance of rendering all registered
+	// metrics. Ideally, Metric implementations should support concurrent
+	// readers.
+	//
+	// While populating dto.Metric, it is the responsibility of the
+	// implementation to ensure validity of the Metric protobuf (like valid
+	// UTF-8 strings or syntactically valid metric and label names). It is
+	// recommended to sort labels lexicographically. Callers of Write should
+	// still make sure of sorting if they depend on it.
+	Write(*dto.Metric) error
+	// TODO(beorn7): The original rationale of passing in a pre-allocated
+	// dto.Metric protobuf to save allocations has disappeared. The
+	// signature of this method should be changed to "Write() (*dto.Metric,
+	// error)".
+}
+
+// Opts bundles the options for creating most Metric types. Each metric
+// implementation XXX has its own XXXOpts type, but in most cases, it is just be
+// an alias of this type (which might change when the requirement arises.)
+//
+// It is mandatory to set Name to a non-empty string. All other fields are
+// optional and can safely be left at their zero value, although it is strongly
+// encouraged to set a Help string.
+type Opts struct {
+	// Namespace, Subsystem, and Name are components of the fully-qualified
+	// name of the Metric (created by joining these components with
+	// "_"). Only Name is mandatory, the others merely help structuring the
+	// name. Note that the fully-qualified name of the metric must be a
+	// valid Prometheus metric name.
+	Namespace string
+	Subsystem string
+	Name      string
+
+	// Help provides information about this metric.
+	//
+	// Metrics with the same fully-qualified name must have the same Help
+	// string.
+	Help string
+
+	// ConstLabels are used to attach fixed labels to this metric. Metrics
+	// with the same fully-qualified name must have the same label names in
+	// their ConstLabels.
+	//
+	// ConstLabels are only used rarely. In particular, do not use them to
+	// attach the same labels to all your metrics. Those use cases are
+	// better covered by target labels set by the scraping Prometheus
+	// server, or by one specific metric (e.g. a build_info or a
+	// machine_role metric). See also
+	// https://prometheus.io/docs/instrumenting/writing_exporters/#target-labels,-not-static-scraped-labels
+	ConstLabels Labels
+}
+
+// BuildFQName joins the given three name components by "_". Empty name
+// components are ignored. If the name parameter itself is empty, an empty
+// string is returned, no matter what. Metric implementations included in this
+// library use this function internally to generate the fully-qualified metric
+// name from the name component in their Opts. Users of the library will only
+// need this function if they implement their own Metric or instantiate a Desc
+// (with NewDesc) directly.
+func BuildFQName(namespace, subsystem, name string) string {
+	if name == "" {
+		return ""
+	}
+	switch {
+	case namespace != "" && subsystem != "":
+		return strings.Join([]string{namespace, subsystem, name}, "_")
+	case namespace != "":
+		return strings.Join([]string{namespace, name}, "_")
+	case subsystem != "":
+		return strings.Join([]string{subsystem, name}, "_")
+	}
+	return name
+}
+
+// labelPairSorter implements sort.Interface. It is used to sort a slice of
+// dto.LabelPair pointers.
+type labelPairSorter []*dto.LabelPair
+
+func (s labelPairSorter) Len() int {
+	return len(s)
+}
+
+func (s labelPairSorter) Swap(i, j int) {
+	s[i], s[j] = s[j], s[i]
+}
+
+func (s labelPairSorter) Less(i, j int) bool {
+	return s[i].GetName() < s[j].GetName()
+}
+
+type invalidMetric struct {
+	desc *Desc
+	err  error
+}
+
+// NewInvalidMetric returns a metric whose Write method always returns the
+// provided error. It is useful if a Collector finds itself unable to collect
+// a metric and wishes to report an error to the registry.
+func NewInvalidMetric(desc *Desc, err error) Metric {
+	return &invalidMetric{desc, err}
+}
+
+func (m *invalidMetric) Desc() *Desc { return m.desc }
+
+func (m *invalidMetric) Write(*dto.Metric) error { return m.err }
+
+type timestampedMetric struct {
+	Metric
+	t time.Time
+}
+
+func (m timestampedMetric) Write(pb *dto.Metric) error {
+	e := m.Metric.Write(pb)
+	pb.TimestampMs = proto.Int64(m.t.Unix()*1000 + int64(m.t.Nanosecond()/1000000))
+	return e
+}
+
+// NewMetricWithTimestamp returns a new Metric wrapping the provided Metric in a
+// way that it has an explicit timestamp set to the provided Time. This is only
+// useful in rare cases as the timestamp of a Prometheus metric should usually
+// be set by the Prometheus server during scraping. Exceptions include mirroring
+// metrics with given timestamps from other metric
+// sources.
+//
+// NewMetricWithTimestamp works best with MustNewConstMetric,
+// MustNewConstHistogram, and MustNewConstSummary, see example.
+//
+// Currently, the exposition formats used by Prometheus are limited to
+// millisecond resolution. Thus, the provided time will be rounded down to the
+// next full millisecond value.
+func NewMetricWithTimestamp(t time.Time, m Metric) Metric {
+	return timestampedMetric{Metric: m, t: t}
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/observer.go b/vendor/github.com/prometheus/client_golang/prometheus/observer.go
new file mode 100644
index 0000000..5806cd0
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/observer.go
@@ -0,0 +1,52 @@
+// Copyright 2017 The Prometheus 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 prometheus
+
+// Observer is the interface that wraps the Observe method, which is used by
+// Histogram and Summary to add observations.
+type Observer interface {
+	Observe(float64)
+}
+
+// The ObserverFunc type is an adapter to allow the use of ordinary
+// functions as Observers. If f is a function with the appropriate
+// signature, ObserverFunc(f) is an Observer that calls f.
+//
+// This adapter is usually used in connection with the Timer type, and there are
+// two general use cases:
+//
+// The most common one is to use a Gauge as the Observer for a Timer.
+// See the "Gauge" Timer example.
+//
+// The more advanced use case is to create a function that dynamically decides
+// which Observer to use for observing the duration. See the "Complex" Timer
+// example.
+type ObserverFunc func(float64)
+
+// Observe calls f(value). It implements Observer.
+func (f ObserverFunc) Observe(value float64) {
+	f(value)
+}
+
+// ObserverVec is an interface implemented by `HistogramVec` and `SummaryVec`.
+type ObserverVec interface {
+	GetMetricWith(Labels) (Observer, error)
+	GetMetricWithLabelValues(lvs ...string) (Observer, error)
+	With(Labels) Observer
+	WithLabelValues(...string) Observer
+	CurryWith(Labels) (ObserverVec, error)
+	MustCurryWith(Labels) ObserverVec
+
+	Collector
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/process_collector.go b/vendor/github.com/prometheus/client_golang/prometheus/process_collector.go
new file mode 100644
index 0000000..9b80979
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/process_collector.go
@@ -0,0 +1,151 @@
+// Copyright 2015 The Prometheus 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 prometheus
+
+import (
+	"errors"
+	"os"
+)
+
+type processCollector struct {
+	collectFn       func(chan<- Metric)
+	pidFn           func() (int, error)
+	reportErrors    bool
+	cpuTotal        *Desc
+	openFDs, maxFDs *Desc
+	vsize, maxVsize *Desc
+	rss             *Desc
+	startTime       *Desc
+}
+
+// ProcessCollectorOpts defines the behavior of a process metrics collector
+// created with NewProcessCollector.
+type ProcessCollectorOpts struct {
+	// PidFn returns the PID of the process the collector collects metrics
+	// for. It is called upon each collection. By default, the PID of the
+	// current process is used, as determined on construction time by
+	// calling os.Getpid().
+	PidFn func() (int, error)
+	// If non-empty, each of the collected metrics is prefixed by the
+	// provided string and an underscore ("_").
+	Namespace string
+	// If true, any error encountered during collection is reported as an
+	// invalid metric (see NewInvalidMetric). Otherwise, errors are ignored
+	// and the collected metrics will be incomplete. (Possibly, no metrics
+	// will be collected at all.) While that's usually not desired, it is
+	// appropriate for the common "mix-in" of process metrics, where process
+	// metrics are nice to have, but failing to collect them should not
+	// disrupt the collection of the remaining metrics.
+	ReportErrors bool
+}
+
+// NewProcessCollector returns a collector which exports the current state of
+// process metrics including CPU, memory and file descriptor usage as well as
+// the process start time. The detailed behavior is defined by the provided
+// ProcessCollectorOpts. The zero value of ProcessCollectorOpts creates a
+// collector for the current process with an empty namespace string and no error
+// reporting.
+//
+// The collector only works on operating systems with a Linux-style proc
+// filesystem and on Microsoft Windows. On other operating systems, it will not
+// collect any metrics.
+func NewProcessCollector(opts ProcessCollectorOpts) Collector {
+	ns := ""
+	if len(opts.Namespace) > 0 {
+		ns = opts.Namespace + "_"
+	}
+
+	c := &processCollector{
+		reportErrors: opts.ReportErrors,
+		cpuTotal: NewDesc(
+			ns+"process_cpu_seconds_total",
+			"Total user and system CPU time spent in seconds.",
+			nil, nil,
+		),
+		openFDs: NewDesc(
+			ns+"process_open_fds",
+			"Number of open file descriptors.",
+			nil, nil,
+		),
+		maxFDs: NewDesc(
+			ns+"process_max_fds",
+			"Maximum number of open file descriptors.",
+			nil, nil,
+		),
+		vsize: NewDesc(
+			ns+"process_virtual_memory_bytes",
+			"Virtual memory size in bytes.",
+			nil, nil,
+		),
+		maxVsize: NewDesc(
+			ns+"process_virtual_memory_max_bytes",
+			"Maximum amount of virtual memory available in bytes.",
+			nil, nil,
+		),
+		rss: NewDesc(
+			ns+"process_resident_memory_bytes",
+			"Resident memory size in bytes.",
+			nil, nil,
+		),
+		startTime: NewDesc(
+			ns+"process_start_time_seconds",
+			"Start time of the process since unix epoch in seconds.",
+			nil, nil,
+		),
+	}
+
+	if opts.PidFn == nil {
+		pid := os.Getpid()
+		c.pidFn = func() (int, error) { return pid, nil }
+	} else {
+		c.pidFn = opts.PidFn
+	}
+
+	// Set up process metric collection if supported by the runtime.
+	if canCollectProcess() {
+		c.collectFn = c.processCollect
+	} else {
+		c.collectFn = func(ch chan<- Metric) {
+			c.reportError(ch, nil, errors.New("process metrics not supported on this platform"))
+		}
+	}
+
+	return c
+}
+
+// Describe returns all descriptions of the collector.
+func (c *processCollector) Describe(ch chan<- *Desc) {
+	ch <- c.cpuTotal
+	ch <- c.openFDs
+	ch <- c.maxFDs
+	ch <- c.vsize
+	ch <- c.maxVsize
+	ch <- c.rss
+	ch <- c.startTime
+}
+
+// Collect returns the current state of all metrics of the collector.
+func (c *processCollector) Collect(ch chan<- Metric) {
+	c.collectFn(ch)
+}
+
+func (c *processCollector) reportError(ch chan<- Metric, desc *Desc, err error) {
+	if !c.reportErrors {
+		return
+	}
+	if desc == nil {
+		desc = NewInvalidDesc(err)
+	}
+	ch <- NewInvalidMetric(desc, err)
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/process_collector_other.go b/vendor/github.com/prometheus/client_golang/prometheus/process_collector_other.go
new file mode 100644
index 0000000..3117461
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/process_collector_other.go
@@ -0,0 +1,65 @@
+// Copyright 2019 The Prometheus 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.
+
+// +build !windows
+
+package prometheus
+
+import (
+	"github.com/prometheus/procfs"
+)
+
+func canCollectProcess() bool {
+	_, err := procfs.NewDefaultFS()
+	return err == nil
+}
+
+func (c *processCollector) processCollect(ch chan<- Metric) {
+	pid, err := c.pidFn()
+	if err != nil {
+		c.reportError(ch, nil, err)
+		return
+	}
+
+	p, err := procfs.NewProc(pid)
+	if err != nil {
+		c.reportError(ch, nil, err)
+		return
+	}
+
+	if stat, err := p.Stat(); err == nil {
+		ch <- MustNewConstMetric(c.cpuTotal, CounterValue, stat.CPUTime())
+		ch <- MustNewConstMetric(c.vsize, GaugeValue, float64(stat.VirtualMemory()))
+		ch <- MustNewConstMetric(c.rss, GaugeValue, float64(stat.ResidentMemory()))
+		if startTime, err := stat.StartTime(); err == nil {
+			ch <- MustNewConstMetric(c.startTime, GaugeValue, startTime)
+		} else {
+			c.reportError(ch, c.startTime, err)
+		}
+	} else {
+		c.reportError(ch, nil, err)
+	}
+
+	if fds, err := p.FileDescriptorsLen(); err == nil {
+		ch <- MustNewConstMetric(c.openFDs, GaugeValue, float64(fds))
+	} else {
+		c.reportError(ch, c.openFDs, err)
+	}
+
+	if limits, err := p.Limits(); err == nil {
+		ch <- MustNewConstMetric(c.maxFDs, GaugeValue, float64(limits.OpenFiles))
+		ch <- MustNewConstMetric(c.maxVsize, GaugeValue, float64(limits.AddressSpace))
+	} else {
+		c.reportError(ch, nil, err)
+	}
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/process_collector_windows.go b/vendor/github.com/prometheus/client_golang/prometheus/process_collector_windows.go
new file mode 100644
index 0000000..e0b935d
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/process_collector_windows.go
@@ -0,0 +1,112 @@
+// Copyright 2019 The Prometheus 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 prometheus
+
+import (
+	"syscall"
+	"unsafe"
+
+	"golang.org/x/sys/windows"
+)
+
+func canCollectProcess() bool {
+	return true
+}
+
+var (
+	modpsapi    = syscall.NewLazyDLL("psapi.dll")
+	modkernel32 = syscall.NewLazyDLL("kernel32.dll")
+
+	procGetProcessMemoryInfo  = modpsapi.NewProc("GetProcessMemoryInfo")
+	procGetProcessHandleCount = modkernel32.NewProc("GetProcessHandleCount")
+)
+
+type processMemoryCounters struct {
+	// https://docs.microsoft.com/en-us/windows/desktop/api/psapi/ns-psapi-_process_memory_counters_ex
+	_                          uint32
+	PageFaultCount             uint32
+	PeakWorkingSetSize         uint64
+	WorkingSetSize             uint64
+	QuotaPeakPagedPoolUsage    uint64
+	QuotaPagedPoolUsage        uint64
+	QuotaPeakNonPagedPoolUsage uint64
+	QuotaNonPagedPoolUsage     uint64
+	PagefileUsage              uint64
+	PeakPagefileUsage          uint64
+	PrivateUsage               uint64
+}
+
+func getProcessMemoryInfo(handle windows.Handle) (processMemoryCounters, error) {
+	mem := processMemoryCounters{}
+	r1, _, err := procGetProcessMemoryInfo.Call(
+		uintptr(handle),
+		uintptr(unsafe.Pointer(&mem)),
+		uintptr(unsafe.Sizeof(mem)),
+	)
+	if r1 != 1 {
+		return mem, err
+	} else {
+		return mem, nil
+	}
+}
+
+func getProcessHandleCount(handle windows.Handle) (uint32, error) {
+	var count uint32
+	r1, _, err := procGetProcessHandleCount.Call(
+		uintptr(handle),
+		uintptr(unsafe.Pointer(&count)),
+	)
+	if r1 != 1 {
+		return 0, err
+	} else {
+		return count, nil
+	}
+}
+
+func (c *processCollector) processCollect(ch chan<- Metric) {
+	h, err := windows.GetCurrentProcess()
+	if err != nil {
+		c.reportError(ch, nil, err)
+		return
+	}
+
+	var startTime, exitTime, kernelTime, userTime windows.Filetime
+	err = windows.GetProcessTimes(h, &startTime, &exitTime, &kernelTime, &userTime)
+	if err != nil {
+		c.reportError(ch, nil, err)
+		return
+	}
+	ch <- MustNewConstMetric(c.startTime, GaugeValue, float64(startTime.Nanoseconds()/1e9))
+	ch <- MustNewConstMetric(c.cpuTotal, CounterValue, fileTimeToSeconds(kernelTime)+fileTimeToSeconds(userTime))
+
+	mem, err := getProcessMemoryInfo(h)
+	if err != nil {
+		c.reportError(ch, nil, err)
+		return
+	}
+	ch <- MustNewConstMetric(c.vsize, GaugeValue, float64(mem.PrivateUsage))
+	ch <- MustNewConstMetric(c.rss, GaugeValue, float64(mem.WorkingSetSize))
+
+	handles, err := getProcessHandleCount(h)
+	if err != nil {
+		c.reportError(ch, nil, err)
+		return
+	}
+	ch <- MustNewConstMetric(c.openFDs, GaugeValue, float64(handles))
+	ch <- MustNewConstMetric(c.maxFDs, GaugeValue, float64(16*1024*1024)) // Windows has a hard-coded max limit, not per-process.
+}
+
+func fileTimeToSeconds(ft windows.Filetime) float64 {
+	return float64(uint64(ft.HighDateTime)<<32+uint64(ft.LowDateTime)) / 1e7
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/promhttp/delegator.go b/vendor/github.com/prometheus/client_golang/prometheus/promhttp/delegator.go
new file mode 100644
index 0000000..fa53568
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/promhttp/delegator.go
@@ -0,0 +1,357 @@
+// Copyright 2017 The Prometheus 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 promhttp
+
+import (
+	"bufio"
+	"io"
+	"net"
+	"net/http"
+)
+
+const (
+	closeNotifier = 1 << iota
+	flusher
+	hijacker
+	readerFrom
+	pusher
+)
+
+type delegator interface {
+	http.ResponseWriter
+
+	Status() int
+	Written() int64
+}
+
+type responseWriterDelegator struct {
+	http.ResponseWriter
+
+	status             int
+	written            int64
+	wroteHeader        bool
+	observeWriteHeader func(int)
+}
+
+func (r *responseWriterDelegator) Status() int {
+	return r.status
+}
+
+func (r *responseWriterDelegator) Written() int64 {
+	return r.written
+}
+
+func (r *responseWriterDelegator) WriteHeader(code int) {
+	r.status = code
+	r.wroteHeader = true
+	r.ResponseWriter.WriteHeader(code)
+	if r.observeWriteHeader != nil {
+		r.observeWriteHeader(code)
+	}
+}
+
+func (r *responseWriterDelegator) Write(b []byte) (int, error) {
+	if !r.wroteHeader {
+		r.WriteHeader(http.StatusOK)
+	}
+	n, err := r.ResponseWriter.Write(b)
+	r.written += int64(n)
+	return n, err
+}
+
+type closeNotifierDelegator struct{ *responseWriterDelegator }
+type flusherDelegator struct{ *responseWriterDelegator }
+type hijackerDelegator struct{ *responseWriterDelegator }
+type readerFromDelegator struct{ *responseWriterDelegator }
+type pusherDelegator struct{ *responseWriterDelegator }
+
+func (d closeNotifierDelegator) CloseNotify() <-chan bool {
+	//lint:ignore SA1019 http.CloseNotifier is deprecated but we don't want to
+	//remove support from client_golang yet.
+	return d.ResponseWriter.(http.CloseNotifier).CloseNotify()
+}
+func (d flusherDelegator) Flush() {
+	d.ResponseWriter.(http.Flusher).Flush()
+}
+func (d hijackerDelegator) Hijack() (net.Conn, *bufio.ReadWriter, error) {
+	return d.ResponseWriter.(http.Hijacker).Hijack()
+}
+func (d readerFromDelegator) ReadFrom(re io.Reader) (int64, error) {
+	if !d.wroteHeader {
+		d.WriteHeader(http.StatusOK)
+	}
+	n, err := d.ResponseWriter.(io.ReaderFrom).ReadFrom(re)
+	d.written += n
+	return n, err
+}
+func (d pusherDelegator) Push(target string, opts *http.PushOptions) error {
+	return d.ResponseWriter.(http.Pusher).Push(target, opts)
+}
+
+var pickDelegator = make([]func(*responseWriterDelegator) delegator, 32)
+
+func init() {
+	// TODO(beorn7): Code generation would help here.
+	pickDelegator[0] = func(d *responseWriterDelegator) delegator { // 0
+		return d
+	}
+	pickDelegator[closeNotifier] = func(d *responseWriterDelegator) delegator { // 1
+		return closeNotifierDelegator{d}
+	}
+	pickDelegator[flusher] = func(d *responseWriterDelegator) delegator { // 2
+		return flusherDelegator{d}
+	}
+	pickDelegator[flusher+closeNotifier] = func(d *responseWriterDelegator) delegator { // 3
+		return struct {
+			*responseWriterDelegator
+			http.Flusher
+			http.CloseNotifier
+		}{d, flusherDelegator{d}, closeNotifierDelegator{d}}
+	}
+	pickDelegator[hijacker] = func(d *responseWriterDelegator) delegator { // 4
+		return hijackerDelegator{d}
+	}
+	pickDelegator[hijacker+closeNotifier] = func(d *responseWriterDelegator) delegator { // 5
+		return struct {
+			*responseWriterDelegator
+			http.Hijacker
+			http.CloseNotifier
+		}{d, hijackerDelegator{d}, closeNotifierDelegator{d}}
+	}
+	pickDelegator[hijacker+flusher] = func(d *responseWriterDelegator) delegator { // 6
+		return struct {
+			*responseWriterDelegator
+			http.Hijacker
+			http.Flusher
+		}{d, hijackerDelegator{d}, flusherDelegator{d}}
+	}
+	pickDelegator[hijacker+flusher+closeNotifier] = func(d *responseWriterDelegator) delegator { // 7
+		return struct {
+			*responseWriterDelegator
+			http.Hijacker
+			http.Flusher
+			http.CloseNotifier
+		}{d, hijackerDelegator{d}, flusherDelegator{d}, closeNotifierDelegator{d}}
+	}
+	pickDelegator[readerFrom] = func(d *responseWriterDelegator) delegator { // 8
+		return readerFromDelegator{d}
+	}
+	pickDelegator[readerFrom+closeNotifier] = func(d *responseWriterDelegator) delegator { // 9
+		return struct {
+			*responseWriterDelegator
+			io.ReaderFrom
+			http.CloseNotifier
+		}{d, readerFromDelegator{d}, closeNotifierDelegator{d}}
+	}
+	pickDelegator[readerFrom+flusher] = func(d *responseWriterDelegator) delegator { // 10
+		return struct {
+			*responseWriterDelegator
+			io.ReaderFrom
+			http.Flusher
+		}{d, readerFromDelegator{d}, flusherDelegator{d}}
+	}
+	pickDelegator[readerFrom+flusher+closeNotifier] = func(d *responseWriterDelegator) delegator { // 11
+		return struct {
+			*responseWriterDelegator
+			io.ReaderFrom
+			http.Flusher
+			http.CloseNotifier
+		}{d, readerFromDelegator{d}, flusherDelegator{d}, closeNotifierDelegator{d}}
+	}
+	pickDelegator[readerFrom+hijacker] = func(d *responseWriterDelegator) delegator { // 12
+		return struct {
+			*responseWriterDelegator
+			io.ReaderFrom
+			http.Hijacker
+		}{d, readerFromDelegator{d}, hijackerDelegator{d}}
+	}
+	pickDelegator[readerFrom+hijacker+closeNotifier] = func(d *responseWriterDelegator) delegator { // 13
+		return struct {
+			*responseWriterDelegator
+			io.ReaderFrom
+			http.Hijacker
+			http.CloseNotifier
+		}{d, readerFromDelegator{d}, hijackerDelegator{d}, closeNotifierDelegator{d}}
+	}
+	pickDelegator[readerFrom+hijacker+flusher] = func(d *responseWriterDelegator) delegator { // 14
+		return struct {
+			*responseWriterDelegator
+			io.ReaderFrom
+			http.Hijacker
+			http.Flusher
+		}{d, readerFromDelegator{d}, hijackerDelegator{d}, flusherDelegator{d}}
+	}
+	pickDelegator[readerFrom+hijacker+flusher+closeNotifier] = func(d *responseWriterDelegator) delegator { // 15
+		return struct {
+			*responseWriterDelegator
+			io.ReaderFrom
+			http.Hijacker
+			http.Flusher
+			http.CloseNotifier
+		}{d, readerFromDelegator{d}, hijackerDelegator{d}, flusherDelegator{d}, closeNotifierDelegator{d}}
+	}
+	pickDelegator[pusher] = func(d *responseWriterDelegator) delegator { // 16
+		return pusherDelegator{d}
+	}
+	pickDelegator[pusher+closeNotifier] = func(d *responseWriterDelegator) delegator { // 17
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			http.CloseNotifier
+		}{d, pusherDelegator{d}, closeNotifierDelegator{d}}
+	}
+	pickDelegator[pusher+flusher] = func(d *responseWriterDelegator) delegator { // 18
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			http.Flusher
+		}{d, pusherDelegator{d}, flusherDelegator{d}}
+	}
+	pickDelegator[pusher+flusher+closeNotifier] = func(d *responseWriterDelegator) delegator { // 19
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			http.Flusher
+			http.CloseNotifier
+		}{d, pusherDelegator{d}, flusherDelegator{d}, closeNotifierDelegator{d}}
+	}
+	pickDelegator[pusher+hijacker] = func(d *responseWriterDelegator) delegator { // 20
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			http.Hijacker
+		}{d, pusherDelegator{d}, hijackerDelegator{d}}
+	}
+	pickDelegator[pusher+hijacker+closeNotifier] = func(d *responseWriterDelegator) delegator { // 21
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			http.Hijacker
+			http.CloseNotifier
+		}{d, pusherDelegator{d}, hijackerDelegator{d}, closeNotifierDelegator{d}}
+	}
+	pickDelegator[pusher+hijacker+flusher] = func(d *responseWriterDelegator) delegator { // 22
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			http.Hijacker
+			http.Flusher
+		}{d, pusherDelegator{d}, hijackerDelegator{d}, flusherDelegator{d}}
+	}
+	pickDelegator[pusher+hijacker+flusher+closeNotifier] = func(d *responseWriterDelegator) delegator { //23
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			http.Hijacker
+			http.Flusher
+			http.CloseNotifier
+		}{d, pusherDelegator{d}, hijackerDelegator{d}, flusherDelegator{d}, closeNotifierDelegator{d}}
+	}
+	pickDelegator[pusher+readerFrom] = func(d *responseWriterDelegator) delegator { // 24
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			io.ReaderFrom
+		}{d, pusherDelegator{d}, readerFromDelegator{d}}
+	}
+	pickDelegator[pusher+readerFrom+closeNotifier] = func(d *responseWriterDelegator) delegator { // 25
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			io.ReaderFrom
+			http.CloseNotifier
+		}{d, pusherDelegator{d}, readerFromDelegator{d}, closeNotifierDelegator{d}}
+	}
+	pickDelegator[pusher+readerFrom+flusher] = func(d *responseWriterDelegator) delegator { // 26
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			io.ReaderFrom
+			http.Flusher
+		}{d, pusherDelegator{d}, readerFromDelegator{d}, flusherDelegator{d}}
+	}
+	pickDelegator[pusher+readerFrom+flusher+closeNotifier] = func(d *responseWriterDelegator) delegator { // 27
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			io.ReaderFrom
+			http.Flusher
+			http.CloseNotifier
+		}{d, pusherDelegator{d}, readerFromDelegator{d}, flusherDelegator{d}, closeNotifierDelegator{d}}
+	}
+	pickDelegator[pusher+readerFrom+hijacker] = func(d *responseWriterDelegator) delegator { // 28
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			io.ReaderFrom
+			http.Hijacker
+		}{d, pusherDelegator{d}, readerFromDelegator{d}, hijackerDelegator{d}}
+	}
+	pickDelegator[pusher+readerFrom+hijacker+closeNotifier] = func(d *responseWriterDelegator) delegator { // 29
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			io.ReaderFrom
+			http.Hijacker
+			http.CloseNotifier
+		}{d, pusherDelegator{d}, readerFromDelegator{d}, hijackerDelegator{d}, closeNotifierDelegator{d}}
+	}
+	pickDelegator[pusher+readerFrom+hijacker+flusher] = func(d *responseWriterDelegator) delegator { // 30
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			io.ReaderFrom
+			http.Hijacker
+			http.Flusher
+		}{d, pusherDelegator{d}, readerFromDelegator{d}, hijackerDelegator{d}, flusherDelegator{d}}
+	}
+	pickDelegator[pusher+readerFrom+hijacker+flusher+closeNotifier] = func(d *responseWriterDelegator) delegator { // 31
+		return struct {
+			*responseWriterDelegator
+			http.Pusher
+			io.ReaderFrom
+			http.Hijacker
+			http.Flusher
+			http.CloseNotifier
+		}{d, pusherDelegator{d}, readerFromDelegator{d}, hijackerDelegator{d}, flusherDelegator{d}, closeNotifierDelegator{d}}
+	}
+}
+
+func newDelegator(w http.ResponseWriter, observeWriteHeaderFunc func(int)) delegator {
+	d := &responseWriterDelegator{
+		ResponseWriter:     w,
+		observeWriteHeader: observeWriteHeaderFunc,
+	}
+
+	id := 0
+	//lint:ignore SA1019 http.CloseNotifier is deprecated but we don't want to
+	//remove support from client_golang yet.
+	if _, ok := w.(http.CloseNotifier); ok {
+		id += closeNotifier
+	}
+	if _, ok := w.(http.Flusher); ok {
+		id += flusher
+	}
+	if _, ok := w.(http.Hijacker); ok {
+		id += hijacker
+	}
+	if _, ok := w.(io.ReaderFrom); ok {
+		id += readerFrom
+	}
+	if _, ok := w.(http.Pusher); ok {
+		id += pusher
+	}
+
+	return pickDelegator[id](d)
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/promhttp/http.go b/vendor/github.com/prometheus/client_golang/prometheus/promhttp/http.go
new file mode 100644
index 0000000..cea5a90
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/promhttp/http.go
@@ -0,0 +1,349 @@
+// Copyright 2016 The Prometheus 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 promhttp provides tooling around HTTP servers and clients.
+//
+// First, the package allows the creation of http.Handler instances to expose
+// Prometheus metrics via HTTP. promhttp.Handler acts on the
+// prometheus.DefaultGatherer. With HandlerFor, you can create a handler for a
+// custom registry or anything that implements the Gatherer interface. It also
+// allows the creation of handlers that act differently on errors or allow to
+// log errors.
+//
+// Second, the package provides tooling to instrument instances of http.Handler
+// via middleware. Middleware wrappers follow the naming scheme
+// InstrumentHandlerX, where X describes the intended use of the middleware.
+// See each function's doc comment for specific details.
+//
+// Finally, the package allows for an http.RoundTripper to be instrumented via
+// middleware. Middleware wrappers follow the naming scheme
+// InstrumentRoundTripperX, where X describes the intended use of the
+// middleware. See each function's doc comment for specific details.
+package promhttp
+
+import (
+	"compress/gzip"
+	"fmt"
+	"io"
+	"net/http"
+	"strings"
+	"sync"
+	"time"
+
+	"github.com/prometheus/common/expfmt"
+
+	"github.com/prometheus/client_golang/prometheus"
+)
+
+const (
+	contentTypeHeader     = "Content-Type"
+	contentEncodingHeader = "Content-Encoding"
+	acceptEncodingHeader  = "Accept-Encoding"
+)
+
+var gzipPool = sync.Pool{
+	New: func() interface{} {
+		return gzip.NewWriter(nil)
+	},
+}
+
+// Handler returns an http.Handler for the prometheus.DefaultGatherer, using
+// default HandlerOpts, i.e. it reports the first error as an HTTP error, it has
+// no error logging, and it applies compression if requested by the client.
+//
+// The returned http.Handler is already instrumented using the
+// InstrumentMetricHandler function and the prometheus.DefaultRegisterer. If you
+// create multiple http.Handlers by separate calls of the Handler function, the
+// metrics used for instrumentation will be shared between them, providing
+// global scrape counts.
+//
+// This function is meant to cover the bulk of basic use cases. If you are doing
+// anything that requires more customization (including using a non-default
+// Gatherer, different instrumentation, and non-default HandlerOpts), use the
+// HandlerFor function. See there for details.
+func Handler() http.Handler {
+	return InstrumentMetricHandler(
+		prometheus.DefaultRegisterer, HandlerFor(prometheus.DefaultGatherer, HandlerOpts{}),
+	)
+}
+
+// HandlerFor returns an uninstrumented http.Handler for the provided
+// Gatherer. The behavior of the Handler is defined by the provided
+// HandlerOpts. Thus, HandlerFor is useful to create http.Handlers for custom
+// Gatherers, with non-default HandlerOpts, and/or with custom (or no)
+// instrumentation. Use the InstrumentMetricHandler function to apply the same
+// kind of instrumentation as it is used by the Handler function.
+func HandlerFor(reg prometheus.Gatherer, opts HandlerOpts) http.Handler {
+	var (
+		inFlightSem chan struct{}
+		errCnt      = prometheus.NewCounterVec(
+			prometheus.CounterOpts{
+				Name: "promhttp_metric_handler_errors_total",
+				Help: "Total number of internal errors encountered by the promhttp metric handler.",
+			},
+			[]string{"cause"},
+		)
+	)
+
+	if opts.MaxRequestsInFlight > 0 {
+		inFlightSem = make(chan struct{}, opts.MaxRequestsInFlight)
+	}
+	if opts.Registry != nil {
+		// Initialize all possibilites that can occur below.
+		errCnt.WithLabelValues("gathering")
+		errCnt.WithLabelValues("encoding")
+		if err := opts.Registry.Register(errCnt); err != nil {
+			if are, ok := err.(prometheus.AlreadyRegisteredError); ok {
+				errCnt = are.ExistingCollector.(*prometheus.CounterVec)
+			} else {
+				panic(err)
+			}
+		}
+	}
+
+	h := http.HandlerFunc(func(rsp http.ResponseWriter, req *http.Request) {
+		if inFlightSem != nil {
+			select {
+			case inFlightSem <- struct{}{}: // All good, carry on.
+				defer func() { <-inFlightSem }()
+			default:
+				http.Error(rsp, fmt.Sprintf(
+					"Limit of concurrent requests reached (%d), try again later.", opts.MaxRequestsInFlight,
+				), http.StatusServiceUnavailable)
+				return
+			}
+		}
+		mfs, err := reg.Gather()
+		if err != nil {
+			if opts.ErrorLog != nil {
+				opts.ErrorLog.Println("error gathering metrics:", err)
+			}
+			errCnt.WithLabelValues("gathering").Inc()
+			switch opts.ErrorHandling {
+			case PanicOnError:
+				panic(err)
+			case ContinueOnError:
+				if len(mfs) == 0 {
+					// Still report the error if no metrics have been gathered.
+					httpError(rsp, err)
+					return
+				}
+			case HTTPErrorOnError:
+				httpError(rsp, err)
+				return
+			}
+		}
+
+		contentType := expfmt.Negotiate(req.Header)
+		header := rsp.Header()
+		header.Set(contentTypeHeader, string(contentType))
+
+		w := io.Writer(rsp)
+		if !opts.DisableCompression && gzipAccepted(req.Header) {
+			header.Set(contentEncodingHeader, "gzip")
+			gz := gzipPool.Get().(*gzip.Writer)
+			defer gzipPool.Put(gz)
+
+			gz.Reset(w)
+			defer gz.Close()
+
+			w = gz
+		}
+
+		enc := expfmt.NewEncoder(w, contentType)
+
+		var lastErr error
+		for _, mf := range mfs {
+			if err := enc.Encode(mf); err != nil {
+				lastErr = err
+				if opts.ErrorLog != nil {
+					opts.ErrorLog.Println("error encoding and sending metric family:", err)
+				}
+				errCnt.WithLabelValues("encoding").Inc()
+				switch opts.ErrorHandling {
+				case PanicOnError:
+					panic(err)
+				case ContinueOnError:
+					// Handled later.
+				case HTTPErrorOnError:
+					httpError(rsp, err)
+					return
+				}
+			}
+		}
+
+		if lastErr != nil {
+			httpError(rsp, lastErr)
+		}
+	})
+
+	if opts.Timeout <= 0 {
+		return h
+	}
+	return http.TimeoutHandler(h, opts.Timeout, fmt.Sprintf(
+		"Exceeded configured timeout of %v.\n",
+		opts.Timeout,
+	))
+}
+
+// InstrumentMetricHandler is usually used with an http.Handler returned by the
+// HandlerFor function. It instruments the provided http.Handler with two
+// metrics: A counter vector "promhttp_metric_handler_requests_total" to count
+// scrapes partitioned by HTTP status code, and a gauge
+// "promhttp_metric_handler_requests_in_flight" to track the number of
+// simultaneous scrapes. This function idempotently registers collectors for
+// both metrics with the provided Registerer. It panics if the registration
+// fails. The provided metrics are useful to see how many scrapes hit the
+// monitored target (which could be from different Prometheus servers or other
+// scrapers), and how often they overlap (which would result in more than one
+// scrape in flight at the same time). Note that the scrapes-in-flight gauge
+// will contain the scrape by which it is exposed, while the scrape counter will
+// only get incremented after the scrape is complete (as only then the status
+// code is known). For tracking scrape durations, use the
+// "scrape_duration_seconds" gauge created by the Prometheus server upon each
+// scrape.
+func InstrumentMetricHandler(reg prometheus.Registerer, handler http.Handler) http.Handler {
+	cnt := prometheus.NewCounterVec(
+		prometheus.CounterOpts{
+			Name: "promhttp_metric_handler_requests_total",
+			Help: "Total number of scrapes by HTTP status code.",
+		},
+		[]string{"code"},
+	)
+	// Initialize the most likely HTTP status codes.
+	cnt.WithLabelValues("200")
+	cnt.WithLabelValues("500")
+	cnt.WithLabelValues("503")
+	if err := reg.Register(cnt); err != nil {
+		if are, ok := err.(prometheus.AlreadyRegisteredError); ok {
+			cnt = are.ExistingCollector.(*prometheus.CounterVec)
+		} else {
+			panic(err)
+		}
+	}
+
+	gge := prometheus.NewGauge(prometheus.GaugeOpts{
+		Name: "promhttp_metric_handler_requests_in_flight",
+		Help: "Current number of scrapes being served.",
+	})
+	if err := reg.Register(gge); err != nil {
+		if are, ok := err.(prometheus.AlreadyRegisteredError); ok {
+			gge = are.ExistingCollector.(prometheus.Gauge)
+		} else {
+			panic(err)
+		}
+	}
+
+	return InstrumentHandlerCounter(cnt, InstrumentHandlerInFlight(gge, handler))
+}
+
+// HandlerErrorHandling defines how a Handler serving metrics will handle
+// errors.
+type HandlerErrorHandling int
+
+// These constants cause handlers serving metrics to behave as described if
+// errors are encountered.
+const (
+	// Serve an HTTP status code 500 upon the first error
+	// encountered. Report the error message in the body.
+	HTTPErrorOnError HandlerErrorHandling = iota
+	// Ignore errors and try to serve as many metrics as possible.  However,
+	// if no metrics can be served, serve an HTTP status code 500 and the
+	// last error message in the body. Only use this in deliberate "best
+	// effort" metrics collection scenarios. In this case, it is highly
+	// recommended to provide other means of detecting errors: By setting an
+	// ErrorLog in HandlerOpts, the errors are logged. By providing a
+	// Registry in HandlerOpts, the exposed metrics include an error counter
+	// "promhttp_metric_handler_errors_total", which can be used for
+	// alerts.
+	ContinueOnError
+	// Panic upon the first error encountered (useful for "crash only" apps).
+	PanicOnError
+)
+
+// Logger is the minimal interface HandlerOpts needs for logging. Note that
+// log.Logger from the standard library implements this interface, and it is
+// easy to implement by custom loggers, if they don't do so already anyway.
+type Logger interface {
+	Println(v ...interface{})
+}
+
+// HandlerOpts specifies options how to serve metrics via an http.Handler. The
+// zero value of HandlerOpts is a reasonable default.
+type HandlerOpts struct {
+	// ErrorLog specifies an optional logger for errors collecting and
+	// serving metrics. If nil, errors are not logged at all.
+	ErrorLog Logger
+	// ErrorHandling defines how errors are handled. Note that errors are
+	// logged regardless of the configured ErrorHandling provided ErrorLog
+	// is not nil.
+	ErrorHandling HandlerErrorHandling
+	// If Registry is not nil, it is used to register a metric
+	// "promhttp_metric_handler_errors_total", partitioned by "cause". A
+	// failed registration causes a panic. Note that this error counter is
+	// different from the instrumentation you get from the various
+	// InstrumentHandler... helpers. It counts errors that don't necessarily
+	// result in a non-2xx HTTP status code. There are two typical cases:
+	// (1) Encoding errors that only happen after streaming of the HTTP body
+	// has already started (and the status code 200 has been sent). This
+	// should only happen with custom collectors. (2) Collection errors with
+	// no effect on the HTTP status code because ErrorHandling is set to
+	// ContinueOnError.
+	Registry prometheus.Registerer
+	// If DisableCompression is true, the handler will never compress the
+	// response, even if requested by the client.
+	DisableCompression bool
+	// The number of concurrent HTTP requests is limited to
+	// MaxRequestsInFlight. Additional requests are responded to with 503
+	// Service Unavailable and a suitable message in the body. If
+	// MaxRequestsInFlight is 0 or negative, no limit is applied.
+	MaxRequestsInFlight int
+	// If handling a request takes longer than Timeout, it is responded to
+	// with 503 ServiceUnavailable and a suitable Message. No timeout is
+	// applied if Timeout is 0 or negative. Note that with the current
+	// implementation, reaching the timeout simply ends the HTTP requests as
+	// described above (and even that only if sending of the body hasn't
+	// started yet), while the bulk work of gathering all the metrics keeps
+	// running in the background (with the eventual result to be thrown
+	// away). Until the implementation is improved, it is recommended to
+	// implement a separate timeout in potentially slow Collectors.
+	Timeout time.Duration
+}
+
+// gzipAccepted returns whether the client will accept gzip-encoded content.
+func gzipAccepted(header http.Header) bool {
+	a := header.Get(acceptEncodingHeader)
+	parts := strings.Split(a, ",")
+	for _, part := range parts {
+		part = strings.TrimSpace(part)
+		if part == "gzip" || strings.HasPrefix(part, "gzip;") {
+			return true
+		}
+	}
+	return false
+}
+
+// httpError removes any content-encoding header and then calls http.Error with
+// the provided error and http.StatusInternalServerErrer. Error contents is
+// supposed to be uncompressed plain text. However, same as with a plain
+// http.Error, any header settings will be void if the header has already been
+// sent. The error message will still be written to the writer, but it will
+// probably be of limited use.
+func httpError(rsp http.ResponseWriter, err error) {
+	rsp.Header().Del(contentEncodingHeader)
+	http.Error(
+		rsp,
+		"An error has occurred while serving metrics:\n\n"+err.Error(),
+		http.StatusInternalServerError,
+	)
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/promhttp/instrument_client.go b/vendor/github.com/prometheus/client_golang/prometheus/promhttp/instrument_client.go
new file mode 100644
index 0000000..83c49b6
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/promhttp/instrument_client.go
@@ -0,0 +1,219 @@
+// Copyright 2017 The Prometheus 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 promhttp
+
+import (
+	"crypto/tls"
+	"net/http"
+	"net/http/httptrace"
+	"time"
+
+	"github.com/prometheus/client_golang/prometheus"
+)
+
+// The RoundTripperFunc type is an adapter to allow the use of ordinary
+// functions as RoundTrippers. If f is a function with the appropriate
+// signature, RountTripperFunc(f) is a RoundTripper that calls f.
+type RoundTripperFunc func(req *http.Request) (*http.Response, error)
+
+// RoundTrip implements the RoundTripper interface.
+func (rt RoundTripperFunc) RoundTrip(r *http.Request) (*http.Response, error) {
+	return rt(r)
+}
+
+// InstrumentRoundTripperInFlight is a middleware that wraps the provided
+// http.RoundTripper. It sets the provided prometheus.Gauge to the number of
+// requests currently handled by the wrapped http.RoundTripper.
+//
+// See the example for ExampleInstrumentRoundTripperDuration for example usage.
+func InstrumentRoundTripperInFlight(gauge prometheus.Gauge, next http.RoundTripper) RoundTripperFunc {
+	return RoundTripperFunc(func(r *http.Request) (*http.Response, error) {
+		gauge.Inc()
+		defer gauge.Dec()
+		return next.RoundTrip(r)
+	})
+}
+
+// InstrumentRoundTripperCounter is a middleware that wraps the provided
+// http.RoundTripper to observe the request result with the provided CounterVec.
+// The CounterVec must have zero, one, or two non-const non-curried labels. For
+// those, the only allowed label names are "code" and "method". The function
+// panics otherwise. Partitioning of the CounterVec happens by HTTP status code
+// and/or HTTP method if the respective instance label names are present in the
+// CounterVec. For unpartitioned counting, use a CounterVec with zero labels.
+//
+// If the wrapped RoundTripper panics or returns a non-nil error, the Counter
+// is not incremented.
+//
+// See the example for ExampleInstrumentRoundTripperDuration for example usage.
+func InstrumentRoundTripperCounter(counter *prometheus.CounterVec, next http.RoundTripper) RoundTripperFunc {
+	code, method := checkLabels(counter)
+
+	return RoundTripperFunc(func(r *http.Request) (*http.Response, error) {
+		resp, err := next.RoundTrip(r)
+		if err == nil {
+			counter.With(labels(code, method, r.Method, resp.StatusCode)).Inc()
+		}
+		return resp, err
+	})
+}
+
+// InstrumentRoundTripperDuration is a middleware that wraps the provided
+// http.RoundTripper to observe the request duration with the provided
+// ObserverVec.  The ObserverVec must have zero, one, or two non-const
+// non-curried labels. For those, the only allowed label names are "code" and
+// "method". The function panics otherwise. The Observe method of the Observer
+// in the ObserverVec is called with the request duration in
+// seconds. Partitioning happens by HTTP status code and/or HTTP method if the
+// respective instance label names are present in the ObserverVec. For
+// unpartitioned observations, use an ObserverVec with zero labels. Note that
+// partitioning of Histograms is expensive and should be used judiciously.
+//
+// If the wrapped RoundTripper panics or returns a non-nil error, no values are
+// reported.
+//
+// Note that this method is only guaranteed to never observe negative durations
+// if used with Go1.9+.
+func InstrumentRoundTripperDuration(obs prometheus.ObserverVec, next http.RoundTripper) RoundTripperFunc {
+	code, method := checkLabels(obs)
+
+	return RoundTripperFunc(func(r *http.Request) (*http.Response, error) {
+		start := time.Now()
+		resp, err := next.RoundTrip(r)
+		if err == nil {
+			obs.With(labels(code, method, r.Method, resp.StatusCode)).Observe(time.Since(start).Seconds())
+		}
+		return resp, err
+	})
+}
+
+// InstrumentTrace is used to offer flexibility in instrumenting the available
+// httptrace.ClientTrace hook functions. Each function is passed a float64
+// representing the time in seconds since the start of the http request. A user
+// may choose to use separately buckets Histograms, or implement custom
+// instance labels on a per function basis.
+type InstrumentTrace struct {
+	GotConn              func(float64)
+	PutIdleConn          func(float64)
+	GotFirstResponseByte func(float64)
+	Got100Continue       func(float64)
+	DNSStart             func(float64)
+	DNSDone              func(float64)
+	ConnectStart         func(float64)
+	ConnectDone          func(float64)
+	TLSHandshakeStart    func(float64)
+	TLSHandshakeDone     func(float64)
+	WroteHeaders         func(float64)
+	Wait100Continue      func(float64)
+	WroteRequest         func(float64)
+}
+
+// InstrumentRoundTripperTrace is a middleware that wraps the provided
+// RoundTripper and reports times to hook functions provided in the
+// InstrumentTrace struct. Hook functions that are not present in the provided
+// InstrumentTrace struct are ignored. Times reported to the hook functions are
+// time since the start of the request. Only with Go1.9+, those times are
+// guaranteed to never be negative. (Earlier Go versions are not using a
+// monotonic clock.) Note that partitioning of Histograms is expensive and
+// should be used judiciously.
+//
+// For hook functions that receive an error as an argument, no observations are
+// made in the event of a non-nil error value.
+//
+// See the example for ExampleInstrumentRoundTripperDuration for example usage.
+func InstrumentRoundTripperTrace(it *InstrumentTrace, next http.RoundTripper) RoundTripperFunc {
+	return RoundTripperFunc(func(r *http.Request) (*http.Response, error) {
+		start := time.Now()
+
+		trace := &httptrace.ClientTrace{
+			GotConn: func(_ httptrace.GotConnInfo) {
+				if it.GotConn != nil {
+					it.GotConn(time.Since(start).Seconds())
+				}
+			},
+			PutIdleConn: func(err error) {
+				if err != nil {
+					return
+				}
+				if it.PutIdleConn != nil {
+					it.PutIdleConn(time.Since(start).Seconds())
+				}
+			},
+			DNSStart: func(_ httptrace.DNSStartInfo) {
+				if it.DNSStart != nil {
+					it.DNSStart(time.Since(start).Seconds())
+				}
+			},
+			DNSDone: func(_ httptrace.DNSDoneInfo) {
+				if it.DNSDone != nil {
+					it.DNSDone(time.Since(start).Seconds())
+				}
+			},
+			ConnectStart: func(_, _ string) {
+				if it.ConnectStart != nil {
+					it.ConnectStart(time.Since(start).Seconds())
+				}
+			},
+			ConnectDone: func(_, _ string, err error) {
+				if err != nil {
+					return
+				}
+				if it.ConnectDone != nil {
+					it.ConnectDone(time.Since(start).Seconds())
+				}
+			},
+			GotFirstResponseByte: func() {
+				if it.GotFirstResponseByte != nil {
+					it.GotFirstResponseByte(time.Since(start).Seconds())
+				}
+			},
+			Got100Continue: func() {
+				if it.Got100Continue != nil {
+					it.Got100Continue(time.Since(start).Seconds())
+				}
+			},
+			TLSHandshakeStart: func() {
+				if it.TLSHandshakeStart != nil {
+					it.TLSHandshakeStart(time.Since(start).Seconds())
+				}
+			},
+			TLSHandshakeDone: func(_ tls.ConnectionState, err error) {
+				if err != nil {
+					return
+				}
+				if it.TLSHandshakeDone != nil {
+					it.TLSHandshakeDone(time.Since(start).Seconds())
+				}
+			},
+			WroteHeaders: func() {
+				if it.WroteHeaders != nil {
+					it.WroteHeaders(time.Since(start).Seconds())
+				}
+			},
+			Wait100Continue: func() {
+				if it.Wait100Continue != nil {
+					it.Wait100Continue(time.Since(start).Seconds())
+				}
+			},
+			WroteRequest: func(_ httptrace.WroteRequestInfo) {
+				if it.WroteRequest != nil {
+					it.WroteRequest(time.Since(start).Seconds())
+				}
+			},
+		}
+		r = r.WithContext(httptrace.WithClientTrace(r.Context(), trace))
+
+		return next.RoundTrip(r)
+	})
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/promhttp/instrument_server.go b/vendor/github.com/prometheus/client_golang/prometheus/promhttp/instrument_server.go
new file mode 100644
index 0000000..9db2438
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/promhttp/instrument_server.go
@@ -0,0 +1,447 @@
+// Copyright 2017 The Prometheus 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 promhttp
+
+import (
+	"errors"
+	"net/http"
+	"strconv"
+	"strings"
+	"time"
+
+	dto "github.com/prometheus/client_model/go"
+
+	"github.com/prometheus/client_golang/prometheus"
+)
+
+// magicString is used for the hacky label test in checkLabels. Remove once fixed.
+const magicString = "zZgWfBxLqvG8kc8IMv3POi2Bb0tZI3vAnBx+gBaFi9FyPzB/CzKUer1yufDa"
+
+// InstrumentHandlerInFlight is a middleware that wraps the provided
+// http.Handler. It sets the provided prometheus.Gauge to the number of
+// requests currently handled by the wrapped http.Handler.
+//
+// See the example for InstrumentHandlerDuration for example usage.
+func InstrumentHandlerInFlight(g prometheus.Gauge, next http.Handler) http.Handler {
+	return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		g.Inc()
+		defer g.Dec()
+		next.ServeHTTP(w, r)
+	})
+}
+
+// InstrumentHandlerDuration is a middleware that wraps the provided
+// http.Handler to observe the request duration with the provided ObserverVec.
+// The ObserverVec must have zero, one, or two non-const non-curried labels. For
+// those, the only allowed label names are "code" and "method". The function
+// panics otherwise. The Observe method of the Observer in the ObserverVec is
+// called with the request duration in seconds. Partitioning happens by HTTP
+// status code and/or HTTP method if the respective instance label names are
+// present in the ObserverVec. For unpartitioned observations, use an
+// ObserverVec with zero labels. Note that partitioning of Histograms is
+// expensive and should be used judiciously.
+//
+// If the wrapped Handler does not set a status code, a status code of 200 is assumed.
+//
+// If the wrapped Handler panics, no values are reported.
+//
+// Note that this method is only guaranteed to never observe negative durations
+// if used with Go1.9+.
+func InstrumentHandlerDuration(obs prometheus.ObserverVec, next http.Handler) http.HandlerFunc {
+	code, method := checkLabels(obs)
+
+	if code {
+		return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+			now := time.Now()
+			d := newDelegator(w, nil)
+			next.ServeHTTP(d, r)
+
+			obs.With(labels(code, method, r.Method, d.Status())).Observe(time.Since(now).Seconds())
+		})
+	}
+
+	return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		now := time.Now()
+		next.ServeHTTP(w, r)
+		obs.With(labels(code, method, r.Method, 0)).Observe(time.Since(now).Seconds())
+	})
+}
+
+// InstrumentHandlerCounter is a middleware that wraps the provided http.Handler
+// to observe the request result with the provided CounterVec.  The CounterVec
+// must have zero, one, or two non-const non-curried labels. For those, the only
+// allowed label names are "code" and "method". The function panics
+// otherwise. Partitioning of the CounterVec happens by HTTP status code and/or
+// HTTP method if the respective instance label names are present in the
+// CounterVec. For unpartitioned counting, use a CounterVec with zero labels.
+//
+// If the wrapped Handler does not set a status code, a status code of 200 is assumed.
+//
+// If the wrapped Handler panics, the Counter is not incremented.
+//
+// See the example for InstrumentHandlerDuration for example usage.
+func InstrumentHandlerCounter(counter *prometheus.CounterVec, next http.Handler) http.HandlerFunc {
+	code, method := checkLabels(counter)
+
+	if code {
+		return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+			d := newDelegator(w, nil)
+			next.ServeHTTP(d, r)
+			counter.With(labels(code, method, r.Method, d.Status())).Inc()
+		})
+	}
+
+	return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		next.ServeHTTP(w, r)
+		counter.With(labels(code, method, r.Method, 0)).Inc()
+	})
+}
+
+// InstrumentHandlerTimeToWriteHeader is a middleware that wraps the provided
+// http.Handler to observe with the provided ObserverVec the request duration
+// until the response headers are written. The ObserverVec must have zero, one,
+// or two non-const non-curried labels. For those, the only allowed label names
+// are "code" and "method". The function panics otherwise. The Observe method of
+// the Observer in the ObserverVec is called with the request duration in
+// seconds. Partitioning happens by HTTP status code and/or HTTP method if the
+// respective instance label names are present in the ObserverVec. For
+// unpartitioned observations, use an ObserverVec with zero labels. Note that
+// partitioning of Histograms is expensive and should be used judiciously.
+//
+// If the wrapped Handler panics before calling WriteHeader, no value is
+// reported.
+//
+// Note that this method is only guaranteed to never observe negative durations
+// if used with Go1.9+.
+//
+// See the example for InstrumentHandlerDuration for example usage.
+func InstrumentHandlerTimeToWriteHeader(obs prometheus.ObserverVec, next http.Handler) http.HandlerFunc {
+	code, method := checkLabels(obs)
+
+	return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		now := time.Now()
+		d := newDelegator(w, func(status int) {
+			obs.With(labels(code, method, r.Method, status)).Observe(time.Since(now).Seconds())
+		})
+		next.ServeHTTP(d, r)
+	})
+}
+
+// InstrumentHandlerRequestSize is a middleware that wraps the provided
+// http.Handler to observe the request size with the provided ObserverVec.  The
+// ObserverVec must have zero, one, or two non-const non-curried labels. For
+// those, the only allowed label names are "code" and "method". The function
+// panics otherwise. The Observe method of the Observer in the ObserverVec is
+// called with the request size in bytes. Partitioning happens by HTTP status
+// code and/or HTTP method if the respective instance label names are present in
+// the ObserverVec. For unpartitioned observations, use an ObserverVec with zero
+// labels. Note that partitioning of Histograms is expensive and should be used
+// judiciously.
+//
+// If the wrapped Handler does not set a status code, a status code of 200 is assumed.
+//
+// If the wrapped Handler panics, no values are reported.
+//
+// See the example for InstrumentHandlerDuration for example usage.
+func InstrumentHandlerRequestSize(obs prometheus.ObserverVec, next http.Handler) http.HandlerFunc {
+	code, method := checkLabels(obs)
+
+	if code {
+		return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+			d := newDelegator(w, nil)
+			next.ServeHTTP(d, r)
+			size := computeApproximateRequestSize(r)
+			obs.With(labels(code, method, r.Method, d.Status())).Observe(float64(size))
+		})
+	}
+
+	return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		next.ServeHTTP(w, r)
+		size := computeApproximateRequestSize(r)
+		obs.With(labels(code, method, r.Method, 0)).Observe(float64(size))
+	})
+}
+
+// InstrumentHandlerResponseSize is a middleware that wraps the provided
+// http.Handler to observe the response size with the provided ObserverVec.  The
+// ObserverVec must have zero, one, or two non-const non-curried labels. For
+// those, the only allowed label names are "code" and "method". The function
+// panics otherwise. The Observe method of the Observer in the ObserverVec is
+// called with the response size in bytes. Partitioning happens by HTTP status
+// code and/or HTTP method if the respective instance label names are present in
+// the ObserverVec. For unpartitioned observations, use an ObserverVec with zero
+// labels. Note that partitioning of Histograms is expensive and should be used
+// judiciously.
+//
+// If the wrapped Handler does not set a status code, a status code of 200 is assumed.
+//
+// If the wrapped Handler panics, no values are reported.
+//
+// See the example for InstrumentHandlerDuration for example usage.
+func InstrumentHandlerResponseSize(obs prometheus.ObserverVec, next http.Handler) http.Handler {
+	code, method := checkLabels(obs)
+	return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		d := newDelegator(w, nil)
+		next.ServeHTTP(d, r)
+		obs.With(labels(code, method, r.Method, d.Status())).Observe(float64(d.Written()))
+	})
+}
+
+func checkLabels(c prometheus.Collector) (code bool, method bool) {
+	// TODO(beorn7): Remove this hacky way to check for instance labels
+	// once Descriptors can have their dimensionality queried.
+	var (
+		desc *prometheus.Desc
+		m    prometheus.Metric
+		pm   dto.Metric
+		lvs  []string
+	)
+
+	// Get the Desc from the Collector.
+	descc := make(chan *prometheus.Desc, 1)
+	c.Describe(descc)
+
+	select {
+	case desc = <-descc:
+	default:
+		panic("no description provided by collector")
+	}
+	select {
+	case <-descc:
+		panic("more than one description provided by collector")
+	default:
+	}
+
+	close(descc)
+
+	// Create a ConstMetric with the Desc. Since we don't know how many
+	// variable labels there are, try for as long as it needs.
+	for err := errors.New("dummy"); err != nil; lvs = append(lvs, magicString) {
+		m, err = prometheus.NewConstMetric(desc, prometheus.UntypedValue, 0, lvs...)
+	}
+
+	// Write out the metric into a proto message and look at the labels.
+	// If the value is not the magicString, it is a constLabel, which doesn't interest us.
+	// If the label is curried, it doesn't interest us.
+	// In all other cases, only "code" or "method" is allowed.
+	if err := m.Write(&pm); err != nil {
+		panic("error checking metric for labels")
+	}
+	for _, label := range pm.Label {
+		name, value := label.GetName(), label.GetValue()
+		if value != magicString || isLabelCurried(c, name) {
+			continue
+		}
+		switch name {
+		case "code":
+			code = true
+		case "method":
+			method = true
+		default:
+			panic("metric partitioned with non-supported labels")
+		}
+	}
+	return
+}
+
+func isLabelCurried(c prometheus.Collector, label string) bool {
+	// This is even hackier than the label test above.
+	// We essentially try to curry again and see if it works.
+	// But for that, we need to type-convert to the two
+	// types we use here, ObserverVec or *CounterVec.
+	switch v := c.(type) {
+	case *prometheus.CounterVec:
+		if _, err := v.CurryWith(prometheus.Labels{label: "dummy"}); err == nil {
+			return false
+		}
+	case prometheus.ObserverVec:
+		if _, err := v.CurryWith(prometheus.Labels{label: "dummy"}); err == nil {
+			return false
+		}
+	default:
+		panic("unsupported metric vec type")
+	}
+	return true
+}
+
+// emptyLabels is a one-time allocation for non-partitioned metrics to avoid
+// unnecessary allocations on each request.
+var emptyLabels = prometheus.Labels{}
+
+func labels(code, method bool, reqMethod string, status int) prometheus.Labels {
+	if !(code || method) {
+		return emptyLabels
+	}
+	labels := prometheus.Labels{}
+
+	if code {
+		labels["code"] = sanitizeCode(status)
+	}
+	if method {
+		labels["method"] = sanitizeMethod(reqMethod)
+	}
+
+	return labels
+}
+
+func computeApproximateRequestSize(r *http.Request) int {
+	s := 0
+	if r.URL != nil {
+		s += len(r.URL.String())
+	}
+
+	s += len(r.Method)
+	s += len(r.Proto)
+	for name, values := range r.Header {
+		s += len(name)
+		for _, value := range values {
+			s += len(value)
+		}
+	}
+	s += len(r.Host)
+
+	// N.B. r.Form and r.MultipartForm are assumed to be included in r.URL.
+
+	if r.ContentLength != -1 {
+		s += int(r.ContentLength)
+	}
+	return s
+}
+
+func sanitizeMethod(m string) string {
+	switch m {
+	case "GET", "get":
+		return "get"
+	case "PUT", "put":
+		return "put"
+	case "HEAD", "head":
+		return "head"
+	case "POST", "post":
+		return "post"
+	case "DELETE", "delete":
+		return "delete"
+	case "CONNECT", "connect":
+		return "connect"
+	case "OPTIONS", "options":
+		return "options"
+	case "NOTIFY", "notify":
+		return "notify"
+	default:
+		return strings.ToLower(m)
+	}
+}
+
+// If the wrapped http.Handler has not set a status code, i.e. the value is
+// currently 0, santizeCode will return 200, for consistency with behavior in
+// the stdlib.
+func sanitizeCode(s int) string {
+	switch s {
+	case 100:
+		return "100"
+	case 101:
+		return "101"
+
+	case 200, 0:
+		return "200"
+	case 201:
+		return "201"
+	case 202:
+		return "202"
+	case 203:
+		return "203"
+	case 204:
+		return "204"
+	case 205:
+		return "205"
+	case 206:
+		return "206"
+
+	case 300:
+		return "300"
+	case 301:
+		return "301"
+	case 302:
+		return "302"
+	case 304:
+		return "304"
+	case 305:
+		return "305"
+	case 307:
+		return "307"
+
+	case 400:
+		return "400"
+	case 401:
+		return "401"
+	case 402:
+		return "402"
+	case 403:
+		return "403"
+	case 404:
+		return "404"
+	case 405:
+		return "405"
+	case 406:
+		return "406"
+	case 407:
+		return "407"
+	case 408:
+		return "408"
+	case 409:
+		return "409"
+	case 410:
+		return "410"
+	case 411:
+		return "411"
+	case 412:
+		return "412"
+	case 413:
+		return "413"
+	case 414:
+		return "414"
+	case 415:
+		return "415"
+	case 416:
+		return "416"
+	case 417:
+		return "417"
+	case 418:
+		return "418"
+
+	case 500:
+		return "500"
+	case 501:
+		return "501"
+	case 502:
+		return "502"
+	case 503:
+		return "503"
+	case 504:
+		return "504"
+	case 505:
+		return "505"
+
+	case 428:
+		return "428"
+	case 429:
+		return "429"
+	case 431:
+		return "431"
+	case 511:
+		return "511"
+
+	default:
+		return strconv.Itoa(s)
+	}
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/registry.go b/vendor/github.com/prometheus/client_golang/prometheus/registry.go
new file mode 100644
index 0000000..6c32516
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/registry.go
@@ -0,0 +1,945 @@
+// Copyright 2014 The Prometheus 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 prometheus
+
+import (
+	"bytes"
+	"fmt"
+	"io/ioutil"
+	"os"
+	"path/filepath"
+	"runtime"
+	"sort"
+	"strings"
+	"sync"
+	"unicode/utf8"
+
+	"github.com/golang/protobuf/proto"
+	"github.com/prometheus/common/expfmt"
+
+	dto "github.com/prometheus/client_model/go"
+
+	"github.com/prometheus/client_golang/prometheus/internal"
+)
+
+const (
+	// Capacity for the channel to collect metrics and descriptors.
+	capMetricChan = 1000
+	capDescChan   = 10
+)
+
+// DefaultRegisterer and DefaultGatherer are the implementations of the
+// Registerer and Gatherer interface a number of convenience functions in this
+// package act on. Initially, both variables point to the same Registry, which
+// has a process collector (currently on Linux only, see NewProcessCollector)
+// and a Go collector (see NewGoCollector, in particular the note about
+// stop-the-world implication with Go versions older than 1.9) already
+// registered. This approach to keep default instances as global state mirrors
+// the approach of other packages in the Go standard library. Note that there
+// are caveats. Change the variables with caution and only if you understand the
+// consequences. Users who want to avoid global state altogether should not use
+// the convenience functions and act on custom instances instead.
+var (
+	defaultRegistry              = NewRegistry()
+	DefaultRegisterer Registerer = defaultRegistry
+	DefaultGatherer   Gatherer   = defaultRegistry
+)
+
+func init() {
+	MustRegister(NewProcessCollector(ProcessCollectorOpts{}))
+	MustRegister(NewGoCollector())
+}
+
+// NewRegistry creates a new vanilla Registry without any Collectors
+// pre-registered.
+func NewRegistry() *Registry {
+	return &Registry{
+		collectorsByID:  map[uint64]Collector{},
+		descIDs:         map[uint64]struct{}{},
+		dimHashesByName: map[string]uint64{},
+	}
+}
+
+// NewPedanticRegistry returns a registry that checks during collection if each
+// collected Metric is consistent with its reported Desc, and if the Desc has
+// actually been registered with the registry. Unchecked Collectors (those whose
+// Describe methed does not yield any descriptors) are excluded from the check.
+//
+// Usually, a Registry will be happy as long as the union of all collected
+// Metrics is consistent and valid even if some metrics are not consistent with
+// their own Desc or a Desc provided by their registered Collector. Well-behaved
+// Collectors and Metrics will only provide consistent Descs. This Registry is
+// useful to test the implementation of Collectors and Metrics.
+func NewPedanticRegistry() *Registry {
+	r := NewRegistry()
+	r.pedanticChecksEnabled = true
+	return r
+}
+
+// Registerer is the interface for the part of a registry in charge of
+// registering and unregistering. Users of custom registries should use
+// Registerer as type for registration purposes (rather than the Registry type
+// directly). In that way, they are free to use custom Registerer implementation
+// (e.g. for testing purposes).
+type Registerer interface {
+	// Register registers a new Collector to be included in metrics
+	// collection. It returns an error if the descriptors provided by the
+	// Collector are invalid or if they — in combination with descriptors of
+	// already registered Collectors — do not fulfill the consistency and
+	// uniqueness criteria described in the documentation of metric.Desc.
+	//
+	// If the provided Collector is equal to a Collector already registered
+	// (which includes the case of re-registering the same Collector), the
+	// returned error is an instance of AlreadyRegisteredError, which
+	// contains the previously registered Collector.
+	//
+	// A Collector whose Describe method does not yield any Desc is treated
+	// as unchecked. Registration will always succeed. No check for
+	// re-registering (see previous paragraph) is performed. Thus, the
+	// caller is responsible for not double-registering the same unchecked
+	// Collector, and for providing a Collector that will not cause
+	// inconsistent metrics on collection. (This would lead to scrape
+	// errors.)
+	Register(Collector) error
+	// MustRegister works like Register but registers any number of
+	// Collectors and panics upon the first registration that causes an
+	// error.
+	MustRegister(...Collector)
+	// Unregister unregisters the Collector that equals the Collector passed
+	// in as an argument.  (Two Collectors are considered equal if their
+	// Describe method yields the same set of descriptors.) The function
+	// returns whether a Collector was unregistered. Note that an unchecked
+	// Collector cannot be unregistered (as its Describe method does not
+	// yield any descriptor).
+	//
+	// Note that even after unregistering, it will not be possible to
+	// register a new Collector that is inconsistent with the unregistered
+	// Collector, e.g. a Collector collecting metrics with the same name but
+	// a different help string. The rationale here is that the same registry
+	// instance must only collect consistent metrics throughout its
+	// lifetime.
+	Unregister(Collector) bool
+}
+
+// Gatherer is the interface for the part of a registry in charge of gathering
+// the collected metrics into a number of MetricFamilies. The Gatherer interface
+// comes with the same general implication as described for the Registerer
+// interface.
+type Gatherer interface {
+	// Gather calls the Collect method of the registered Collectors and then
+	// gathers the collected metrics into a lexicographically sorted slice
+	// of uniquely named MetricFamily protobufs. Gather ensures that the
+	// returned slice is valid and self-consistent so that it can be used
+	// for valid exposition. As an exception to the strict consistency
+	// requirements described for metric.Desc, Gather will tolerate
+	// different sets of label names for metrics of the same metric family.
+	//
+	// Even if an error occurs, Gather attempts to gather as many metrics as
+	// possible. Hence, if a non-nil error is returned, the returned
+	// MetricFamily slice could be nil (in case of a fatal error that
+	// prevented any meaningful metric collection) or contain a number of
+	// MetricFamily protobufs, some of which might be incomplete, and some
+	// might be missing altogether. The returned error (which might be a
+	// MultiError) explains the details. Note that this is mostly useful for
+	// debugging purposes. If the gathered protobufs are to be used for
+	// exposition in actual monitoring, it is almost always better to not
+	// expose an incomplete result and instead disregard the returned
+	// MetricFamily protobufs in case the returned error is non-nil.
+	Gather() ([]*dto.MetricFamily, error)
+}
+
+// Register registers the provided Collector with the DefaultRegisterer.
+//
+// Register is a shortcut for DefaultRegisterer.Register(c). See there for more
+// details.
+func Register(c Collector) error {
+	return DefaultRegisterer.Register(c)
+}
+
+// MustRegister registers the provided Collectors with the DefaultRegisterer and
+// panics if any error occurs.
+//
+// MustRegister is a shortcut for DefaultRegisterer.MustRegister(cs...). See
+// there for more details.
+func MustRegister(cs ...Collector) {
+	DefaultRegisterer.MustRegister(cs...)
+}
+
+// Unregister removes the registration of the provided Collector from the
+// DefaultRegisterer.
+//
+// Unregister is a shortcut for DefaultRegisterer.Unregister(c). See there for
+// more details.
+func Unregister(c Collector) bool {
+	return DefaultRegisterer.Unregister(c)
+}
+
+// GathererFunc turns a function into a Gatherer.
+type GathererFunc func() ([]*dto.MetricFamily, error)
+
+// Gather implements Gatherer.
+func (gf GathererFunc) Gather() ([]*dto.MetricFamily, error) {
+	return gf()
+}
+
+// AlreadyRegisteredError is returned by the Register method if the Collector to
+// be registered has already been registered before, or a different Collector
+// that collects the same metrics has been registered before. Registration fails
+// in that case, but you can detect from the kind of error what has
+// happened. The error contains fields for the existing Collector and the
+// (rejected) new Collector that equals the existing one. This can be used to
+// find out if an equal Collector has been registered before and switch over to
+// using the old one, as demonstrated in the example.
+type AlreadyRegisteredError struct {
+	ExistingCollector, NewCollector Collector
+}
+
+func (err AlreadyRegisteredError) Error() string {
+	return "duplicate metrics collector registration attempted"
+}
+
+// MultiError is a slice of errors implementing the error interface. It is used
+// by a Gatherer to report multiple errors during MetricFamily gathering.
+type MultiError []error
+
+func (errs MultiError) Error() string {
+	if len(errs) == 0 {
+		return ""
+	}
+	buf := &bytes.Buffer{}
+	fmt.Fprintf(buf, "%d error(s) occurred:", len(errs))
+	for _, err := range errs {
+		fmt.Fprintf(buf, "\n* %s", err)
+	}
+	return buf.String()
+}
+
+// Append appends the provided error if it is not nil.
+func (errs *MultiError) Append(err error) {
+	if err != nil {
+		*errs = append(*errs, err)
+	}
+}
+
+// MaybeUnwrap returns nil if len(errs) is 0. It returns the first and only
+// contained error as error if len(errs is 1). In all other cases, it returns
+// the MultiError directly. This is helpful for returning a MultiError in a way
+// that only uses the MultiError if needed.
+func (errs MultiError) MaybeUnwrap() error {
+	switch len(errs) {
+	case 0:
+		return nil
+	case 1:
+		return errs[0]
+	default:
+		return errs
+	}
+}
+
+// Registry registers Prometheus collectors, collects their metrics, and gathers
+// them into MetricFamilies for exposition. It implements both Registerer and
+// Gatherer. The zero value is not usable. Create instances with NewRegistry or
+// NewPedanticRegistry.
+type Registry struct {
+	mtx                   sync.RWMutex
+	collectorsByID        map[uint64]Collector // ID is a hash of the descIDs.
+	descIDs               map[uint64]struct{}
+	dimHashesByName       map[string]uint64
+	uncheckedCollectors   []Collector
+	pedanticChecksEnabled bool
+}
+
+// Register implements Registerer.
+func (r *Registry) Register(c Collector) error {
+	var (
+		descChan           = make(chan *Desc, capDescChan)
+		newDescIDs         = map[uint64]struct{}{}
+		newDimHashesByName = map[string]uint64{}
+		collectorID        uint64 // Just a sum of all desc IDs.
+		duplicateDescErr   error
+	)
+	go func() {
+		c.Describe(descChan)
+		close(descChan)
+	}()
+	r.mtx.Lock()
+	defer func() {
+		// Drain channel in case of premature return to not leak a goroutine.
+		for range descChan {
+		}
+		r.mtx.Unlock()
+	}()
+	// Conduct various tests...
+	for desc := range descChan {
+
+		// Is the descriptor valid at all?
+		if desc.err != nil {
+			return fmt.Errorf("descriptor %s is invalid: %s", desc, desc.err)
+		}
+
+		// Is the descID unique?
+		// (In other words: Is the fqName + constLabel combination unique?)
+		if _, exists := r.descIDs[desc.id]; exists {
+			duplicateDescErr = fmt.Errorf("descriptor %s already exists with the same fully-qualified name and const label values", desc)
+		}
+		// If it is not a duplicate desc in this collector, add it to
+		// the collectorID.  (We allow duplicate descs within the same
+		// collector, but their existence must be a no-op.)
+		if _, exists := newDescIDs[desc.id]; !exists {
+			newDescIDs[desc.id] = struct{}{}
+			collectorID += desc.id
+		}
+
+		// Are all the label names and the help string consistent with
+		// previous descriptors of the same name?
+		// First check existing descriptors...
+		if dimHash, exists := r.dimHashesByName[desc.fqName]; exists {
+			if dimHash != desc.dimHash {
+				return fmt.Errorf("a previously registered descriptor with the same fully-qualified name as %s has different label names or a different help string", desc)
+			}
+		} else {
+			// ...then check the new descriptors already seen.
+			if dimHash, exists := newDimHashesByName[desc.fqName]; exists {
+				if dimHash != desc.dimHash {
+					return fmt.Errorf("descriptors reported by collector have inconsistent label names or help strings for the same fully-qualified name, offender is %s", desc)
+				}
+			} else {
+				newDimHashesByName[desc.fqName] = desc.dimHash
+			}
+		}
+	}
+	// A Collector yielding no Desc at all is considered unchecked.
+	if len(newDescIDs) == 0 {
+		r.uncheckedCollectors = append(r.uncheckedCollectors, c)
+		return nil
+	}
+	if existing, exists := r.collectorsByID[collectorID]; exists {
+		switch e := existing.(type) {
+		case *wrappingCollector:
+			return AlreadyRegisteredError{
+				ExistingCollector: e.unwrapRecursively(),
+				NewCollector:      c,
+			}
+		default:
+			return AlreadyRegisteredError{
+				ExistingCollector: e,
+				NewCollector:      c,
+			}
+		}
+	}
+	// If the collectorID is new, but at least one of the descs existed
+	// before, we are in trouble.
+	if duplicateDescErr != nil {
+		return duplicateDescErr
+	}
+
+	// Only after all tests have passed, actually register.
+	r.collectorsByID[collectorID] = c
+	for hash := range newDescIDs {
+		r.descIDs[hash] = struct{}{}
+	}
+	for name, dimHash := range newDimHashesByName {
+		r.dimHashesByName[name] = dimHash
+	}
+	return nil
+}
+
+// Unregister implements Registerer.
+func (r *Registry) Unregister(c Collector) bool {
+	var (
+		descChan    = make(chan *Desc, capDescChan)
+		descIDs     = map[uint64]struct{}{}
+		collectorID uint64 // Just a sum of the desc IDs.
+	)
+	go func() {
+		c.Describe(descChan)
+		close(descChan)
+	}()
+	for desc := range descChan {
+		if _, exists := descIDs[desc.id]; !exists {
+			collectorID += desc.id
+			descIDs[desc.id] = struct{}{}
+		}
+	}
+
+	r.mtx.RLock()
+	if _, exists := r.collectorsByID[collectorID]; !exists {
+		r.mtx.RUnlock()
+		return false
+	}
+	r.mtx.RUnlock()
+
+	r.mtx.Lock()
+	defer r.mtx.Unlock()
+
+	delete(r.collectorsByID, collectorID)
+	for id := range descIDs {
+		delete(r.descIDs, id)
+	}
+	// dimHashesByName is left untouched as those must be consistent
+	// throughout the lifetime of a program.
+	return true
+}
+
+// MustRegister implements Registerer.
+func (r *Registry) MustRegister(cs ...Collector) {
+	for _, c := range cs {
+		if err := r.Register(c); err != nil {
+			panic(err)
+		}
+	}
+}
+
+// Gather implements Gatherer.
+func (r *Registry) Gather() ([]*dto.MetricFamily, error) {
+	var (
+		checkedMetricChan   = make(chan Metric, capMetricChan)
+		uncheckedMetricChan = make(chan Metric, capMetricChan)
+		metricHashes        = map[uint64]struct{}{}
+		wg                  sync.WaitGroup
+		errs                MultiError          // The collected errors to return in the end.
+		registeredDescIDs   map[uint64]struct{} // Only used for pedantic checks
+	)
+
+	r.mtx.RLock()
+	goroutineBudget := len(r.collectorsByID) + len(r.uncheckedCollectors)
+	metricFamiliesByName := make(map[string]*dto.MetricFamily, len(r.dimHashesByName))
+	checkedCollectors := make(chan Collector, len(r.collectorsByID))
+	uncheckedCollectors := make(chan Collector, len(r.uncheckedCollectors))
+	for _, collector := range r.collectorsByID {
+		checkedCollectors <- collector
+	}
+	for _, collector := range r.uncheckedCollectors {
+		uncheckedCollectors <- collector
+	}
+	// In case pedantic checks are enabled, we have to copy the map before
+	// giving up the RLock.
+	if r.pedanticChecksEnabled {
+		registeredDescIDs = make(map[uint64]struct{}, len(r.descIDs))
+		for id := range r.descIDs {
+			registeredDescIDs[id] = struct{}{}
+		}
+	}
+	r.mtx.RUnlock()
+
+	wg.Add(goroutineBudget)
+
+	collectWorker := func() {
+		for {
+			select {
+			case collector := <-checkedCollectors:
+				collector.Collect(checkedMetricChan)
+			case collector := <-uncheckedCollectors:
+				collector.Collect(uncheckedMetricChan)
+			default:
+				return
+			}
+			wg.Done()
+		}
+	}
+
+	// Start the first worker now to make sure at least one is running.
+	go collectWorker()
+	goroutineBudget--
+
+	// Close checkedMetricChan and uncheckedMetricChan once all collectors
+	// are collected.
+	go func() {
+		wg.Wait()
+		close(checkedMetricChan)
+		close(uncheckedMetricChan)
+	}()
+
+	// Drain checkedMetricChan and uncheckedMetricChan in case of premature return.
+	defer func() {
+		if checkedMetricChan != nil {
+			for range checkedMetricChan {
+			}
+		}
+		if uncheckedMetricChan != nil {
+			for range uncheckedMetricChan {
+			}
+		}
+	}()
+
+	// Copy the channel references so we can nil them out later to remove
+	// them from the select statements below.
+	cmc := checkedMetricChan
+	umc := uncheckedMetricChan
+
+	for {
+		select {
+		case metric, ok := <-cmc:
+			if !ok {
+				cmc = nil
+				break
+			}
+			errs.Append(processMetric(
+				metric, metricFamiliesByName,
+				metricHashes,
+				registeredDescIDs,
+			))
+		case metric, ok := <-umc:
+			if !ok {
+				umc = nil
+				break
+			}
+			errs.Append(processMetric(
+				metric, metricFamiliesByName,
+				metricHashes,
+				nil,
+			))
+		default:
+			if goroutineBudget <= 0 || len(checkedCollectors)+len(uncheckedCollectors) == 0 {
+				// All collectors are already being worked on or
+				// we have already as many goroutines started as
+				// there are collectors. Do the same as above,
+				// just without the default.
+				select {
+				case metric, ok := <-cmc:
+					if !ok {
+						cmc = nil
+						break
+					}
+					errs.Append(processMetric(
+						metric, metricFamiliesByName,
+						metricHashes,
+						registeredDescIDs,
+					))
+				case metric, ok := <-umc:
+					if !ok {
+						umc = nil
+						break
+					}
+					errs.Append(processMetric(
+						metric, metricFamiliesByName,
+						metricHashes,
+						nil,
+					))
+				}
+				break
+			}
+			// Start more workers.
+			go collectWorker()
+			goroutineBudget--
+			runtime.Gosched()
+		}
+		// Once both checkedMetricChan and uncheckdMetricChan are closed
+		// and drained, the contraption above will nil out cmc and umc,
+		// and then we can leave the collect loop here.
+		if cmc == nil && umc == nil {
+			break
+		}
+	}
+	return internal.NormalizeMetricFamilies(metricFamiliesByName), errs.MaybeUnwrap()
+}
+
+// WriteToTextfile calls Gather on the provided Gatherer, encodes the result in the
+// Prometheus text format, and writes it to a temporary file. Upon success, the
+// temporary file is renamed to the provided filename.
+//
+// This is intended for use with the textfile collector of the node exporter.
+// Note that the node exporter expects the filename to be suffixed with ".prom".
+func WriteToTextfile(filename string, g Gatherer) error {
+	tmp, err := ioutil.TempFile(filepath.Dir(filename), filepath.Base(filename))
+	if err != nil {
+		return err
+	}
+	defer os.Remove(tmp.Name())
+
+	mfs, err := g.Gather()
+	if err != nil {
+		return err
+	}
+	for _, mf := range mfs {
+		if _, err := expfmt.MetricFamilyToText(tmp, mf); err != nil {
+			return err
+		}
+	}
+	if err := tmp.Close(); err != nil {
+		return err
+	}
+
+	if err := os.Chmod(tmp.Name(), 0644); err != nil {
+		return err
+	}
+	return os.Rename(tmp.Name(), filename)
+}
+
+// processMetric is an internal helper method only used by the Gather method.
+func processMetric(
+	metric Metric,
+	metricFamiliesByName map[string]*dto.MetricFamily,
+	metricHashes map[uint64]struct{},
+	registeredDescIDs map[uint64]struct{},
+) error {
+	desc := metric.Desc()
+	// Wrapped metrics collected by an unchecked Collector can have an
+	// invalid Desc.
+	if desc.err != nil {
+		return desc.err
+	}
+	dtoMetric := &dto.Metric{}
+	if err := metric.Write(dtoMetric); err != nil {
+		return fmt.Errorf("error collecting metric %v: %s", desc, err)
+	}
+	metricFamily, ok := metricFamiliesByName[desc.fqName]
+	if ok { // Existing name.
+		if metricFamily.GetHelp() != desc.help {
+			return fmt.Errorf(
+				"collected metric %s %s has help %q but should have %q",
+				desc.fqName, dtoMetric, desc.help, metricFamily.GetHelp(),
+			)
+		}
+		// TODO(beorn7): Simplify switch once Desc has type.
+		switch metricFamily.GetType() {
+		case dto.MetricType_COUNTER:
+			if dtoMetric.Counter == nil {
+				return fmt.Errorf(
+					"collected metric %s %s should be a Counter",
+					desc.fqName, dtoMetric,
+				)
+			}
+		case dto.MetricType_GAUGE:
+			if dtoMetric.Gauge == nil {
+				return fmt.Errorf(
+					"collected metric %s %s should be a Gauge",
+					desc.fqName, dtoMetric,
+				)
+			}
+		case dto.MetricType_SUMMARY:
+			if dtoMetric.Summary == nil {
+				return fmt.Errorf(
+					"collected metric %s %s should be a Summary",
+					desc.fqName, dtoMetric,
+				)
+			}
+		case dto.MetricType_UNTYPED:
+			if dtoMetric.Untyped == nil {
+				return fmt.Errorf(
+					"collected metric %s %s should be Untyped",
+					desc.fqName, dtoMetric,
+				)
+			}
+		case dto.MetricType_HISTOGRAM:
+			if dtoMetric.Histogram == nil {
+				return fmt.Errorf(
+					"collected metric %s %s should be a Histogram",
+					desc.fqName, dtoMetric,
+				)
+			}
+		default:
+			panic("encountered MetricFamily with invalid type")
+		}
+	} else { // New name.
+		metricFamily = &dto.MetricFamily{}
+		metricFamily.Name = proto.String(desc.fqName)
+		metricFamily.Help = proto.String(desc.help)
+		// TODO(beorn7): Simplify switch once Desc has type.
+		switch {
+		case dtoMetric.Gauge != nil:
+			metricFamily.Type = dto.MetricType_GAUGE.Enum()
+		case dtoMetric.Counter != nil:
+			metricFamily.Type = dto.MetricType_COUNTER.Enum()
+		case dtoMetric.Summary != nil:
+			metricFamily.Type = dto.MetricType_SUMMARY.Enum()
+		case dtoMetric.Untyped != nil:
+			metricFamily.Type = dto.MetricType_UNTYPED.Enum()
+		case dtoMetric.Histogram != nil:
+			metricFamily.Type = dto.MetricType_HISTOGRAM.Enum()
+		default:
+			return fmt.Errorf("empty metric collected: %s", dtoMetric)
+		}
+		if err := checkSuffixCollisions(metricFamily, metricFamiliesByName); err != nil {
+			return err
+		}
+		metricFamiliesByName[desc.fqName] = metricFamily
+	}
+	if err := checkMetricConsistency(metricFamily, dtoMetric, metricHashes); err != nil {
+		return err
+	}
+	if registeredDescIDs != nil {
+		// Is the desc registered at all?
+		if _, exist := registeredDescIDs[desc.id]; !exist {
+			return fmt.Errorf(
+				"collected metric %s %s with unregistered descriptor %s",
+				metricFamily.GetName(), dtoMetric, desc,
+			)
+		}
+		if err := checkDescConsistency(metricFamily, dtoMetric, desc); err != nil {
+			return err
+		}
+	}
+	metricFamily.Metric = append(metricFamily.Metric, dtoMetric)
+	return nil
+}
+
+// Gatherers is a slice of Gatherer instances that implements the Gatherer
+// interface itself. Its Gather method calls Gather on all Gatherers in the
+// slice in order and returns the merged results. Errors returned from the
+// Gather calls are all returned in a flattened MultiError. Duplicate and
+// inconsistent Metrics are skipped (first occurrence in slice order wins) and
+// reported in the returned error.
+//
+// Gatherers can be used to merge the Gather results from multiple
+// Registries. It also provides a way to directly inject existing MetricFamily
+// protobufs into the gathering by creating a custom Gatherer with a Gather
+// method that simply returns the existing MetricFamily protobufs. Note that no
+// registration is involved (in contrast to Collector registration), so
+// obviously registration-time checks cannot happen. Any inconsistencies between
+// the gathered MetricFamilies are reported as errors by the Gather method, and
+// inconsistent Metrics are dropped. Invalid parts of the MetricFamilies
+// (e.g. syntactically invalid metric or label names) will go undetected.
+type Gatherers []Gatherer
+
+// Gather implements Gatherer.
+func (gs Gatherers) Gather() ([]*dto.MetricFamily, error) {
+	var (
+		metricFamiliesByName = map[string]*dto.MetricFamily{}
+		metricHashes         = map[uint64]struct{}{}
+		errs                 MultiError // The collected errors to return in the end.
+	)
+
+	for i, g := range gs {
+		mfs, err := g.Gather()
+		if err != nil {
+			if multiErr, ok := err.(MultiError); ok {
+				for _, err := range multiErr {
+					errs = append(errs, fmt.Errorf("[from Gatherer #%d] %s", i+1, err))
+				}
+			} else {
+				errs = append(errs, fmt.Errorf("[from Gatherer #%d] %s", i+1, err))
+			}
+		}
+		for _, mf := range mfs {
+			existingMF, exists := metricFamiliesByName[mf.GetName()]
+			if exists {
+				if existingMF.GetHelp() != mf.GetHelp() {
+					errs = append(errs, fmt.Errorf(
+						"gathered metric family %s has help %q but should have %q",
+						mf.GetName(), mf.GetHelp(), existingMF.GetHelp(),
+					))
+					continue
+				}
+				if existingMF.GetType() != mf.GetType() {
+					errs = append(errs, fmt.Errorf(
+						"gathered metric family %s has type %s but should have %s",
+						mf.GetName(), mf.GetType(), existingMF.GetType(),
+					))
+					continue
+				}
+			} else {
+				existingMF = &dto.MetricFamily{}
+				existingMF.Name = mf.Name
+				existingMF.Help = mf.Help
+				existingMF.Type = mf.Type
+				if err := checkSuffixCollisions(existingMF, metricFamiliesByName); err != nil {
+					errs = append(errs, err)
+					continue
+				}
+				metricFamiliesByName[mf.GetName()] = existingMF
+			}
+			for _, m := range mf.Metric {
+				if err := checkMetricConsistency(existingMF, m, metricHashes); err != nil {
+					errs = append(errs, err)
+					continue
+				}
+				existingMF.Metric = append(existingMF.Metric, m)
+			}
+		}
+	}
+	return internal.NormalizeMetricFamilies(metricFamiliesByName), errs.MaybeUnwrap()
+}
+
+// checkSuffixCollisions checks for collisions with the “magic” suffixes the
+// Prometheus text format and the internal metric representation of the
+// Prometheus server add while flattening Summaries and Histograms.
+func checkSuffixCollisions(mf *dto.MetricFamily, mfs map[string]*dto.MetricFamily) error {
+	var (
+		newName              = mf.GetName()
+		newType              = mf.GetType()
+		newNameWithoutSuffix = ""
+	)
+	switch {
+	case strings.HasSuffix(newName, "_count"):
+		newNameWithoutSuffix = newName[:len(newName)-6]
+	case strings.HasSuffix(newName, "_sum"):
+		newNameWithoutSuffix = newName[:len(newName)-4]
+	case strings.HasSuffix(newName, "_bucket"):
+		newNameWithoutSuffix = newName[:len(newName)-7]
+	}
+	if newNameWithoutSuffix != "" {
+		if existingMF, ok := mfs[newNameWithoutSuffix]; ok {
+			switch existingMF.GetType() {
+			case dto.MetricType_SUMMARY:
+				if !strings.HasSuffix(newName, "_bucket") {
+					return fmt.Errorf(
+						"collected metric named %q collides with previously collected summary named %q",
+						newName, newNameWithoutSuffix,
+					)
+				}
+			case dto.MetricType_HISTOGRAM:
+				return fmt.Errorf(
+					"collected metric named %q collides with previously collected histogram named %q",
+					newName, newNameWithoutSuffix,
+				)
+			}
+		}
+	}
+	if newType == dto.MetricType_SUMMARY || newType == dto.MetricType_HISTOGRAM {
+		if _, ok := mfs[newName+"_count"]; ok {
+			return fmt.Errorf(
+				"collected histogram or summary named %q collides with previously collected metric named %q",
+				newName, newName+"_count",
+			)
+		}
+		if _, ok := mfs[newName+"_sum"]; ok {
+			return fmt.Errorf(
+				"collected histogram or summary named %q collides with previously collected metric named %q",
+				newName, newName+"_sum",
+			)
+		}
+	}
+	if newType == dto.MetricType_HISTOGRAM {
+		if _, ok := mfs[newName+"_bucket"]; ok {
+			return fmt.Errorf(
+				"collected histogram named %q collides with previously collected metric named %q",
+				newName, newName+"_bucket",
+			)
+		}
+	}
+	return nil
+}
+
+// checkMetricConsistency checks if the provided Metric is consistent with the
+// provided MetricFamily. It also hashes the Metric labels and the MetricFamily
+// name. If the resulting hash is already in the provided metricHashes, an error
+// is returned. If not, it is added to metricHashes.
+func checkMetricConsistency(
+	metricFamily *dto.MetricFamily,
+	dtoMetric *dto.Metric,
+	metricHashes map[uint64]struct{},
+) error {
+	name := metricFamily.GetName()
+
+	// Type consistency with metric family.
+	if metricFamily.GetType() == dto.MetricType_GAUGE && dtoMetric.Gauge == nil ||
+		metricFamily.GetType() == dto.MetricType_COUNTER && dtoMetric.Counter == nil ||
+		metricFamily.GetType() == dto.MetricType_SUMMARY && dtoMetric.Summary == nil ||
+		metricFamily.GetType() == dto.MetricType_HISTOGRAM && dtoMetric.Histogram == nil ||
+		metricFamily.GetType() == dto.MetricType_UNTYPED && dtoMetric.Untyped == nil {
+		return fmt.Errorf(
+			"collected metric %q { %s} is not a %s",
+			name, dtoMetric, metricFamily.GetType(),
+		)
+	}
+
+	previousLabelName := ""
+	for _, labelPair := range dtoMetric.GetLabel() {
+		labelName := labelPair.GetName()
+		if labelName == previousLabelName {
+			return fmt.Errorf(
+				"collected metric %q { %s} has two or more labels with the same name: %s",
+				name, dtoMetric, labelName,
+			)
+		}
+		if !checkLabelName(labelName) {
+			return fmt.Errorf(
+				"collected metric %q { %s} has a label with an invalid name: %s",
+				name, dtoMetric, labelName,
+			)
+		}
+		if dtoMetric.Summary != nil && labelName == quantileLabel {
+			return fmt.Errorf(
+				"collected metric %q { %s} must not have an explicit %q label",
+				name, dtoMetric, quantileLabel,
+			)
+		}
+		if !utf8.ValidString(labelPair.GetValue()) {
+			return fmt.Errorf(
+				"collected metric %q { %s} has a label named %q whose value is not utf8: %#v",
+				name, dtoMetric, labelName, labelPair.GetValue())
+		}
+		previousLabelName = labelName
+	}
+
+	// Is the metric unique (i.e. no other metric with the same name and the same labels)?
+	h := hashNew()
+	h = hashAdd(h, name)
+	h = hashAddByte(h, separatorByte)
+	// Make sure label pairs are sorted. We depend on it for the consistency
+	// check.
+	if !sort.IsSorted(labelPairSorter(dtoMetric.Label)) {
+		// We cannot sort dtoMetric.Label in place as it is immutable by contract.
+		copiedLabels := make([]*dto.LabelPair, len(dtoMetric.Label))
+		copy(copiedLabels, dtoMetric.Label)
+		sort.Sort(labelPairSorter(copiedLabels))
+		dtoMetric.Label = copiedLabels
+	}
+	for _, lp := range dtoMetric.Label {
+		h = hashAdd(h, lp.GetName())
+		h = hashAddByte(h, separatorByte)
+		h = hashAdd(h, lp.GetValue())
+		h = hashAddByte(h, separatorByte)
+	}
+	if _, exists := metricHashes[h]; exists {
+		return fmt.Errorf(
+			"collected metric %q { %s} was collected before with the same name and label values",
+			name, dtoMetric,
+		)
+	}
+	metricHashes[h] = struct{}{}
+	return nil
+}
+
+func checkDescConsistency(
+	metricFamily *dto.MetricFamily,
+	dtoMetric *dto.Metric,
+	desc *Desc,
+) error {
+	// Desc help consistency with metric family help.
+	if metricFamily.GetHelp() != desc.help {
+		return fmt.Errorf(
+			"collected metric %s %s has help %q but should have %q",
+			metricFamily.GetName(), dtoMetric, metricFamily.GetHelp(), desc.help,
+		)
+	}
+
+	// Is the desc consistent with the content of the metric?
+	lpsFromDesc := make([]*dto.LabelPair, len(desc.constLabelPairs), len(dtoMetric.Label))
+	copy(lpsFromDesc, desc.constLabelPairs)
+	for _, l := range desc.variableLabels {
+		lpsFromDesc = append(lpsFromDesc, &dto.LabelPair{
+			Name: proto.String(l),
+		})
+	}
+	if len(lpsFromDesc) != len(dtoMetric.Label) {
+		return fmt.Errorf(
+			"labels in collected metric %s %s are inconsistent with descriptor %s",
+			metricFamily.GetName(), dtoMetric, desc,
+		)
+	}
+	sort.Sort(labelPairSorter(lpsFromDesc))
+	for i, lpFromDesc := range lpsFromDesc {
+		lpFromMetric := dtoMetric.Label[i]
+		if lpFromDesc.GetName() != lpFromMetric.GetName() ||
+			lpFromDesc.Value != nil && lpFromDesc.GetValue() != lpFromMetric.GetValue() {
+			return fmt.Errorf(
+				"labels in collected metric %s %s are inconsistent with descriptor %s",
+				metricFamily.GetName(), dtoMetric, desc,
+			)
+		}
+	}
+	return nil
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/summary.go b/vendor/github.com/prometheus/client_golang/prometheus/summary.go
new file mode 100644
index 0000000..c970fde
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/summary.go
@@ -0,0 +1,736 @@
+// Copyright 2014 The Prometheus 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 prometheus
+
+import (
+	"fmt"
+	"math"
+	"runtime"
+	"sort"
+	"sync"
+	"sync/atomic"
+	"time"
+
+	"github.com/beorn7/perks/quantile"
+	"github.com/golang/protobuf/proto"
+
+	dto "github.com/prometheus/client_model/go"
+)
+
+// quantileLabel is used for the label that defines the quantile in a
+// summary.
+const quantileLabel = "quantile"
+
+// A Summary captures individual observations from an event or sample stream and
+// summarizes them in a manner similar to traditional summary statistics: 1. sum
+// of observations, 2. observation count, 3. rank estimations.
+//
+// A typical use-case is the observation of request latencies. By default, a
+// Summary provides the median, the 90th and the 99th percentile of the latency
+// as rank estimations. However, the default behavior will change in the
+// upcoming v1.0.0 of the library. There will be no rank estimations at all by
+// default. For a sane transition, it is recommended to set the desired rank
+// estimations explicitly.
+//
+// Note that the rank estimations cannot be aggregated in a meaningful way with
+// the Prometheus query language (i.e. you cannot average or add them). If you
+// need aggregatable quantiles (e.g. you want the 99th percentile latency of all
+// queries served across all instances of a service), consider the Histogram
+// metric type. See the Prometheus documentation for more details.
+//
+// To create Summary instances, use NewSummary.
+type Summary interface {
+	Metric
+	Collector
+
+	// Observe adds a single observation to the summary.
+	Observe(float64)
+}
+
+var errQuantileLabelNotAllowed = fmt.Errorf(
+	"%q is not allowed as label name in summaries", quantileLabel,
+)
+
+// Default values for SummaryOpts.
+const (
+	// DefMaxAge is the default duration for which observations stay
+	// relevant.
+	DefMaxAge time.Duration = 10 * time.Minute
+	// DefAgeBuckets is the default number of buckets used to calculate the
+	// age of observations.
+	DefAgeBuckets = 5
+	// DefBufCap is the standard buffer size for collecting Summary observations.
+	DefBufCap = 500
+)
+
+// SummaryOpts bundles the options for creating a Summary metric. It is
+// mandatory to set Name to a non-empty string. While all other fields are
+// optional and can safely be left at their zero value, it is recommended to set
+// a help string and to explicitly set the Objectives field to the desired value
+// as the default value will change in the upcoming v1.0.0 of the library.
+type SummaryOpts struct {
+	// Namespace, Subsystem, and Name are components of the fully-qualified
+	// name of the Summary (created by joining these components with
+	// "_"). Only Name is mandatory, the others merely help structuring the
+	// name. Note that the fully-qualified name of the Summary must be a
+	// valid Prometheus metric name.
+	Namespace string
+	Subsystem string
+	Name      string
+
+	// Help provides information about this Summary.
+	//
+	// Metrics with the same fully-qualified name must have the same Help
+	// string.
+	Help string
+
+	// ConstLabels are used to attach fixed labels to this metric. Metrics
+	// with the same fully-qualified name must have the same label names in
+	// their ConstLabels.
+	//
+	// Due to the way a Summary is represented in the Prometheus text format
+	// and how it is handled by the Prometheus server internally, “quantile”
+	// is an illegal label name. Construction of a Summary or SummaryVec
+	// will panic if this label name is used in ConstLabels.
+	//
+	// ConstLabels are only used rarely. In particular, do not use them to
+	// attach the same labels to all your metrics. Those use cases are
+	// better covered by target labels set by the scraping Prometheus
+	// server, or by one specific metric (e.g. a build_info or a
+	// machine_role metric). See also
+	// https://prometheus.io/docs/instrumenting/writing_exporters/#target-labels,-not-static-scraped-labels
+	ConstLabels Labels
+
+	// Objectives defines the quantile rank estimates with their respective
+	// absolute error. If Objectives[q] = e, then the value reported for q
+	// will be the φ-quantile value for some φ between q-e and q+e.  The
+	// default value is an empty map, resulting in a summary without
+	// quantiles.
+	Objectives map[float64]float64
+
+	// MaxAge defines the duration for which an observation stays relevant
+	// for the summary. Must be positive. The default value is DefMaxAge.
+	MaxAge time.Duration
+
+	// AgeBuckets is the number of buckets used to exclude observations that
+	// are older than MaxAge from the summary. A higher number has a
+	// resource penalty, so only increase it if the higher resolution is
+	// really required. For very high observation rates, you might want to
+	// reduce the number of age buckets. With only one age bucket, you will
+	// effectively see a complete reset of the summary each time MaxAge has
+	// passed. The default value is DefAgeBuckets.
+	AgeBuckets uint32
+
+	// BufCap defines the default sample stream buffer size.  The default
+	// value of DefBufCap should suffice for most uses. If there is a need
+	// to increase the value, a multiple of 500 is recommended (because that
+	// is the internal buffer size of the underlying package
+	// "github.com/bmizerany/perks/quantile").
+	BufCap uint32
+}
+
+// Problem with the sliding-window decay algorithm... The Merge method of
+// perk/quantile is actually not working as advertised - and it might be
+// unfixable, as the underlying algorithm is apparently not capable of merging
+// summaries in the first place. To avoid using Merge, we are currently adding
+// observations to _each_ age bucket, i.e. the effort to add a sample is
+// essentially multiplied by the number of age buckets. When rotating age
+// buckets, we empty the previous head stream. On scrape time, we simply take
+// the quantiles from the head stream (no merging required). Result: More effort
+// on observation time, less effort on scrape time, which is exactly the
+// opposite of what we try to accomplish, but at least the results are correct.
+//
+// The quite elegant previous contraption to merge the age buckets efficiently
+// on scrape time (see code up commit 6b9530d72ea715f0ba612c0120e6e09fbf1d49d0)
+// can't be used anymore.
+
+// NewSummary creates a new Summary based on the provided SummaryOpts.
+func NewSummary(opts SummaryOpts) Summary {
+	return newSummary(
+		NewDesc(
+			BuildFQName(opts.Namespace, opts.Subsystem, opts.Name),
+			opts.Help,
+			nil,
+			opts.ConstLabels,
+		),
+		opts,
+	)
+}
+
+func newSummary(desc *Desc, opts SummaryOpts, labelValues ...string) Summary {
+	if len(desc.variableLabels) != len(labelValues) {
+		panic(makeInconsistentCardinalityError(desc.fqName, desc.variableLabels, labelValues))
+	}
+
+	for _, n := range desc.variableLabels {
+		if n == quantileLabel {
+			panic(errQuantileLabelNotAllowed)
+		}
+	}
+	for _, lp := range desc.constLabelPairs {
+		if lp.GetName() == quantileLabel {
+			panic(errQuantileLabelNotAllowed)
+		}
+	}
+
+	if opts.Objectives == nil {
+		opts.Objectives = map[float64]float64{}
+	}
+
+	if opts.MaxAge < 0 {
+		panic(fmt.Errorf("illegal max age MaxAge=%v", opts.MaxAge))
+	}
+	if opts.MaxAge == 0 {
+		opts.MaxAge = DefMaxAge
+	}
+
+	if opts.AgeBuckets == 0 {
+		opts.AgeBuckets = DefAgeBuckets
+	}
+
+	if opts.BufCap == 0 {
+		opts.BufCap = DefBufCap
+	}
+
+	if len(opts.Objectives) == 0 {
+		// Use the lock-free implementation of a Summary without objectives.
+		s := &noObjectivesSummary{
+			desc:       desc,
+			labelPairs: makeLabelPairs(desc, labelValues),
+			counts:     [2]*summaryCounts{&summaryCounts{}, &summaryCounts{}},
+		}
+		s.init(s) // Init self-collection.
+		return s
+	}
+
+	s := &summary{
+		desc: desc,
+
+		objectives:       opts.Objectives,
+		sortedObjectives: make([]float64, 0, len(opts.Objectives)),
+
+		labelPairs: makeLabelPairs(desc, labelValues),
+
+		hotBuf:         make([]float64, 0, opts.BufCap),
+		coldBuf:        make([]float64, 0, opts.BufCap),
+		streamDuration: opts.MaxAge / time.Duration(opts.AgeBuckets),
+	}
+	s.headStreamExpTime = time.Now().Add(s.streamDuration)
+	s.hotBufExpTime = s.headStreamExpTime
+
+	for i := uint32(0); i < opts.AgeBuckets; i++ {
+		s.streams = append(s.streams, s.newStream())
+	}
+	s.headStream = s.streams[0]
+
+	for qu := range s.objectives {
+		s.sortedObjectives = append(s.sortedObjectives, qu)
+	}
+	sort.Float64s(s.sortedObjectives)
+
+	s.init(s) // Init self-collection.
+	return s
+}
+
+type summary struct {
+	selfCollector
+
+	bufMtx sync.Mutex // Protects hotBuf and hotBufExpTime.
+	mtx    sync.Mutex // Protects every other moving part.
+	// Lock bufMtx before mtx if both are needed.
+
+	desc *Desc
+
+	objectives       map[float64]float64
+	sortedObjectives []float64
+
+	labelPairs []*dto.LabelPair
+
+	sum float64
+	cnt uint64
+
+	hotBuf, coldBuf []float64
+
+	streams                          []*quantile.Stream
+	streamDuration                   time.Duration
+	headStream                       *quantile.Stream
+	headStreamIdx                    int
+	headStreamExpTime, hotBufExpTime time.Time
+}
+
+func (s *summary) Desc() *Desc {
+	return s.desc
+}
+
+func (s *summary) Observe(v float64) {
+	s.bufMtx.Lock()
+	defer s.bufMtx.Unlock()
+
+	now := time.Now()
+	if now.After(s.hotBufExpTime) {
+		s.asyncFlush(now)
+	}
+	s.hotBuf = append(s.hotBuf, v)
+	if len(s.hotBuf) == cap(s.hotBuf) {
+		s.asyncFlush(now)
+	}
+}
+
+func (s *summary) Write(out *dto.Metric) error {
+	sum := &dto.Summary{}
+	qs := make([]*dto.Quantile, 0, len(s.objectives))
+
+	s.bufMtx.Lock()
+	s.mtx.Lock()
+	// Swap bufs even if hotBuf is empty to set new hotBufExpTime.
+	s.swapBufs(time.Now())
+	s.bufMtx.Unlock()
+
+	s.flushColdBuf()
+	sum.SampleCount = proto.Uint64(s.cnt)
+	sum.SampleSum = proto.Float64(s.sum)
+
+	for _, rank := range s.sortedObjectives {
+		var q float64
+		if s.headStream.Count() == 0 {
+			q = math.NaN()
+		} else {
+			q = s.headStream.Query(rank)
+		}
+		qs = append(qs, &dto.Quantile{
+			Quantile: proto.Float64(rank),
+			Value:    proto.Float64(q),
+		})
+	}
+
+	s.mtx.Unlock()
+
+	if len(qs) > 0 {
+		sort.Sort(quantSort(qs))
+	}
+	sum.Quantile = qs
+
+	out.Summary = sum
+	out.Label = s.labelPairs
+	return nil
+}
+
+func (s *summary) newStream() *quantile.Stream {
+	return quantile.NewTargeted(s.objectives)
+}
+
+// asyncFlush needs bufMtx locked.
+func (s *summary) asyncFlush(now time.Time) {
+	s.mtx.Lock()
+	s.swapBufs(now)
+
+	// Unblock the original goroutine that was responsible for the mutation
+	// that triggered the compaction.  But hold onto the global non-buffer
+	// state mutex until the operation finishes.
+	go func() {
+		s.flushColdBuf()
+		s.mtx.Unlock()
+	}()
+}
+
+// rotateStreams needs mtx AND bufMtx locked.
+func (s *summary) maybeRotateStreams() {
+	for !s.hotBufExpTime.Equal(s.headStreamExpTime) {
+		s.headStream.Reset()
+		s.headStreamIdx++
+		if s.headStreamIdx >= len(s.streams) {
+			s.headStreamIdx = 0
+		}
+		s.headStream = s.streams[s.headStreamIdx]
+		s.headStreamExpTime = s.headStreamExpTime.Add(s.streamDuration)
+	}
+}
+
+// flushColdBuf needs mtx locked.
+func (s *summary) flushColdBuf() {
+	for _, v := range s.coldBuf {
+		for _, stream := range s.streams {
+			stream.Insert(v)
+		}
+		s.cnt++
+		s.sum += v
+	}
+	s.coldBuf = s.coldBuf[0:0]
+	s.maybeRotateStreams()
+}
+
+// swapBufs needs mtx AND bufMtx locked, coldBuf must be empty.
+func (s *summary) swapBufs(now time.Time) {
+	if len(s.coldBuf) != 0 {
+		panic("coldBuf is not empty")
+	}
+	s.hotBuf, s.coldBuf = s.coldBuf, s.hotBuf
+	// hotBuf is now empty and gets new expiration set.
+	for now.After(s.hotBufExpTime) {
+		s.hotBufExpTime = s.hotBufExpTime.Add(s.streamDuration)
+	}
+}
+
+type summaryCounts struct {
+	// sumBits contains the bits of the float64 representing the sum of all
+	// observations. sumBits and count have to go first in the struct to
+	// guarantee alignment for atomic operations.
+	// http://golang.org/pkg/sync/atomic/#pkg-note-BUG
+	sumBits uint64
+	count   uint64
+}
+
+type noObjectivesSummary struct {
+	// countAndHotIdx enables lock-free writes with use of atomic updates.
+	// The most significant bit is the hot index [0 or 1] of the count field
+	// below. Observe calls update the hot one. All remaining bits count the
+	// number of Observe calls. Observe starts by incrementing this counter,
+	// and finish by incrementing the count field in the respective
+	// summaryCounts, as a marker for completion.
+	//
+	// Calls of the Write method (which are non-mutating reads from the
+	// perspective of the summary) swap the hot–cold under the writeMtx
+	// lock. A cooldown is awaited (while locked) by comparing the number of
+	// observations with the initiation count. Once they match, then the
+	// last observation on the now cool one has completed. All cool fields must
+	// be merged into the new hot before releasing writeMtx.
+
+	// Fields with atomic access first! See alignment constraint:
+	// http://golang.org/pkg/sync/atomic/#pkg-note-BUG
+	countAndHotIdx uint64
+
+	selfCollector
+	desc     *Desc
+	writeMtx sync.Mutex // Only used in the Write method.
+
+	// Two counts, one is "hot" for lock-free observations, the other is
+	// "cold" for writing out a dto.Metric. It has to be an array of
+	// pointers to guarantee 64bit alignment of the histogramCounts, see
+	// http://golang.org/pkg/sync/atomic/#pkg-note-BUG.
+	counts [2]*summaryCounts
+
+	labelPairs []*dto.LabelPair
+}
+
+func (s *noObjectivesSummary) Desc() *Desc {
+	return s.desc
+}
+
+func (s *noObjectivesSummary) Observe(v float64) {
+	// We increment h.countAndHotIdx so that the counter in the lower
+	// 63 bits gets incremented. At the same time, we get the new value
+	// back, which we can use to find the currently-hot counts.
+	n := atomic.AddUint64(&s.countAndHotIdx, 1)
+	hotCounts := s.counts[n>>63]
+
+	for {
+		oldBits := atomic.LoadUint64(&hotCounts.sumBits)
+		newBits := math.Float64bits(math.Float64frombits(oldBits) + v)
+		if atomic.CompareAndSwapUint64(&hotCounts.sumBits, oldBits, newBits) {
+			break
+		}
+	}
+	// Increment count last as we take it as a signal that the observation
+	// is complete.
+	atomic.AddUint64(&hotCounts.count, 1)
+}
+
+func (s *noObjectivesSummary) Write(out *dto.Metric) error {
+	// For simplicity, we protect this whole method by a mutex. It is not in
+	// the hot path, i.e. Observe is called much more often than Write. The
+	// complication of making Write lock-free isn't worth it, if possible at
+	// all.
+	s.writeMtx.Lock()
+	defer s.writeMtx.Unlock()
+
+	// Adding 1<<63 switches the hot index (from 0 to 1 or from 1 to 0)
+	// without touching the count bits. See the struct comments for a full
+	// description of the algorithm.
+	n := atomic.AddUint64(&s.countAndHotIdx, 1<<63)
+	// count is contained unchanged in the lower 63 bits.
+	count := n & ((1 << 63) - 1)
+	// The most significant bit tells us which counts is hot. The complement
+	// is thus the cold one.
+	hotCounts := s.counts[n>>63]
+	coldCounts := s.counts[(^n)>>63]
+
+	// Await cooldown.
+	for count != atomic.LoadUint64(&coldCounts.count) {
+		runtime.Gosched() // Let observations get work done.
+	}
+
+	sum := &dto.Summary{
+		SampleCount: proto.Uint64(count),
+		SampleSum:   proto.Float64(math.Float64frombits(atomic.LoadUint64(&coldCounts.sumBits))),
+	}
+
+	out.Summary = sum
+	out.Label = s.labelPairs
+
+	// Finally add all the cold counts to the new hot counts and reset the cold counts.
+	atomic.AddUint64(&hotCounts.count, count)
+	atomic.StoreUint64(&coldCounts.count, 0)
+	for {
+		oldBits := atomic.LoadUint64(&hotCounts.sumBits)
+		newBits := math.Float64bits(math.Float64frombits(oldBits) + sum.GetSampleSum())
+		if atomic.CompareAndSwapUint64(&hotCounts.sumBits, oldBits, newBits) {
+			atomic.StoreUint64(&coldCounts.sumBits, 0)
+			break
+		}
+	}
+	return nil
+}
+
+type quantSort []*dto.Quantile
+
+func (s quantSort) Len() int {
+	return len(s)
+}
+
+func (s quantSort) Swap(i, j int) {
+	s[i], s[j] = s[j], s[i]
+}
+
+func (s quantSort) Less(i, j int) bool {
+	return s[i].GetQuantile() < s[j].GetQuantile()
+}
+
+// SummaryVec is a Collector that bundles a set of Summaries that all share the
+// same Desc, but have different values for their variable labels. This is used
+// if you want to count the same thing partitioned by various dimensions
+// (e.g. HTTP request latencies, partitioned by status code and method). Create
+// instances with NewSummaryVec.
+type SummaryVec struct {
+	*metricVec
+}
+
+// NewSummaryVec creates a new SummaryVec based on the provided SummaryOpts and
+// partitioned by the given label names.
+//
+// Due to the way a Summary is represented in the Prometheus text format and how
+// it is handled by the Prometheus server internally, “quantile” is an illegal
+// label name. NewSummaryVec will panic if this label name is used.
+func NewSummaryVec(opts SummaryOpts, labelNames []string) *SummaryVec {
+	for _, ln := range labelNames {
+		if ln == quantileLabel {
+			panic(errQuantileLabelNotAllowed)
+		}
+	}
+	desc := NewDesc(
+		BuildFQName(opts.Namespace, opts.Subsystem, opts.Name),
+		opts.Help,
+		labelNames,
+		opts.ConstLabels,
+	)
+	return &SummaryVec{
+		metricVec: newMetricVec(desc, func(lvs ...string) Metric {
+			return newSummary(desc, opts, lvs...)
+		}),
+	}
+}
+
+// GetMetricWithLabelValues returns the Summary for the given slice of label
+// values (same order as the VariableLabels in Desc). If that combination of
+// label values is accessed for the first time, a new Summary is created.
+//
+// It is possible to call this method without using the returned Summary to only
+// create the new Summary but leave it at its starting value, a Summary without
+// any observations.
+//
+// Keeping the Summary for later use is possible (and should be considered if
+// performance is critical), but keep in mind that Reset, DeleteLabelValues and
+// Delete can be used to delete the Summary from the SummaryVec. In that case,
+// the Summary will still exist, but it will not be exported anymore, even if a
+// Summary with the same label values is created later. See also the CounterVec
+// example.
+//
+// An error is returned if the number of label values is not the same as the
+// number of VariableLabels in Desc (minus any curried labels).
+//
+// Note that for more than one label value, this method is prone to mistakes
+// caused by an incorrect order of arguments. Consider GetMetricWith(Labels) as
+// an alternative to avoid that type of mistake. For higher label numbers, the
+// latter has a much more readable (albeit more verbose) syntax, but it comes
+// with a performance overhead (for creating and processing the Labels map).
+// See also the GaugeVec example.
+func (v *SummaryVec) GetMetricWithLabelValues(lvs ...string) (Observer, error) {
+	metric, err := v.metricVec.getMetricWithLabelValues(lvs...)
+	if metric != nil {
+		return metric.(Observer), err
+	}
+	return nil, err
+}
+
+// GetMetricWith returns the Summary for the given Labels map (the label names
+// must match those of the VariableLabels in Desc). If that label map is
+// accessed for the first time, a new Summary is created. Implications of
+// creating a Summary without using it and keeping the Summary for later use are
+// the same as for GetMetricWithLabelValues.
+//
+// An error is returned if the number and names of the Labels are inconsistent
+// with those of the VariableLabels in Desc (minus any curried labels).
+//
+// This method is used for the same purpose as
+// GetMetricWithLabelValues(...string). See there for pros and cons of the two
+// methods.
+func (v *SummaryVec) GetMetricWith(labels Labels) (Observer, error) {
+	metric, err := v.metricVec.getMetricWith(labels)
+	if metric != nil {
+		return metric.(Observer), err
+	}
+	return nil, err
+}
+
+// WithLabelValues works as GetMetricWithLabelValues, but panics where
+// GetMetricWithLabelValues would have returned an error. Not returning an
+// error allows shortcuts like
+//     myVec.WithLabelValues("404", "GET").Observe(42.21)
+func (v *SummaryVec) WithLabelValues(lvs ...string) Observer {
+	s, err := v.GetMetricWithLabelValues(lvs...)
+	if err != nil {
+		panic(err)
+	}
+	return s
+}
+
+// With works as GetMetricWith, but panics where GetMetricWithLabels would have
+// returned an error. Not returning an error allows shortcuts like
+//     myVec.With(prometheus.Labels{"code": "404", "method": "GET"}).Observe(42.21)
+func (v *SummaryVec) With(labels Labels) Observer {
+	s, err := v.GetMetricWith(labels)
+	if err != nil {
+		panic(err)
+	}
+	return s
+}
+
+// CurryWith returns a vector curried with the provided labels, i.e. the
+// returned vector has those labels pre-set for all labeled operations performed
+// on it. The cardinality of the curried vector is reduced accordingly. The
+// order of the remaining labels stays the same (just with the curried labels
+// taken out of the sequence – which is relevant for the
+// (GetMetric)WithLabelValues methods). It is possible to curry a curried
+// vector, but only with labels not yet used for currying before.
+//
+// The metrics contained in the SummaryVec are shared between the curried and
+// uncurried vectors. They are just accessed differently. Curried and uncurried
+// vectors behave identically in terms of collection. Only one must be
+// registered with a given registry (usually the uncurried version). The Reset
+// method deletes all metrics, even if called on a curried vector.
+func (v *SummaryVec) CurryWith(labels Labels) (ObserverVec, error) {
+	vec, err := v.curryWith(labels)
+	if vec != nil {
+		return &SummaryVec{vec}, err
+	}
+	return nil, err
+}
+
+// MustCurryWith works as CurryWith but panics where CurryWith would have
+// returned an error.
+func (v *SummaryVec) MustCurryWith(labels Labels) ObserverVec {
+	vec, err := v.CurryWith(labels)
+	if err != nil {
+		panic(err)
+	}
+	return vec
+}
+
+type constSummary struct {
+	desc       *Desc
+	count      uint64
+	sum        float64
+	quantiles  map[float64]float64
+	labelPairs []*dto.LabelPair
+}
+
+func (s *constSummary) Desc() *Desc {
+	return s.desc
+}
+
+func (s *constSummary) Write(out *dto.Metric) error {
+	sum := &dto.Summary{}
+	qs := make([]*dto.Quantile, 0, len(s.quantiles))
+
+	sum.SampleCount = proto.Uint64(s.count)
+	sum.SampleSum = proto.Float64(s.sum)
+
+	for rank, q := range s.quantiles {
+		qs = append(qs, &dto.Quantile{
+			Quantile: proto.Float64(rank),
+			Value:    proto.Float64(q),
+		})
+	}
+
+	if len(qs) > 0 {
+		sort.Sort(quantSort(qs))
+	}
+	sum.Quantile = qs
+
+	out.Summary = sum
+	out.Label = s.labelPairs
+
+	return nil
+}
+
+// NewConstSummary returns a metric representing a Prometheus summary with fixed
+// values for the count, sum, and quantiles. As those parameters cannot be
+// changed, the returned value does not implement the Summary interface (but
+// only the Metric interface). Users of this package will not have much use for
+// it in regular operations. However, when implementing custom Collectors, it is
+// useful as a throw-away metric that is generated on the fly to send it to
+// Prometheus in the Collect method.
+//
+// quantiles maps ranks to quantile values. For example, a median latency of
+// 0.23s and a 99th percentile latency of 0.56s would be expressed as:
+//     map[float64]float64{0.5: 0.23, 0.99: 0.56}
+//
+// NewConstSummary returns an error if the length of labelValues is not
+// consistent with the variable labels in Desc or if Desc is invalid.
+func NewConstSummary(
+	desc *Desc,
+	count uint64,
+	sum float64,
+	quantiles map[float64]float64,
+	labelValues ...string,
+) (Metric, error) {
+	if desc.err != nil {
+		return nil, desc.err
+	}
+	if err := validateLabelValues(labelValues, len(desc.variableLabels)); err != nil {
+		return nil, err
+	}
+	return &constSummary{
+		desc:       desc,
+		count:      count,
+		sum:        sum,
+		quantiles:  quantiles,
+		labelPairs: makeLabelPairs(desc, labelValues),
+	}, nil
+}
+
+// MustNewConstSummary is a version of NewConstSummary that panics where
+// NewConstMetric would have returned an error.
+func MustNewConstSummary(
+	desc *Desc,
+	count uint64,
+	sum float64,
+	quantiles map[float64]float64,
+	labelValues ...string,
+) Metric {
+	m, err := NewConstSummary(desc, count, sum, quantiles, labelValues...)
+	if err != nil {
+		panic(err)
+	}
+	return m
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/timer.go b/vendor/github.com/prometheus/client_golang/prometheus/timer.go
new file mode 100644
index 0000000..8d5f105
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/timer.go
@@ -0,0 +1,54 @@
+// Copyright 2016 The Prometheus 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 prometheus
+
+import "time"
+
+// Timer is a helper type to time functions. Use NewTimer to create new
+// instances.
+type Timer struct {
+	begin    time.Time
+	observer Observer
+}
+
+// NewTimer creates a new Timer. The provided Observer is used to observe a
+// duration in seconds. Timer is usually used to time a function call in the
+// following way:
+//    func TimeMe() {
+//        timer := NewTimer(myHistogram)
+//        defer timer.ObserveDuration()
+//        // Do actual work.
+//    }
+func NewTimer(o Observer) *Timer {
+	return &Timer{
+		begin:    time.Now(),
+		observer: o,
+	}
+}
+
+// ObserveDuration records the duration passed since the Timer was created with
+// NewTimer. It calls the Observe method of the Observer provided during
+// construction with the duration in seconds as an argument. The observed
+// duration is also returned. ObserveDuration is usually called with a defer
+// statement.
+//
+// Note that this method is only guaranteed to never observe negative durations
+// if used with Go1.9+.
+func (t *Timer) ObserveDuration() time.Duration {
+	d := time.Since(t.begin)
+	if t.observer != nil {
+		t.observer.Observe(d.Seconds())
+	}
+	return d
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/untyped.go b/vendor/github.com/prometheus/client_golang/prometheus/untyped.go
new file mode 100644
index 0000000..0f9ce63
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/untyped.go
@@ -0,0 +1,42 @@
+// Copyright 2014 The Prometheus 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 prometheus
+
+// UntypedOpts is an alias for Opts. See there for doc comments.
+type UntypedOpts Opts
+
+// UntypedFunc works like GaugeFunc but the collected metric is of type
+// "Untyped". UntypedFunc is useful to mirror an external metric of unknown
+// type.
+//
+// To create UntypedFunc instances, use NewUntypedFunc.
+type UntypedFunc interface {
+	Metric
+	Collector
+}
+
+// NewUntypedFunc creates a new UntypedFunc based on the provided
+// UntypedOpts. The value reported is determined by calling the given function
+// from within the Write method. Take into account that metric collection may
+// happen concurrently. If that results in concurrent calls to Write, like in
+// the case where an UntypedFunc is directly registered with Prometheus, the
+// provided function must be concurrency-safe.
+func NewUntypedFunc(opts UntypedOpts, function func() float64) UntypedFunc {
+	return newValueFunc(NewDesc(
+		BuildFQName(opts.Namespace, opts.Subsystem, opts.Name),
+		opts.Help,
+		nil,
+		opts.ConstLabels,
+	), UntypedValue, function)
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/value.go b/vendor/github.com/prometheus/client_golang/prometheus/value.go
new file mode 100644
index 0000000..eb248f1
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/value.go
@@ -0,0 +1,162 @@
+// Copyright 2014 The Prometheus 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 prometheus
+
+import (
+	"fmt"
+	"sort"
+
+	"github.com/golang/protobuf/proto"
+
+	dto "github.com/prometheus/client_model/go"
+)
+
+// ValueType is an enumeration of metric types that represent a simple value.
+type ValueType int
+
+// Possible values for the ValueType enum.
+const (
+	_ ValueType = iota
+	CounterValue
+	GaugeValue
+	UntypedValue
+)
+
+// valueFunc is a generic metric for simple values retrieved on collect time
+// from a function. It implements Metric and Collector. Its effective type is
+// determined by ValueType. This is a low-level building block used by the
+// library to back the implementations of CounterFunc, GaugeFunc, and
+// UntypedFunc.
+type valueFunc struct {
+	selfCollector
+
+	desc       *Desc
+	valType    ValueType
+	function   func() float64
+	labelPairs []*dto.LabelPair
+}
+
+// newValueFunc returns a newly allocated valueFunc with the given Desc and
+// ValueType. The value reported is determined by calling the given function
+// from within the Write method. Take into account that metric collection may
+// happen concurrently. If that results in concurrent calls to Write, like in
+// the case where a valueFunc is directly registered with Prometheus, the
+// provided function must be concurrency-safe.
+func newValueFunc(desc *Desc, valueType ValueType, function func() float64) *valueFunc {
+	result := &valueFunc{
+		desc:       desc,
+		valType:    valueType,
+		function:   function,
+		labelPairs: makeLabelPairs(desc, nil),
+	}
+	result.init(result)
+	return result
+}
+
+func (v *valueFunc) Desc() *Desc {
+	return v.desc
+}
+
+func (v *valueFunc) Write(out *dto.Metric) error {
+	return populateMetric(v.valType, v.function(), v.labelPairs, out)
+}
+
+// NewConstMetric returns a metric with one fixed value that cannot be
+// changed. Users of this package will not have much use for it in regular
+// operations. However, when implementing custom Collectors, it is useful as a
+// throw-away metric that is generated on the fly to send it to Prometheus in
+// the Collect method. NewConstMetric returns an error if the length of
+// labelValues is not consistent with the variable labels in Desc or if Desc is
+// invalid.
+func NewConstMetric(desc *Desc, valueType ValueType, value float64, labelValues ...string) (Metric, error) {
+	if desc.err != nil {
+		return nil, desc.err
+	}
+	if err := validateLabelValues(labelValues, len(desc.variableLabels)); err != nil {
+		return nil, err
+	}
+	return &constMetric{
+		desc:       desc,
+		valType:    valueType,
+		val:        value,
+		labelPairs: makeLabelPairs(desc, labelValues),
+	}, nil
+}
+
+// MustNewConstMetric is a version of NewConstMetric that panics where
+// NewConstMetric would have returned an error.
+func MustNewConstMetric(desc *Desc, valueType ValueType, value float64, labelValues ...string) Metric {
+	m, err := NewConstMetric(desc, valueType, value, labelValues...)
+	if err != nil {
+		panic(err)
+	}
+	return m
+}
+
+type constMetric struct {
+	desc       *Desc
+	valType    ValueType
+	val        float64
+	labelPairs []*dto.LabelPair
+}
+
+func (m *constMetric) Desc() *Desc {
+	return m.desc
+}
+
+func (m *constMetric) Write(out *dto.Metric) error {
+	return populateMetric(m.valType, m.val, m.labelPairs, out)
+}
+
+func populateMetric(
+	t ValueType,
+	v float64,
+	labelPairs []*dto.LabelPair,
+	m *dto.Metric,
+) error {
+	m.Label = labelPairs
+	switch t {
+	case CounterValue:
+		m.Counter = &dto.Counter{Value: proto.Float64(v)}
+	case GaugeValue:
+		m.Gauge = &dto.Gauge{Value: proto.Float64(v)}
+	case UntypedValue:
+		m.Untyped = &dto.Untyped{Value: proto.Float64(v)}
+	default:
+		return fmt.Errorf("encountered unknown type %v", t)
+	}
+	return nil
+}
+
+func makeLabelPairs(desc *Desc, labelValues []string) []*dto.LabelPair {
+	totalLen := len(desc.variableLabels) + len(desc.constLabelPairs)
+	if totalLen == 0 {
+		// Super fast path.
+		return nil
+	}
+	if len(desc.variableLabels) == 0 {
+		// Moderately fast path.
+		return desc.constLabelPairs
+	}
+	labelPairs := make([]*dto.LabelPair, 0, totalLen)
+	for i, n := range desc.variableLabels {
+		labelPairs = append(labelPairs, &dto.LabelPair{
+			Name:  proto.String(n),
+			Value: proto.String(labelValues[i]),
+		})
+	}
+	labelPairs = append(labelPairs, desc.constLabelPairs...)
+	sort.Sort(labelPairSorter(labelPairs))
+	return labelPairs
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/vec.go b/vendor/github.com/prometheus/client_golang/prometheus/vec.go
new file mode 100644
index 0000000..14ed9e8
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/vec.go
@@ -0,0 +1,472 @@
+// Copyright 2014 The Prometheus 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 prometheus
+
+import (
+	"fmt"
+	"sync"
+
+	"github.com/prometheus/common/model"
+)
+
+// metricVec is a Collector to bundle metrics of the same name that differ in
+// their label values. metricVec is not used directly (and therefore
+// unexported). It is used as a building block for implementations of vectors of
+// a given metric type, like GaugeVec, CounterVec, SummaryVec, and HistogramVec.
+// It also handles label currying. It uses basicMetricVec internally.
+type metricVec struct {
+	*metricMap
+
+	curry []curriedLabelValue
+
+	// hashAdd and hashAddByte can be replaced for testing collision handling.
+	hashAdd     func(h uint64, s string) uint64
+	hashAddByte func(h uint64, b byte) uint64
+}
+
+// newMetricVec returns an initialized metricVec.
+func newMetricVec(desc *Desc, newMetric func(lvs ...string) Metric) *metricVec {
+	return &metricVec{
+		metricMap: &metricMap{
+			metrics:   map[uint64][]metricWithLabelValues{},
+			desc:      desc,
+			newMetric: newMetric,
+		},
+		hashAdd:     hashAdd,
+		hashAddByte: hashAddByte,
+	}
+}
+
+// DeleteLabelValues removes the metric where the variable labels are the same
+// as those passed in as labels (same order as the VariableLabels in Desc). It
+// returns true if a metric was deleted.
+//
+// It is not an error if the number of label values is not the same as the
+// number of VariableLabels in Desc. However, such inconsistent label count can
+// never match an actual metric, so the method will always return false in that
+// case.
+//
+// Note that for more than one label value, this method is prone to mistakes
+// caused by an incorrect order of arguments. Consider Delete(Labels) as an
+// alternative to avoid that type of mistake. For higher label numbers, the
+// latter has a much more readable (albeit more verbose) syntax, but it comes
+// with a performance overhead (for creating and processing the Labels map).
+// See also the CounterVec example.
+func (m *metricVec) DeleteLabelValues(lvs ...string) bool {
+	h, err := m.hashLabelValues(lvs)
+	if err != nil {
+		return false
+	}
+
+	return m.metricMap.deleteByHashWithLabelValues(h, lvs, m.curry)
+}
+
+// Delete deletes the metric where the variable labels are the same as those
+// passed in as labels. It returns true if a metric was deleted.
+//
+// It is not an error if the number and names of the Labels are inconsistent
+// with those of the VariableLabels in Desc. However, such inconsistent Labels
+// can never match an actual metric, so the method will always return false in
+// that case.
+//
+// This method is used for the same purpose as DeleteLabelValues(...string). See
+// there for pros and cons of the two methods.
+func (m *metricVec) Delete(labels Labels) bool {
+	h, err := m.hashLabels(labels)
+	if err != nil {
+		return false
+	}
+
+	return m.metricMap.deleteByHashWithLabels(h, labels, m.curry)
+}
+
+func (m *metricVec) curryWith(labels Labels) (*metricVec, error) {
+	var (
+		newCurry []curriedLabelValue
+		oldCurry = m.curry
+		iCurry   int
+	)
+	for i, label := range m.desc.variableLabels {
+		val, ok := labels[label]
+		if iCurry < len(oldCurry) && oldCurry[iCurry].index == i {
+			if ok {
+				return nil, fmt.Errorf("label name %q is already curried", label)
+			}
+			newCurry = append(newCurry, oldCurry[iCurry])
+			iCurry++
+		} else {
+			if !ok {
+				continue // Label stays uncurried.
+			}
+			newCurry = append(newCurry, curriedLabelValue{i, val})
+		}
+	}
+	if l := len(oldCurry) + len(labels) - len(newCurry); l > 0 {
+		return nil, fmt.Errorf("%d unknown label(s) found during currying", l)
+	}
+
+	return &metricVec{
+		metricMap:   m.metricMap,
+		curry:       newCurry,
+		hashAdd:     m.hashAdd,
+		hashAddByte: m.hashAddByte,
+	}, nil
+}
+
+func (m *metricVec) getMetricWithLabelValues(lvs ...string) (Metric, error) {
+	h, err := m.hashLabelValues(lvs)
+	if err != nil {
+		return nil, err
+	}
+
+	return m.metricMap.getOrCreateMetricWithLabelValues(h, lvs, m.curry), nil
+}
+
+func (m *metricVec) getMetricWith(labels Labels) (Metric, error) {
+	h, err := m.hashLabels(labels)
+	if err != nil {
+		return nil, err
+	}
+
+	return m.metricMap.getOrCreateMetricWithLabels(h, labels, m.curry), nil
+}
+
+func (m *metricVec) hashLabelValues(vals []string) (uint64, error) {
+	if err := validateLabelValues(vals, len(m.desc.variableLabels)-len(m.curry)); err != nil {
+		return 0, err
+	}
+
+	var (
+		h             = hashNew()
+		curry         = m.curry
+		iVals, iCurry int
+	)
+	for i := 0; i < len(m.desc.variableLabels); i++ {
+		if iCurry < len(curry) && curry[iCurry].index == i {
+			h = m.hashAdd(h, curry[iCurry].value)
+			iCurry++
+		} else {
+			h = m.hashAdd(h, vals[iVals])
+			iVals++
+		}
+		h = m.hashAddByte(h, model.SeparatorByte)
+	}
+	return h, nil
+}
+
+func (m *metricVec) hashLabels(labels Labels) (uint64, error) {
+	if err := validateValuesInLabels(labels, len(m.desc.variableLabels)-len(m.curry)); err != nil {
+		return 0, err
+	}
+
+	var (
+		h      = hashNew()
+		curry  = m.curry
+		iCurry int
+	)
+	for i, label := range m.desc.variableLabels {
+		val, ok := labels[label]
+		if iCurry < len(curry) && curry[iCurry].index == i {
+			if ok {
+				return 0, fmt.Errorf("label name %q is already curried", label)
+			}
+			h = m.hashAdd(h, curry[iCurry].value)
+			iCurry++
+		} else {
+			if !ok {
+				return 0, fmt.Errorf("label name %q missing in label map", label)
+			}
+			h = m.hashAdd(h, val)
+		}
+		h = m.hashAddByte(h, model.SeparatorByte)
+	}
+	return h, nil
+}
+
+// metricWithLabelValues provides the metric and its label values for
+// disambiguation on hash collision.
+type metricWithLabelValues struct {
+	values []string
+	metric Metric
+}
+
+// curriedLabelValue sets the curried value for a label at the given index.
+type curriedLabelValue struct {
+	index int
+	value string
+}
+
+// metricMap is a helper for metricVec and shared between differently curried
+// metricVecs.
+type metricMap struct {
+	mtx       sync.RWMutex // Protects metrics.
+	metrics   map[uint64][]metricWithLabelValues
+	desc      *Desc
+	newMetric func(labelValues ...string) Metric
+}
+
+// Describe implements Collector. It will send exactly one Desc to the provided
+// channel.
+func (m *metricMap) Describe(ch chan<- *Desc) {
+	ch <- m.desc
+}
+
+// Collect implements Collector.
+func (m *metricMap) Collect(ch chan<- Metric) {
+	m.mtx.RLock()
+	defer m.mtx.RUnlock()
+
+	for _, metrics := range m.metrics {
+		for _, metric := range metrics {
+			ch <- metric.metric
+		}
+	}
+}
+
+// Reset deletes all metrics in this vector.
+func (m *metricMap) Reset() {
+	m.mtx.Lock()
+	defer m.mtx.Unlock()
+
+	for h := range m.metrics {
+		delete(m.metrics, h)
+	}
+}
+
+// deleteByHashWithLabelValues removes the metric from the hash bucket h. If
+// there are multiple matches in the bucket, use lvs to select a metric and
+// remove only that metric.
+func (m *metricMap) deleteByHashWithLabelValues(
+	h uint64, lvs []string, curry []curriedLabelValue,
+) bool {
+	m.mtx.Lock()
+	defer m.mtx.Unlock()
+
+	metrics, ok := m.metrics[h]
+	if !ok {
+		return false
+	}
+
+	i := findMetricWithLabelValues(metrics, lvs, curry)
+	if i >= len(metrics) {
+		return false
+	}
+
+	if len(metrics) > 1 {
+		m.metrics[h] = append(metrics[:i], metrics[i+1:]...)
+	} else {
+		delete(m.metrics, h)
+	}
+	return true
+}
+
+// deleteByHashWithLabels removes the metric from the hash bucket h. If there
+// are multiple matches in the bucket, use lvs to select a metric and remove
+// only that metric.
+func (m *metricMap) deleteByHashWithLabels(
+	h uint64, labels Labels, curry []curriedLabelValue,
+) bool {
+	m.mtx.Lock()
+	defer m.mtx.Unlock()
+
+	metrics, ok := m.metrics[h]
+	if !ok {
+		return false
+	}
+	i := findMetricWithLabels(m.desc, metrics, labels, curry)
+	if i >= len(metrics) {
+		return false
+	}
+
+	if len(metrics) > 1 {
+		m.metrics[h] = append(metrics[:i], metrics[i+1:]...)
+	} else {
+		delete(m.metrics, h)
+	}
+	return true
+}
+
+// getOrCreateMetricWithLabelValues retrieves the metric by hash and label value
+// or creates it and returns the new one.
+//
+// This function holds the mutex.
+func (m *metricMap) getOrCreateMetricWithLabelValues(
+	hash uint64, lvs []string, curry []curriedLabelValue,
+) Metric {
+	m.mtx.RLock()
+	metric, ok := m.getMetricWithHashAndLabelValues(hash, lvs, curry)
+	m.mtx.RUnlock()
+	if ok {
+		return metric
+	}
+
+	m.mtx.Lock()
+	defer m.mtx.Unlock()
+	metric, ok = m.getMetricWithHashAndLabelValues(hash, lvs, curry)
+	if !ok {
+		inlinedLVs := inlineLabelValues(lvs, curry)
+		metric = m.newMetric(inlinedLVs...)
+		m.metrics[hash] = append(m.metrics[hash], metricWithLabelValues{values: inlinedLVs, metric: metric})
+	}
+	return metric
+}
+
+// getOrCreateMetricWithLabelValues retrieves the metric by hash and label value
+// or creates it and returns the new one.
+//
+// This function holds the mutex.
+func (m *metricMap) getOrCreateMetricWithLabels(
+	hash uint64, labels Labels, curry []curriedLabelValue,
+) Metric {
+	m.mtx.RLock()
+	metric, ok := m.getMetricWithHashAndLabels(hash, labels, curry)
+	m.mtx.RUnlock()
+	if ok {
+		return metric
+	}
+
+	m.mtx.Lock()
+	defer m.mtx.Unlock()
+	metric, ok = m.getMetricWithHashAndLabels(hash, labels, curry)
+	if !ok {
+		lvs := extractLabelValues(m.desc, labels, curry)
+		metric = m.newMetric(lvs...)
+		m.metrics[hash] = append(m.metrics[hash], metricWithLabelValues{values: lvs, metric: metric})
+	}
+	return metric
+}
+
+// getMetricWithHashAndLabelValues gets a metric while handling possible
+// collisions in the hash space. Must be called while holding the read mutex.
+func (m *metricMap) getMetricWithHashAndLabelValues(
+	h uint64, lvs []string, curry []curriedLabelValue,
+) (Metric, bool) {
+	metrics, ok := m.metrics[h]
+	if ok {
+		if i := findMetricWithLabelValues(metrics, lvs, curry); i < len(metrics) {
+			return metrics[i].metric, true
+		}
+	}
+	return nil, false
+}
+
+// getMetricWithHashAndLabels gets a metric while handling possible collisions in
+// the hash space. Must be called while holding read mutex.
+func (m *metricMap) getMetricWithHashAndLabels(
+	h uint64, labels Labels, curry []curriedLabelValue,
+) (Metric, bool) {
+	metrics, ok := m.metrics[h]
+	if ok {
+		if i := findMetricWithLabels(m.desc, metrics, labels, curry); i < len(metrics) {
+			return metrics[i].metric, true
+		}
+	}
+	return nil, false
+}
+
+// findMetricWithLabelValues returns the index of the matching metric or
+// len(metrics) if not found.
+func findMetricWithLabelValues(
+	metrics []metricWithLabelValues, lvs []string, curry []curriedLabelValue,
+) int {
+	for i, metric := range metrics {
+		if matchLabelValues(metric.values, lvs, curry) {
+			return i
+		}
+	}
+	return len(metrics)
+}
+
+// findMetricWithLabels returns the index of the matching metric or len(metrics)
+// if not found.
+func findMetricWithLabels(
+	desc *Desc, metrics []metricWithLabelValues, labels Labels, curry []curriedLabelValue,
+) int {
+	for i, metric := range metrics {
+		if matchLabels(desc, metric.values, labels, curry) {
+			return i
+		}
+	}
+	return len(metrics)
+}
+
+func matchLabelValues(values []string, lvs []string, curry []curriedLabelValue) bool {
+	if len(values) != len(lvs)+len(curry) {
+		return false
+	}
+	var iLVs, iCurry int
+	for i, v := range values {
+		if iCurry < len(curry) && curry[iCurry].index == i {
+			if v != curry[iCurry].value {
+				return false
+			}
+			iCurry++
+			continue
+		}
+		if v != lvs[iLVs] {
+			return false
+		}
+		iLVs++
+	}
+	return true
+}
+
+func matchLabels(desc *Desc, values []string, labels Labels, curry []curriedLabelValue) bool {
+	if len(values) != len(labels)+len(curry) {
+		return false
+	}
+	iCurry := 0
+	for i, k := range desc.variableLabels {
+		if iCurry < len(curry) && curry[iCurry].index == i {
+			if values[i] != curry[iCurry].value {
+				return false
+			}
+			iCurry++
+			continue
+		}
+		if values[i] != labels[k] {
+			return false
+		}
+	}
+	return true
+}
+
+func extractLabelValues(desc *Desc, labels Labels, curry []curriedLabelValue) []string {
+	labelValues := make([]string, len(labels)+len(curry))
+	iCurry := 0
+	for i, k := range desc.variableLabels {
+		if iCurry < len(curry) && curry[iCurry].index == i {
+			labelValues[i] = curry[iCurry].value
+			iCurry++
+			continue
+		}
+		labelValues[i] = labels[k]
+	}
+	return labelValues
+}
+
+func inlineLabelValues(lvs []string, curry []curriedLabelValue) []string {
+	labelValues := make([]string, len(lvs)+len(curry))
+	var iCurry, iLVs int
+	for i := range labelValues {
+		if iCurry < len(curry) && curry[iCurry].index == i {
+			labelValues[i] = curry[iCurry].value
+			iCurry++
+			continue
+		}
+		labelValues[i] = lvs[iLVs]
+		iLVs++
+	}
+	return labelValues
+}
diff --git a/vendor/github.com/prometheus/client_golang/prometheus/wrap.go b/vendor/github.com/prometheus/client_golang/prometheus/wrap.go
new file mode 100644
index 0000000..e303eef
--- /dev/null
+++ b/vendor/github.com/prometheus/client_golang/prometheus/wrap.go
@@ -0,0 +1,200 @@
+// Copyright 2018 The Prometheus 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 prometheus
+
+import (
+	"fmt"
+	"sort"
+
+	"github.com/golang/protobuf/proto"
+
+	dto "github.com/prometheus/client_model/go"
+)
+
+// WrapRegistererWith returns a Registerer wrapping the provided
+// Registerer. Collectors registered with the returned Registerer will be
+// registered with the wrapped Registerer in a modified way. The modified
+// Collector adds the provided Labels to all Metrics it collects (as
+// ConstLabels). The Metrics collected by the unmodified Collector must not
+// duplicate any of those labels.
+//
+// WrapRegistererWith provides a way to add fixed labels to a subset of
+// Collectors. It should not be used to add fixed labels to all metrics exposed.
+//
+// Conflicts between Collectors registered through the original Registerer with
+// Collectors registered through the wrapping Registerer will still be
+// detected. Any AlreadyRegisteredError returned by the Register method of
+// either Registerer will contain the ExistingCollector in the form it was
+// provided to the respective registry.
+//
+// The Collector example demonstrates a use of WrapRegistererWith.
+func WrapRegistererWith(labels Labels, reg Registerer) Registerer {
+	return &wrappingRegisterer{
+		wrappedRegisterer: reg,
+		labels:            labels,
+	}
+}
+
+// WrapRegistererWithPrefix returns a Registerer wrapping the provided
+// Registerer. Collectors registered with the returned Registerer will be
+// registered with the wrapped Registerer in a modified way. The modified
+// Collector adds the provided prefix to the name of all Metrics it collects.
+//
+// WrapRegistererWithPrefix is useful to have one place to prefix all metrics of
+// a sub-system. To make this work, register metrics of the sub-system with the
+// wrapping Registerer returned by WrapRegistererWithPrefix. It is rarely useful
+// to use the same prefix for all metrics exposed. In particular, do not prefix
+// metric names that are standardized across applications, as that would break
+// horizontal monitoring, for example the metrics provided by the Go collector
+// (see NewGoCollector) and the process collector (see NewProcessCollector). (In
+// fact, those metrics are already prefixed with “go_” or “process_”,
+// respectively.)
+//
+// Conflicts between Collectors registered through the original Registerer with
+// Collectors registered through the wrapping Registerer will still be
+// detected. Any AlreadyRegisteredError returned by the Register method of
+// either Registerer will contain the ExistingCollector in the form it was
+// provided to the respective registry.
+func WrapRegistererWithPrefix(prefix string, reg Registerer) Registerer {
+	return &wrappingRegisterer{
+		wrappedRegisterer: reg,
+		prefix:            prefix,
+	}
+}
+
+type wrappingRegisterer struct {
+	wrappedRegisterer Registerer
+	prefix            string
+	labels            Labels
+}
+
+func (r *wrappingRegisterer) Register(c Collector) error {
+	return r.wrappedRegisterer.Register(&wrappingCollector{
+		wrappedCollector: c,
+		prefix:           r.prefix,
+		labels:           r.labels,
+	})
+}
+
+func (r *wrappingRegisterer) MustRegister(cs ...Collector) {
+	for _, c := range cs {
+		if err := r.Register(c); err != nil {
+			panic(err)
+		}
+	}
+}
+
+func (r *wrappingRegisterer) Unregister(c Collector) bool {
+	return r.wrappedRegisterer.Unregister(&wrappingCollector{
+		wrappedCollector: c,
+		prefix:           r.prefix,
+		labels:           r.labels,
+	})
+}
+
+type wrappingCollector struct {
+	wrappedCollector Collector
+	prefix           string
+	labels           Labels
+}
+
+func (c *wrappingCollector) Collect(ch chan<- Metric) {
+	wrappedCh := make(chan Metric)
+	go func() {
+		c.wrappedCollector.Collect(wrappedCh)
+		close(wrappedCh)
+	}()
+	for m := range wrappedCh {
+		ch <- &wrappingMetric{
+			wrappedMetric: m,
+			prefix:        c.prefix,
+			labels:        c.labels,
+		}
+	}
+}
+
+func (c *wrappingCollector) Describe(ch chan<- *Desc) {
+	wrappedCh := make(chan *Desc)
+	go func() {
+		c.wrappedCollector.Describe(wrappedCh)
+		close(wrappedCh)
+	}()
+	for desc := range wrappedCh {
+		ch <- wrapDesc(desc, c.prefix, c.labels)
+	}
+}
+
+func (c *wrappingCollector) unwrapRecursively() Collector {
+	switch wc := c.wrappedCollector.(type) {
+	case *wrappingCollector:
+		return wc.unwrapRecursively()
+	default:
+		return wc
+	}
+}
+
+type wrappingMetric struct {
+	wrappedMetric Metric
+	prefix        string
+	labels        Labels
+}
+
+func (m *wrappingMetric) Desc() *Desc {
+	return wrapDesc(m.wrappedMetric.Desc(), m.prefix, m.labels)
+}
+
+func (m *wrappingMetric) Write(out *dto.Metric) error {
+	if err := m.wrappedMetric.Write(out); err != nil {
+		return err
+	}
+	if len(m.labels) == 0 {
+		// No wrapping labels.
+		return nil
+	}
+	for ln, lv := range m.labels {
+		out.Label = append(out.Label, &dto.LabelPair{
+			Name:  proto.String(ln),
+			Value: proto.String(lv),
+		})
+	}
+	sort.Sort(labelPairSorter(out.Label))
+	return nil
+}
+
+func wrapDesc(desc *Desc, prefix string, labels Labels) *Desc {
+	constLabels := Labels{}
+	for _, lp := range desc.constLabelPairs {
+		constLabels[*lp.Name] = *lp.Value
+	}
+	for ln, lv := range labels {
+		if _, alreadyUsed := constLabels[ln]; alreadyUsed {
+			return &Desc{
+				fqName:          desc.fqName,
+				help:            desc.help,
+				variableLabels:  desc.variableLabels,
+				constLabelPairs: desc.constLabelPairs,
+				err:             fmt.Errorf("attempted wrapping with already existing label name %q", ln),
+			}
+		}
+		constLabels[ln] = lv
+	}
+	// NewDesc will do remaining validations.
+	newDesc := NewDesc(prefix+desc.fqName, desc.help, desc.variableLabels, constLabels)
+	// Propagate errors if there was any. This will override any errer
+	// created by NewDesc above, i.e. earlier errors get precedence.
+	if desc.err != nil {
+		newDesc.err = desc.err
+	}
+	return newDesc
+}
diff --git a/vendor/github.com/prometheus/client_model/LICENSE b/vendor/github.com/prometheus/client_model/LICENSE
new file mode 100644
index 0000000..261eeb9
--- /dev/null
+++ b/vendor/github.com/prometheus/client_model/LICENSE
@@ -0,0 +1,201 @@
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
diff --git a/vendor/github.com/prometheus/client_model/NOTICE b/vendor/github.com/prometheus/client_model/NOTICE
new file mode 100644
index 0000000..20110e4
--- /dev/null
+++ b/vendor/github.com/prometheus/client_model/NOTICE
@@ -0,0 +1,5 @@
+Data model artifacts for Prometheus.
+Copyright 2012-2015 The Prometheus Authors
+
+This product includes software developed at
+SoundCloud Ltd. (http://soundcloud.com/).
diff --git a/vendor/github.com/prometheus/client_model/go/metrics.pb.go b/vendor/github.com/prometheus/client_model/go/metrics.pb.go
new file mode 100644
index 0000000..9805432
--- /dev/null
+++ b/vendor/github.com/prometheus/client_model/go/metrics.pb.go
@@ -0,0 +1,629 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: metrics.proto
+
+package io_prometheus_client // import "github.com/prometheus/client_model/go"
+
+import proto "github.com/golang/protobuf/proto"
+import fmt "fmt"
+import math "math"
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
+
+type MetricType int32
+
+const (
+	MetricType_COUNTER   MetricType = 0
+	MetricType_GAUGE     MetricType = 1
+	MetricType_SUMMARY   MetricType = 2
+	MetricType_UNTYPED   MetricType = 3
+	MetricType_HISTOGRAM MetricType = 4
+)
+
+var MetricType_name = map[int32]string{
+	0: "COUNTER",
+	1: "GAUGE",
+	2: "SUMMARY",
+	3: "UNTYPED",
+	4: "HISTOGRAM",
+}
+var MetricType_value = map[string]int32{
+	"COUNTER":   0,
+	"GAUGE":     1,
+	"SUMMARY":   2,
+	"UNTYPED":   3,
+	"HISTOGRAM": 4,
+}
+
+func (x MetricType) Enum() *MetricType {
+	p := new(MetricType)
+	*p = x
+	return p
+}
+func (x MetricType) String() string {
+	return proto.EnumName(MetricType_name, int32(x))
+}
+func (x *MetricType) UnmarshalJSON(data []byte) error {
+	value, err := proto.UnmarshalJSONEnum(MetricType_value, data, "MetricType")
+	if err != nil {
+		return err
+	}
+	*x = MetricType(value)
+	return nil
+}
+func (MetricType) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_metrics_c97c9a2b9560cb8f, []int{0}
+}
+
+type LabelPair struct {
+	Name                 *string  `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
+	Value                *string  `protobuf:"bytes,2,opt,name=value" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *LabelPair) Reset()         { *m = LabelPair{} }
+func (m *LabelPair) String() string { return proto.CompactTextString(m) }
+func (*LabelPair) ProtoMessage()    {}
+func (*LabelPair) Descriptor() ([]byte, []int) {
+	return fileDescriptor_metrics_c97c9a2b9560cb8f, []int{0}
+}
+func (m *LabelPair) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_LabelPair.Unmarshal(m, b)
+}
+func (m *LabelPair) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_LabelPair.Marshal(b, m, deterministic)
+}
+func (dst *LabelPair) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LabelPair.Merge(dst, src)
+}
+func (m *LabelPair) XXX_Size() int {
+	return xxx_messageInfo_LabelPair.Size(m)
+}
+func (m *LabelPair) XXX_DiscardUnknown() {
+	xxx_messageInfo_LabelPair.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_LabelPair proto.InternalMessageInfo
+
+func (m *LabelPair) GetName() string {
+	if m != nil && m.Name != nil {
+		return *m.Name
+	}
+	return ""
+}
+
+func (m *LabelPair) GetValue() string {
+	if m != nil && m.Value != nil {
+		return *m.Value
+	}
+	return ""
+}
+
+type Gauge struct {
+	Value                *float64 `protobuf:"fixed64,1,opt,name=value" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Gauge) Reset()         { *m = Gauge{} }
+func (m *Gauge) String() string { return proto.CompactTextString(m) }
+func (*Gauge) ProtoMessage()    {}
+func (*Gauge) Descriptor() ([]byte, []int) {
+	return fileDescriptor_metrics_c97c9a2b9560cb8f, []int{1}
+}
+func (m *Gauge) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Gauge.Unmarshal(m, b)
+}
+func (m *Gauge) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Gauge.Marshal(b, m, deterministic)
+}
+func (dst *Gauge) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Gauge.Merge(dst, src)
+}
+func (m *Gauge) XXX_Size() int {
+	return xxx_messageInfo_Gauge.Size(m)
+}
+func (m *Gauge) XXX_DiscardUnknown() {
+	xxx_messageInfo_Gauge.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Gauge proto.InternalMessageInfo
+
+func (m *Gauge) GetValue() float64 {
+	if m != nil && m.Value != nil {
+		return *m.Value
+	}
+	return 0
+}
+
+type Counter struct {
+	Value                *float64 `protobuf:"fixed64,1,opt,name=value" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Counter) Reset()         { *m = Counter{} }
+func (m *Counter) String() string { return proto.CompactTextString(m) }
+func (*Counter) ProtoMessage()    {}
+func (*Counter) Descriptor() ([]byte, []int) {
+	return fileDescriptor_metrics_c97c9a2b9560cb8f, []int{2}
+}
+func (m *Counter) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Counter.Unmarshal(m, b)
+}
+func (m *Counter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Counter.Marshal(b, m, deterministic)
+}
+func (dst *Counter) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Counter.Merge(dst, src)
+}
+func (m *Counter) XXX_Size() int {
+	return xxx_messageInfo_Counter.Size(m)
+}
+func (m *Counter) XXX_DiscardUnknown() {
+	xxx_messageInfo_Counter.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Counter proto.InternalMessageInfo
+
+func (m *Counter) GetValue() float64 {
+	if m != nil && m.Value != nil {
+		return *m.Value
+	}
+	return 0
+}
+
+type Quantile struct {
+	Quantile             *float64 `protobuf:"fixed64,1,opt,name=quantile" json:"quantile,omitempty"`
+	Value                *float64 `protobuf:"fixed64,2,opt,name=value" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Quantile) Reset()         { *m = Quantile{} }
+func (m *Quantile) String() string { return proto.CompactTextString(m) }
+func (*Quantile) ProtoMessage()    {}
+func (*Quantile) Descriptor() ([]byte, []int) {
+	return fileDescriptor_metrics_c97c9a2b9560cb8f, []int{3}
+}
+func (m *Quantile) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Quantile.Unmarshal(m, b)
+}
+func (m *Quantile) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Quantile.Marshal(b, m, deterministic)
+}
+func (dst *Quantile) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Quantile.Merge(dst, src)
+}
+func (m *Quantile) XXX_Size() int {
+	return xxx_messageInfo_Quantile.Size(m)
+}
+func (m *Quantile) XXX_DiscardUnknown() {
+	xxx_messageInfo_Quantile.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Quantile proto.InternalMessageInfo
+
+func (m *Quantile) GetQuantile() float64 {
+	if m != nil && m.Quantile != nil {
+		return *m.Quantile
+	}
+	return 0
+}
+
+func (m *Quantile) GetValue() float64 {
+	if m != nil && m.Value != nil {
+		return *m.Value
+	}
+	return 0
+}
+
+type Summary struct {
+	SampleCount          *uint64     `protobuf:"varint,1,opt,name=sample_count,json=sampleCount" json:"sample_count,omitempty"`
+	SampleSum            *float64    `protobuf:"fixed64,2,opt,name=sample_sum,json=sampleSum" json:"sample_sum,omitempty"`
+	Quantile             []*Quantile `protobuf:"bytes,3,rep,name=quantile" json:"quantile,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
+	XXX_unrecognized     []byte      `json:"-"`
+	XXX_sizecache        int32       `json:"-"`
+}
+
+func (m *Summary) Reset()         { *m = Summary{} }
+func (m *Summary) String() string { return proto.CompactTextString(m) }
+func (*Summary) ProtoMessage()    {}
+func (*Summary) Descriptor() ([]byte, []int) {
+	return fileDescriptor_metrics_c97c9a2b9560cb8f, []int{4}
+}
+func (m *Summary) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Summary.Unmarshal(m, b)
+}
+func (m *Summary) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Summary.Marshal(b, m, deterministic)
+}
+func (dst *Summary) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Summary.Merge(dst, src)
+}
+func (m *Summary) XXX_Size() int {
+	return xxx_messageInfo_Summary.Size(m)
+}
+func (m *Summary) XXX_DiscardUnknown() {
+	xxx_messageInfo_Summary.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Summary proto.InternalMessageInfo
+
+func (m *Summary) GetSampleCount() uint64 {
+	if m != nil && m.SampleCount != nil {
+		return *m.SampleCount
+	}
+	return 0
+}
+
+func (m *Summary) GetSampleSum() float64 {
+	if m != nil && m.SampleSum != nil {
+		return *m.SampleSum
+	}
+	return 0
+}
+
+func (m *Summary) GetQuantile() []*Quantile {
+	if m != nil {
+		return m.Quantile
+	}
+	return nil
+}
+
+type Untyped struct {
+	Value                *float64 `protobuf:"fixed64,1,opt,name=value" json:"value,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Untyped) Reset()         { *m = Untyped{} }
+func (m *Untyped) String() string { return proto.CompactTextString(m) }
+func (*Untyped) ProtoMessage()    {}
+func (*Untyped) Descriptor() ([]byte, []int) {
+	return fileDescriptor_metrics_c97c9a2b9560cb8f, []int{5}
+}
+func (m *Untyped) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Untyped.Unmarshal(m, b)
+}
+func (m *Untyped) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Untyped.Marshal(b, m, deterministic)
+}
+func (dst *Untyped) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Untyped.Merge(dst, src)
+}
+func (m *Untyped) XXX_Size() int {
+	return xxx_messageInfo_Untyped.Size(m)
+}
+func (m *Untyped) XXX_DiscardUnknown() {
+	xxx_messageInfo_Untyped.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Untyped proto.InternalMessageInfo
+
+func (m *Untyped) GetValue() float64 {
+	if m != nil && m.Value != nil {
+		return *m.Value
+	}
+	return 0
+}
+
+type Histogram struct {
+	SampleCount          *uint64   `protobuf:"varint,1,opt,name=sample_count,json=sampleCount" json:"sample_count,omitempty"`
+	SampleSum            *float64  `protobuf:"fixed64,2,opt,name=sample_sum,json=sampleSum" json:"sample_sum,omitempty"`
+	Bucket               []*Bucket `protobuf:"bytes,3,rep,name=bucket" json:"bucket,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
+	XXX_unrecognized     []byte    `json:"-"`
+	XXX_sizecache        int32     `json:"-"`
+}
+
+func (m *Histogram) Reset()         { *m = Histogram{} }
+func (m *Histogram) String() string { return proto.CompactTextString(m) }
+func (*Histogram) ProtoMessage()    {}
+func (*Histogram) Descriptor() ([]byte, []int) {
+	return fileDescriptor_metrics_c97c9a2b9560cb8f, []int{6}
+}
+func (m *Histogram) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Histogram.Unmarshal(m, b)
+}
+func (m *Histogram) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Histogram.Marshal(b, m, deterministic)
+}
+func (dst *Histogram) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Histogram.Merge(dst, src)
+}
+func (m *Histogram) XXX_Size() int {
+	return xxx_messageInfo_Histogram.Size(m)
+}
+func (m *Histogram) XXX_DiscardUnknown() {
+	xxx_messageInfo_Histogram.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Histogram proto.InternalMessageInfo
+
+func (m *Histogram) GetSampleCount() uint64 {
+	if m != nil && m.SampleCount != nil {
+		return *m.SampleCount
+	}
+	return 0
+}
+
+func (m *Histogram) GetSampleSum() float64 {
+	if m != nil && m.SampleSum != nil {
+		return *m.SampleSum
+	}
+	return 0
+}
+
+func (m *Histogram) GetBucket() []*Bucket {
+	if m != nil {
+		return m.Bucket
+	}
+	return nil
+}
+
+type Bucket struct {
+	CumulativeCount      *uint64  `protobuf:"varint,1,opt,name=cumulative_count,json=cumulativeCount" json:"cumulative_count,omitempty"`
+	UpperBound           *float64 `protobuf:"fixed64,2,opt,name=upper_bound,json=upperBound" json:"upper_bound,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *Bucket) Reset()         { *m = Bucket{} }
+func (m *Bucket) String() string { return proto.CompactTextString(m) }
+func (*Bucket) ProtoMessage()    {}
+func (*Bucket) Descriptor() ([]byte, []int) {
+	return fileDescriptor_metrics_c97c9a2b9560cb8f, []int{7}
+}
+func (m *Bucket) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Bucket.Unmarshal(m, b)
+}
+func (m *Bucket) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Bucket.Marshal(b, m, deterministic)
+}
+func (dst *Bucket) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Bucket.Merge(dst, src)
+}
+func (m *Bucket) XXX_Size() int {
+	return xxx_messageInfo_Bucket.Size(m)
+}
+func (m *Bucket) XXX_DiscardUnknown() {
+	xxx_messageInfo_Bucket.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Bucket proto.InternalMessageInfo
+
+func (m *Bucket) GetCumulativeCount() uint64 {
+	if m != nil && m.CumulativeCount != nil {
+		return *m.CumulativeCount
+	}
+	return 0
+}
+
+func (m *Bucket) GetUpperBound() float64 {
+	if m != nil && m.UpperBound != nil {
+		return *m.UpperBound
+	}
+	return 0
+}
+
+type Metric struct {
+	Label                []*LabelPair `protobuf:"bytes,1,rep,name=label" json:"label,omitempty"`
+	Gauge                *Gauge       `protobuf:"bytes,2,opt,name=gauge" json:"gauge,omitempty"`
+	Counter              *Counter     `protobuf:"bytes,3,opt,name=counter" json:"counter,omitempty"`
+	Summary              *Summary     `protobuf:"bytes,4,opt,name=summary" json:"summary,omitempty"`
+	Untyped              *Untyped     `protobuf:"bytes,5,opt,name=untyped" json:"untyped,omitempty"`
+	Histogram            *Histogram   `protobuf:"bytes,7,opt,name=histogram" json:"histogram,omitempty"`
+	TimestampMs          *int64       `protobuf:"varint,6,opt,name=timestamp_ms,json=timestampMs" json:"timestamp_ms,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
+	XXX_unrecognized     []byte       `json:"-"`
+	XXX_sizecache        int32        `json:"-"`
+}
+
+func (m *Metric) Reset()         { *m = Metric{} }
+func (m *Metric) String() string { return proto.CompactTextString(m) }
+func (*Metric) ProtoMessage()    {}
+func (*Metric) Descriptor() ([]byte, []int) {
+	return fileDescriptor_metrics_c97c9a2b9560cb8f, []int{8}
+}
+func (m *Metric) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_Metric.Unmarshal(m, b)
+}
+func (m *Metric) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_Metric.Marshal(b, m, deterministic)
+}
+func (dst *Metric) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Metric.Merge(dst, src)
+}
+func (m *Metric) XXX_Size() int {
+	return xxx_messageInfo_Metric.Size(m)
+}
+func (m *Metric) XXX_DiscardUnknown() {
+	xxx_messageInfo_Metric.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Metric proto.InternalMessageInfo
+
+func (m *Metric) GetLabel() []*LabelPair {
+	if m != nil {
+		return m.Label
+	}
+	return nil
+}
+
+func (m *Metric) GetGauge() *Gauge {
+	if m != nil {
+		return m.Gauge
+	}
+	return nil
+}
+
+func (m *Metric) GetCounter() *Counter {
+	if m != nil {
+		return m.Counter
+	}
+	return nil
+}
+
+func (m *Metric) GetSummary() *Summary {
+	if m != nil {
+		return m.Summary
+	}
+	return nil
+}
+
+func (m *Metric) GetUntyped() *Untyped {
+	if m != nil {
+		return m.Untyped
+	}
+	return nil
+}
+
+func (m *Metric) GetHistogram() *Histogram {
+	if m != nil {
+		return m.Histogram
+	}
+	return nil
+}
+
+func (m *Metric) GetTimestampMs() int64 {
+	if m != nil && m.TimestampMs != nil {
+		return *m.TimestampMs
+	}
+	return 0
+}
+
+type MetricFamily struct {
+	Name                 *string     `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"`
+	Help                 *string     `protobuf:"bytes,2,opt,name=help" json:"help,omitempty"`
+	Type                 *MetricType `protobuf:"varint,3,opt,name=type,enum=io.prometheus.client.MetricType" json:"type,omitempty"`
+	Metric               []*Metric   `protobuf:"bytes,4,rep,name=metric" json:"metric,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
+	XXX_unrecognized     []byte      `json:"-"`
+	XXX_sizecache        int32       `json:"-"`
+}
+
+func (m *MetricFamily) Reset()         { *m = MetricFamily{} }
+func (m *MetricFamily) String() string { return proto.CompactTextString(m) }
+func (*MetricFamily) ProtoMessage()    {}
+func (*MetricFamily) Descriptor() ([]byte, []int) {
+	return fileDescriptor_metrics_c97c9a2b9560cb8f, []int{9}
+}
+func (m *MetricFamily) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_MetricFamily.Unmarshal(m, b)
+}
+func (m *MetricFamily) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_MetricFamily.Marshal(b, m, deterministic)
+}
+func (dst *MetricFamily) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MetricFamily.Merge(dst, src)
+}
+func (m *MetricFamily) XXX_Size() int {
+	return xxx_messageInfo_MetricFamily.Size(m)
+}
+func (m *MetricFamily) XXX_DiscardUnknown() {
+	xxx_messageInfo_MetricFamily.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_MetricFamily proto.InternalMessageInfo
+
+func (m *MetricFamily) GetName() string {
+	if m != nil && m.Name != nil {
+		return *m.Name
+	}
+	return ""
+}
+
+func (m *MetricFamily) GetHelp() string {
+	if m != nil && m.Help != nil {
+		return *m.Help
+	}
+	return ""
+}
+
+func (m *MetricFamily) GetType() MetricType {
+	if m != nil && m.Type != nil {
+		return *m.Type
+	}
+	return MetricType_COUNTER
+}
+
+func (m *MetricFamily) GetMetric() []*Metric {
+	if m != nil {
+		return m.Metric
+	}
+	return nil
+}
+
+func init() {
+	proto.RegisterType((*LabelPair)(nil), "io.prometheus.client.LabelPair")
+	proto.RegisterType((*Gauge)(nil), "io.prometheus.client.Gauge")
+	proto.RegisterType((*Counter)(nil), "io.prometheus.client.Counter")
+	proto.RegisterType((*Quantile)(nil), "io.prometheus.client.Quantile")
+	proto.RegisterType((*Summary)(nil), "io.prometheus.client.Summary")
+	proto.RegisterType((*Untyped)(nil), "io.prometheus.client.Untyped")
+	proto.RegisterType((*Histogram)(nil), "io.prometheus.client.Histogram")
+	proto.RegisterType((*Bucket)(nil), "io.prometheus.client.Bucket")
+	proto.RegisterType((*Metric)(nil), "io.prometheus.client.Metric")
+	proto.RegisterType((*MetricFamily)(nil), "io.prometheus.client.MetricFamily")
+	proto.RegisterEnum("io.prometheus.client.MetricType", MetricType_name, MetricType_value)
+}
+
+func init() { proto.RegisterFile("metrics.proto", fileDescriptor_metrics_c97c9a2b9560cb8f) }
+
+var fileDescriptor_metrics_c97c9a2b9560cb8f = []byte{
+	// 591 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x54, 0x4f, 0x4f, 0xdb, 0x4e,
+	0x14, 0xfc, 0x99, 0xd8, 0x09, 0x7e, 0x86, 0x5f, 0xad, 0x15, 0x07, 0xab, 0x2d, 0x25, 0xcd, 0x89,
+	0xf6, 0x10, 0x54, 0x04, 0xaa, 0x44, 0xdb, 0x03, 0x50, 0x1a, 0x2a, 0xd5, 0x40, 0x37, 0xc9, 0x81,
+	0x5e, 0xac, 0x8d, 0x59, 0x25, 0x56, 0xbd, 0xb6, 0x6b, 0xef, 0x22, 0xe5, 0xdc, 0x43, 0xbf, 0x47,
+	0xbf, 0x68, 0xab, 0xfd, 0xe3, 0x18, 0x24, 0xc3, 0xa9, 0xb7, 0xb7, 0xf3, 0x66, 0xde, 0x8e, 0x77,
+	0xc7, 0x0b, 0x9b, 0x8c, 0xf2, 0x32, 0x89, 0xab, 0x61, 0x51, 0xe6, 0x3c, 0x47, 0x5b, 0x49, 0x2e,
+	0x2b, 0x46, 0xf9, 0x82, 0x8a, 0x6a, 0x18, 0xa7, 0x09, 0xcd, 0xf8, 0xe0, 0x10, 0xdc, 0x2f, 0x64,
+	0x46, 0xd3, 0x2b, 0x92, 0x94, 0x08, 0x81, 0x9d, 0x11, 0x46, 0x03, 0xab, 0x6f, 0xed, 0xba, 0x58,
+	0xd5, 0x68, 0x0b, 0x9c, 0x5b, 0x92, 0x0a, 0x1a, 0xac, 0x29, 0x50, 0x2f, 0x06, 0xdb, 0xe0, 0x8c,
+	0x88, 0x98, 0xdf, 0x69, 0x4b, 0x8d, 0x55, 0xb7, 0x77, 0xa0, 0x77, 0x9a, 0x8b, 0x8c, 0xd3, 0xf2,
+	0x01, 0xc2, 0x7b, 0x58, 0xff, 0x2a, 0x48, 0xc6, 0x93, 0x94, 0xa2, 0xa7, 0xb0, 0xfe, 0xc3, 0xd4,
+	0x86, 0xb4, 0x5a, 0xdf, 0xdf, 0x7d, 0xa5, 0xfe, 0x65, 0x41, 0x6f, 0x2c, 0x18, 0x23, 0xe5, 0x12,
+	0xbd, 0x84, 0x8d, 0x8a, 0xb0, 0x22, 0xa5, 0x51, 0x2c, 0x77, 0x54, 0x13, 0x6c, 0xec, 0x69, 0x4c,
+	0x99, 0x40, 0xdb, 0x00, 0x86, 0x52, 0x09, 0x66, 0x26, 0xb9, 0x1a, 0x19, 0x0b, 0x86, 0x8e, 0xee,
+	0xec, 0xdf, 0xe9, 0x77, 0x76, 0xbd, 0xfd, 0x17, 0xc3, 0xb6, 0xb3, 0x1a, 0xd6, 0x8e, 0x1b, 0x7f,
+	0xf2, 0x43, 0xa7, 0x19, 0x5f, 0x16, 0xf4, 0xe6, 0x81, 0x0f, 0xfd, 0x69, 0x81, 0x7b, 0x9e, 0x54,
+	0x3c, 0x9f, 0x97, 0x84, 0xfd, 0x03, 0xb3, 0x07, 0xd0, 0x9d, 0x89, 0xf8, 0x3b, 0xe5, 0xc6, 0xea,
+	0xf3, 0x76, 0xab, 0x27, 0x8a, 0x83, 0x0d, 0x77, 0x30, 0x81, 0xae, 0x46, 0xd0, 0x2b, 0xf0, 0x63,
+	0xc1, 0x44, 0x4a, 0x78, 0x72, 0x7b, 0xdf, 0xc5, 0x93, 0x06, 0xd7, 0x4e, 0x76, 0xc0, 0x13, 0x45,
+	0x41, 0xcb, 0x68, 0x96, 0x8b, 0xec, 0xc6, 0x58, 0x01, 0x05, 0x9d, 0x48, 0x64, 0xf0, 0x67, 0x0d,
+	0xba, 0xa1, 0xca, 0x18, 0x3a, 0x04, 0x27, 0x95, 0x31, 0x0a, 0x2c, 0xe5, 0x6a, 0xa7, 0xdd, 0xd5,
+	0x2a, 0x69, 0x58, 0xb3, 0xd1, 0x1b, 0x70, 0xe6, 0x32, 0x46, 0x6a, 0xb8, 0xb7, 0xff, 0xac, 0x5d,
+	0xa6, 0x92, 0x86, 0x35, 0x13, 0xbd, 0x85, 0x5e, 0xac, 0xa3, 0x15, 0x74, 0x94, 0x68, 0xbb, 0x5d,
+	0x64, 0xf2, 0x87, 0x6b, 0xb6, 0x14, 0x56, 0x3a, 0x33, 0x81, 0xfd, 0x98, 0xd0, 0x04, 0x0b, 0xd7,
+	0x6c, 0x29, 0x14, 0xfa, 0x8e, 0x03, 0xe7, 0x31, 0xa1, 0x09, 0x02, 0xae, 0xd9, 0xe8, 0x03, 0xb8,
+	0x8b, 0xfa, 0xea, 0x83, 0x9e, 0x92, 0x3e, 0x70, 0x30, 0xab, 0x84, 0xe0, 0x46, 0x21, 0xc3, 0xc2,
+	0x13, 0x46, 0x2b, 0x4e, 0x58, 0x11, 0xb1, 0x2a, 0xe8, 0xf6, 0xad, 0xdd, 0x0e, 0xf6, 0x56, 0x58,
+	0x58, 0x0d, 0x7e, 0x5b, 0xb0, 0xa1, 0x6f, 0xe0, 0x13, 0x61, 0x49, 0xba, 0x6c, 0xfd, 0x83, 0x11,
+	0xd8, 0x0b, 0x9a, 0x16, 0xe6, 0x07, 0x56, 0x35, 0x3a, 0x00, 0x5b, 0x7a, 0x54, 0x47, 0xf8, 0xff,
+	0x7e, 0xbf, 0xdd, 0x95, 0x9e, 0x3c, 0x59, 0x16, 0x14, 0x2b, 0xb6, 0x0c, 0x9f, 0x7e, 0x53, 0x02,
+	0xfb, 0xb1, 0xf0, 0x69, 0x1d, 0x36, 0xdc, 0xd7, 0x21, 0x40, 0x33, 0x09, 0x79, 0xd0, 0x3b, 0xbd,
+	0x9c, 0x5e, 0x4c, 0xce, 0xb0, 0xff, 0x1f, 0x72, 0xc1, 0x19, 0x1d, 0x4f, 0x47, 0x67, 0xbe, 0x25,
+	0xf1, 0xf1, 0x34, 0x0c, 0x8f, 0xf1, 0xb5, 0xbf, 0x26, 0x17, 0xd3, 0x8b, 0xc9, 0xf5, 0xd5, 0xd9,
+	0x47, 0xbf, 0x83, 0x36, 0xc1, 0x3d, 0xff, 0x3c, 0x9e, 0x5c, 0x8e, 0xf0, 0x71, 0xe8, 0xdb, 0x27,
+	0x18, 0x5a, 0x5f, 0xb2, 0x6f, 0x47, 0xf3, 0x84, 0x2f, 0xc4, 0x6c, 0x18, 0xe7, 0x6c, 0xaf, 0xe9,
+	0xee, 0xe9, 0x6e, 0xc4, 0xf2, 0x1b, 0x9a, 0xee, 0xcd, 0xf3, 0x77, 0x49, 0x1e, 0x35, 0xdd, 0x48,
+	0x77, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0x45, 0x21, 0x7f, 0x64, 0x2b, 0x05, 0x00, 0x00,
+}
diff --git a/vendor/github.com/prometheus/common/LICENSE b/vendor/github.com/prometheus/common/LICENSE
new file mode 100644
index 0000000..261eeb9
--- /dev/null
+++ b/vendor/github.com/prometheus/common/LICENSE
@@ -0,0 +1,201 @@
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
diff --git a/vendor/github.com/prometheus/common/NOTICE b/vendor/github.com/prometheus/common/NOTICE
new file mode 100644
index 0000000..636a2c1
--- /dev/null
+++ b/vendor/github.com/prometheus/common/NOTICE
@@ -0,0 +1,5 @@
+Common libraries shared by Prometheus Go components.
+Copyright 2015 The Prometheus Authors
+
+This product includes software developed at
+SoundCloud Ltd. (http://soundcloud.com/).
diff --git a/vendor/github.com/prometheus/common/expfmt/decode.go b/vendor/github.com/prometheus/common/expfmt/decode.go
new file mode 100644
index 0000000..c092723
--- /dev/null
+++ b/vendor/github.com/prometheus/common/expfmt/decode.go
@@ -0,0 +1,429 @@
+// Copyright 2015 The Prometheus 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 expfmt
+
+import (
+	"fmt"
+	"io"
+	"math"
+	"mime"
+	"net/http"
+
+	dto "github.com/prometheus/client_model/go"
+
+	"github.com/matttproud/golang_protobuf_extensions/pbutil"
+	"github.com/prometheus/common/model"
+)
+
+// Decoder types decode an input stream into metric families.
+type Decoder interface {
+	Decode(*dto.MetricFamily) error
+}
+
+// DecodeOptions contains options used by the Decoder and in sample extraction.
+type DecodeOptions struct {
+	// Timestamp is added to each value from the stream that has no explicit timestamp set.
+	Timestamp model.Time
+}
+
+// ResponseFormat extracts the correct format from a HTTP response header.
+// If no matching format can be found FormatUnknown is returned.
+func ResponseFormat(h http.Header) Format {
+	ct := h.Get(hdrContentType)
+
+	mediatype, params, err := mime.ParseMediaType(ct)
+	if err != nil {
+		return FmtUnknown
+	}
+
+	const textType = "text/plain"
+
+	switch mediatype {
+	case ProtoType:
+		if p, ok := params["proto"]; ok && p != ProtoProtocol {
+			return FmtUnknown
+		}
+		if e, ok := params["encoding"]; ok && e != "delimited" {
+			return FmtUnknown
+		}
+		return FmtProtoDelim
+
+	case textType:
+		if v, ok := params["version"]; ok && v != TextVersion {
+			return FmtUnknown
+		}
+		return FmtText
+	}
+
+	return FmtUnknown
+}
+
+// NewDecoder returns a new decoder based on the given input format.
+// If the input format does not imply otherwise, a text format decoder is returned.
+func NewDecoder(r io.Reader, format Format) Decoder {
+	switch format {
+	case FmtProtoDelim:
+		return &protoDecoder{r: r}
+	}
+	return &textDecoder{r: r}
+}
+
+// protoDecoder implements the Decoder interface for protocol buffers.
+type protoDecoder struct {
+	r io.Reader
+}
+
+// Decode implements the Decoder interface.
+func (d *protoDecoder) Decode(v *dto.MetricFamily) error {
+	_, err := pbutil.ReadDelimited(d.r, v)
+	if err != nil {
+		return err
+	}
+	if !model.IsValidMetricName(model.LabelValue(v.GetName())) {
+		return fmt.Errorf("invalid metric name %q", v.GetName())
+	}
+	for _, m := range v.GetMetric() {
+		if m == nil {
+			continue
+		}
+		for _, l := range m.GetLabel() {
+			if l == nil {
+				continue
+			}
+			if !model.LabelValue(l.GetValue()).IsValid() {
+				return fmt.Errorf("invalid label value %q", l.GetValue())
+			}
+			if !model.LabelName(l.GetName()).IsValid() {
+				return fmt.Errorf("invalid label name %q", l.GetName())
+			}
+		}
+	}
+	return nil
+}
+
+// textDecoder implements the Decoder interface for the text protocol.
+type textDecoder struct {
+	r    io.Reader
+	p    TextParser
+	fams []*dto.MetricFamily
+}
+
+// Decode implements the Decoder interface.
+func (d *textDecoder) Decode(v *dto.MetricFamily) error {
+	// TODO(fabxc): Wrap this as a line reader to make streaming safer.
+	if len(d.fams) == 0 {
+		// No cached metric families, read everything and parse metrics.
+		fams, err := d.p.TextToMetricFamilies(d.r)
+		if err != nil {
+			return err
+		}
+		if len(fams) == 0 {
+			return io.EOF
+		}
+		d.fams = make([]*dto.MetricFamily, 0, len(fams))
+		for _, f := range fams {
+			d.fams = append(d.fams, f)
+		}
+	}
+
+	*v = *d.fams[0]
+	d.fams = d.fams[1:]
+
+	return nil
+}
+
+// SampleDecoder wraps a Decoder to extract samples from the metric families
+// decoded by the wrapped Decoder.
+type SampleDecoder struct {
+	Dec  Decoder
+	Opts *DecodeOptions
+
+	f dto.MetricFamily
+}
+
+// Decode calls the Decode method of the wrapped Decoder and then extracts the
+// samples from the decoded MetricFamily into the provided model.Vector.
+func (sd *SampleDecoder) Decode(s *model.Vector) error {
+	err := sd.Dec.Decode(&sd.f)
+	if err != nil {
+		return err
+	}
+	*s, err = extractSamples(&sd.f, sd.Opts)
+	return err
+}
+
+// ExtractSamples builds a slice of samples from the provided metric
+// families. If an error occurrs during sample extraction, it continues to
+// extract from the remaining metric families. The returned error is the last
+// error that has occurred.
+func ExtractSamples(o *DecodeOptions, fams ...*dto.MetricFamily) (model.Vector, error) {
+	var (
+		all     model.Vector
+		lastErr error
+	)
+	for _, f := range fams {
+		some, err := extractSamples(f, o)
+		if err != nil {
+			lastErr = err
+			continue
+		}
+		all = append(all, some...)
+	}
+	return all, lastErr
+}
+
+func extractSamples(f *dto.MetricFamily, o *DecodeOptions) (model.Vector, error) {
+	switch f.GetType() {
+	case dto.MetricType_COUNTER:
+		return extractCounter(o, f), nil
+	case dto.MetricType_GAUGE:
+		return extractGauge(o, f), nil
+	case dto.MetricType_SUMMARY:
+		return extractSummary(o, f), nil
+	case dto.MetricType_UNTYPED:
+		return extractUntyped(o, f), nil
+	case dto.MetricType_HISTOGRAM:
+		return extractHistogram(o, f), nil
+	}
+	return nil, fmt.Errorf("expfmt.extractSamples: unknown metric family type %v", f.GetType())
+}
+
+func extractCounter(o *DecodeOptions, f *dto.MetricFamily) model.Vector {
+	samples := make(model.Vector, 0, len(f.Metric))
+
+	for _, m := range f.Metric {
+		if m.Counter == nil {
+			continue
+		}
+
+		lset := make(model.LabelSet, len(m.Label)+1)
+		for _, p := range m.Label {
+			lset[model.LabelName(p.GetName())] = model.LabelValue(p.GetValue())
+		}
+		lset[model.MetricNameLabel] = model.LabelValue(f.GetName())
+
+		smpl := &model.Sample{
+			Metric: model.Metric(lset),
+			Value:  model.SampleValue(m.Counter.GetValue()),
+		}
+
+		if m.TimestampMs != nil {
+			smpl.Timestamp = model.TimeFromUnixNano(*m.TimestampMs * 1000000)
+		} else {
+			smpl.Timestamp = o.Timestamp
+		}
+
+		samples = append(samples, smpl)
+	}
+
+	return samples
+}
+
+func extractGauge(o *DecodeOptions, f *dto.MetricFamily) model.Vector {
+	samples := make(model.Vector, 0, len(f.Metric))
+
+	for _, m := range f.Metric {
+		if m.Gauge == nil {
+			continue
+		}
+
+		lset := make(model.LabelSet, len(m.Label)+1)
+		for _, p := range m.Label {
+			lset[model.LabelName(p.GetName())] = model.LabelValue(p.GetValue())
+		}
+		lset[model.MetricNameLabel] = model.LabelValue(f.GetName())
+
+		smpl := &model.Sample{
+			Metric: model.Metric(lset),
+			Value:  model.SampleValue(m.Gauge.GetValue()),
+		}
+
+		if m.TimestampMs != nil {
+			smpl.Timestamp = model.TimeFromUnixNano(*m.TimestampMs * 1000000)
+		} else {
+			smpl.Timestamp = o.Timestamp
+		}
+
+		samples = append(samples, smpl)
+	}
+
+	return samples
+}
+
+func extractUntyped(o *DecodeOptions, f *dto.MetricFamily) model.Vector {
+	samples := make(model.Vector, 0, len(f.Metric))
+
+	for _, m := range f.Metric {
+		if m.Untyped == nil {
+			continue
+		}
+
+		lset := make(model.LabelSet, len(m.Label)+1)
+		for _, p := range m.Label {
+			lset[model.LabelName(p.GetName())] = model.LabelValue(p.GetValue())
+		}
+		lset[model.MetricNameLabel] = model.LabelValue(f.GetName())
+
+		smpl := &model.Sample{
+			Metric: model.Metric(lset),
+			Value:  model.SampleValue(m.Untyped.GetValue()),
+		}
+
+		if m.TimestampMs != nil {
+			smpl.Timestamp = model.TimeFromUnixNano(*m.TimestampMs * 1000000)
+		} else {
+			smpl.Timestamp = o.Timestamp
+		}
+
+		samples = append(samples, smpl)
+	}
+
+	return samples
+}
+
+func extractSummary(o *DecodeOptions, f *dto.MetricFamily) model.Vector {
+	samples := make(model.Vector, 0, len(f.Metric))
+
+	for _, m := range f.Metric {
+		if m.Summary == nil {
+			continue
+		}
+
+		timestamp := o.Timestamp
+		if m.TimestampMs != nil {
+			timestamp = model.TimeFromUnixNano(*m.TimestampMs * 1000000)
+		}
+
+		for _, q := range m.Summary.Quantile {
+			lset := make(model.LabelSet, len(m.Label)+2)
+			for _, p := range m.Label {
+				lset[model.LabelName(p.GetName())] = model.LabelValue(p.GetValue())
+			}
+			// BUG(matt): Update other names to "quantile".
+			lset[model.LabelName(model.QuantileLabel)] = model.LabelValue(fmt.Sprint(q.GetQuantile()))
+			lset[model.MetricNameLabel] = model.LabelValue(f.GetName())
+
+			samples = append(samples, &model.Sample{
+				Metric:    model.Metric(lset),
+				Value:     model.SampleValue(q.GetValue()),
+				Timestamp: timestamp,
+			})
+		}
+
+		lset := make(model.LabelSet, len(m.Label)+1)
+		for _, p := range m.Label {
+			lset[model.LabelName(p.GetName())] = model.LabelValue(p.GetValue())
+		}
+		lset[model.MetricNameLabel] = model.LabelValue(f.GetName() + "_sum")
+
+		samples = append(samples, &model.Sample{
+			Metric:    model.Metric(lset),
+			Value:     model.SampleValue(m.Summary.GetSampleSum()),
+			Timestamp: timestamp,
+		})
+
+		lset = make(model.LabelSet, len(m.Label)+1)
+		for _, p := range m.Label {
+			lset[model.LabelName(p.GetName())] = model.LabelValue(p.GetValue())
+		}
+		lset[model.MetricNameLabel] = model.LabelValue(f.GetName() + "_count")
+
+		samples = append(samples, &model.Sample{
+			Metric:    model.Metric(lset),
+			Value:     model.SampleValue(m.Summary.GetSampleCount()),
+			Timestamp: timestamp,
+		})
+	}
+
+	return samples
+}
+
+func extractHistogram(o *DecodeOptions, f *dto.MetricFamily) model.Vector {
+	samples := make(model.Vector, 0, len(f.Metric))
+
+	for _, m := range f.Metric {
+		if m.Histogram == nil {
+			continue
+		}
+
+		timestamp := o.Timestamp
+		if m.TimestampMs != nil {
+			timestamp = model.TimeFromUnixNano(*m.TimestampMs * 1000000)
+		}
+
+		infSeen := false
+
+		for _, q := range m.Histogram.Bucket {
+			lset := make(model.LabelSet, len(m.Label)+2)
+			for _, p := range m.Label {
+				lset[model.LabelName(p.GetName())] = model.LabelValue(p.GetValue())
+			}
+			lset[model.LabelName(model.BucketLabel)] = model.LabelValue(fmt.Sprint(q.GetUpperBound()))
+			lset[model.MetricNameLabel] = model.LabelValue(f.GetName() + "_bucket")
+
+			if math.IsInf(q.GetUpperBound(), +1) {
+				infSeen = true
+			}
+
+			samples = append(samples, &model.Sample{
+				Metric:    model.Metric(lset),
+				Value:     model.SampleValue(q.GetCumulativeCount()),
+				Timestamp: timestamp,
+			})
+		}
+
+		lset := make(model.LabelSet, len(m.Label)+1)
+		for _, p := range m.Label {
+			lset[model.LabelName(p.GetName())] = model.LabelValue(p.GetValue())
+		}
+		lset[model.MetricNameLabel] = model.LabelValue(f.GetName() + "_sum")
+
+		samples = append(samples, &model.Sample{
+			Metric:    model.Metric(lset),
+			Value:     model.SampleValue(m.Histogram.GetSampleSum()),
+			Timestamp: timestamp,
+		})
+
+		lset = make(model.LabelSet, len(m.Label)+1)
+		for _, p := range m.Label {
+			lset[model.LabelName(p.GetName())] = model.LabelValue(p.GetValue())
+		}
+		lset[model.MetricNameLabel] = model.LabelValue(f.GetName() + "_count")
+
+		count := &model.Sample{
+			Metric:    model.Metric(lset),
+			Value:     model.SampleValue(m.Histogram.GetSampleCount()),
+			Timestamp: timestamp,
+		}
+		samples = append(samples, count)
+
+		if !infSeen {
+			// Append an infinity bucket sample.
+			lset := make(model.LabelSet, len(m.Label)+2)
+			for _, p := range m.Label {
+				lset[model.LabelName(p.GetName())] = model.LabelValue(p.GetValue())
+			}
+			lset[model.LabelName(model.BucketLabel)] = model.LabelValue("+Inf")
+			lset[model.MetricNameLabel] = model.LabelValue(f.GetName() + "_bucket")
+
+			samples = append(samples, &model.Sample{
+				Metric:    model.Metric(lset),
+				Value:     count.Value,
+				Timestamp: timestamp,
+			})
+		}
+	}
+
+	return samples
+}
diff --git a/vendor/github.com/prometheus/common/expfmt/encode.go b/vendor/github.com/prometheus/common/expfmt/encode.go
new file mode 100644
index 0000000..11839ed
--- /dev/null
+++ b/vendor/github.com/prometheus/common/expfmt/encode.go
@@ -0,0 +1,88 @@
+// Copyright 2015 The Prometheus 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 expfmt
+
+import (
+	"fmt"
+	"io"
+	"net/http"
+
+	"github.com/golang/protobuf/proto"
+	"github.com/matttproud/golang_protobuf_extensions/pbutil"
+	"github.com/prometheus/common/internal/bitbucket.org/ww/goautoneg"
+
+	dto "github.com/prometheus/client_model/go"
+)
+
+// Encoder types encode metric families into an underlying wire protocol.
+type Encoder interface {
+	Encode(*dto.MetricFamily) error
+}
+
+type encoder func(*dto.MetricFamily) error
+
+func (e encoder) Encode(v *dto.MetricFamily) error {
+	return e(v)
+}
+
+// Negotiate returns the Content-Type based on the given Accept header.
+// If no appropriate accepted type is found, FmtText is returned.
+func Negotiate(h http.Header) Format {
+	for _, ac := range goautoneg.ParseAccept(h.Get(hdrAccept)) {
+		// Check for protocol buffer
+		if ac.Type+"/"+ac.SubType == ProtoType && ac.Params["proto"] == ProtoProtocol {
+			switch ac.Params["encoding"] {
+			case "delimited":
+				return FmtProtoDelim
+			case "text":
+				return FmtProtoText
+			case "compact-text":
+				return FmtProtoCompact
+			}
+		}
+		// Check for text format.
+		ver := ac.Params["version"]
+		if ac.Type == "text" && ac.SubType == "plain" && (ver == TextVersion || ver == "") {
+			return FmtText
+		}
+	}
+	return FmtText
+}
+
+// NewEncoder returns a new encoder based on content type negotiation.
+func NewEncoder(w io.Writer, format Format) Encoder {
+	switch format {
+	case FmtProtoDelim:
+		return encoder(func(v *dto.MetricFamily) error {
+			_, err := pbutil.WriteDelimited(w, v)
+			return err
+		})
+	case FmtProtoCompact:
+		return encoder(func(v *dto.MetricFamily) error {
+			_, err := fmt.Fprintln(w, v.String())
+			return err
+		})
+	case FmtProtoText:
+		return encoder(func(v *dto.MetricFamily) error {
+			_, err := fmt.Fprintln(w, proto.MarshalTextString(v))
+			return err
+		})
+	case FmtText:
+		return encoder(func(v *dto.MetricFamily) error {
+			_, err := MetricFamilyToText(w, v)
+			return err
+		})
+	}
+	panic("expfmt.NewEncoder: unknown format")
+}
diff --git a/vendor/github.com/prometheus/common/expfmt/expfmt.go b/vendor/github.com/prometheus/common/expfmt/expfmt.go
new file mode 100644
index 0000000..c71bcb9
--- /dev/null
+++ b/vendor/github.com/prometheus/common/expfmt/expfmt.go
@@ -0,0 +1,38 @@
+// Copyright 2015 The Prometheus 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 expfmt contains tools for reading and writing Prometheus metrics.
+package expfmt
+
+// Format specifies the HTTP content type of the different wire protocols.
+type Format string
+
+// Constants to assemble the Content-Type values for the different wire protocols.
+const (
+	TextVersion   = "0.0.4"
+	ProtoType     = `application/vnd.google.protobuf`
+	ProtoProtocol = `io.prometheus.client.MetricFamily`
+	ProtoFmt      = ProtoType + "; proto=" + ProtoProtocol + ";"
+
+	// The Content-Type values for the different wire protocols.
+	FmtUnknown      Format = `<unknown>`
+	FmtText         Format = `text/plain; version=` + TextVersion + `; charset=utf-8`
+	FmtProtoDelim   Format = ProtoFmt + ` encoding=delimited`
+	FmtProtoText    Format = ProtoFmt + ` encoding=text`
+	FmtProtoCompact Format = ProtoFmt + ` encoding=compact-text`
+)
+
+const (
+	hdrContentType = "Content-Type"
+	hdrAccept      = "Accept"
+)
diff --git a/vendor/github.com/prometheus/common/expfmt/fuzz.go b/vendor/github.com/prometheus/common/expfmt/fuzz.go
new file mode 100644
index 0000000..dc2eede
--- /dev/null
+++ b/vendor/github.com/prometheus/common/expfmt/fuzz.go
@@ -0,0 +1,36 @@
+// Copyright 2014 The Prometheus 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.
+
+// Build only when actually fuzzing
+// +build gofuzz
+
+package expfmt
+
+import "bytes"
+
+// Fuzz text metric parser with with github.com/dvyukov/go-fuzz:
+//
+//     go-fuzz-build github.com/prometheus/common/expfmt
+//     go-fuzz -bin expfmt-fuzz.zip -workdir fuzz
+//
+// Further input samples should go in the folder fuzz/corpus.
+func Fuzz(in []byte) int {
+	parser := TextParser{}
+	_, err := parser.TextToMetricFamilies(bytes.NewReader(in))
+
+	if err != nil {
+		return 0
+	}
+
+	return 1
+}
diff --git a/vendor/github.com/prometheus/common/expfmt/text_create.go b/vendor/github.com/prometheus/common/expfmt/text_create.go
new file mode 100644
index 0000000..8e473d0
--- /dev/null
+++ b/vendor/github.com/prometheus/common/expfmt/text_create.go
@@ -0,0 +1,468 @@
+// Copyright 2014 The Prometheus 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 expfmt
+
+import (
+	"bytes"
+	"fmt"
+	"io"
+	"math"
+	"strconv"
+	"strings"
+	"sync"
+
+	"github.com/prometheus/common/model"
+
+	dto "github.com/prometheus/client_model/go"
+)
+
+// enhancedWriter has all the enhanced write functions needed here. bytes.Buffer
+// implements it.
+type enhancedWriter interface {
+	io.Writer
+	WriteRune(r rune) (n int, err error)
+	WriteString(s string) (n int, err error)
+	WriteByte(c byte) error
+}
+
+const (
+	initialBufSize    = 512
+	initialNumBufSize = 24
+)
+
+var (
+	bufPool = sync.Pool{
+		New: func() interface{} {
+			return bytes.NewBuffer(make([]byte, 0, initialBufSize))
+		},
+	}
+	numBufPool = sync.Pool{
+		New: func() interface{} {
+			b := make([]byte, 0, initialNumBufSize)
+			return &b
+		},
+	}
+)
+
+// MetricFamilyToText converts a MetricFamily proto message into text format and
+// writes the resulting lines to 'out'. It returns the number of bytes written
+// and any error encountered. The output will have the same order as the input,
+// no further sorting is performed. Furthermore, this function assumes the input
+// is already sanitized and does not perform any sanity checks. If the input
+// contains duplicate metrics or invalid metric or label names, the conversion
+// will result in invalid text format output.
+//
+// This method fulfills the type 'prometheus.encoder'.
+func MetricFamilyToText(out io.Writer, in *dto.MetricFamily) (written int, err error) {
+	// Fail-fast checks.
+	if len(in.Metric) == 0 {
+		return 0, fmt.Errorf("MetricFamily has no metrics: %s", in)
+	}
+	name := in.GetName()
+	if name == "" {
+		return 0, fmt.Errorf("MetricFamily has no name: %s", in)
+	}
+
+	// Try the interface upgrade. If it doesn't work, we'll use a
+	// bytes.Buffer from the sync.Pool and write out its content to out in a
+	// single go in the end.
+	w, ok := out.(enhancedWriter)
+	if !ok {
+		b := bufPool.Get().(*bytes.Buffer)
+		b.Reset()
+		w = b
+		defer func() {
+			bWritten, bErr := out.Write(b.Bytes())
+			written = bWritten
+			if err == nil {
+				err = bErr
+			}
+			bufPool.Put(b)
+		}()
+	}
+
+	var n int
+
+	// Comments, first HELP, then TYPE.
+	if in.Help != nil {
+		n, err = w.WriteString("# HELP ")
+		written += n
+		if err != nil {
+			return
+		}
+		n, err = w.WriteString(name)
+		written += n
+		if err != nil {
+			return
+		}
+		err = w.WriteByte(' ')
+		written++
+		if err != nil {
+			return
+		}
+		n, err = writeEscapedString(w, *in.Help, false)
+		written += n
+		if err != nil {
+			return
+		}
+		err = w.WriteByte('\n')
+		written++
+		if err != nil {
+			return
+		}
+	}
+	n, err = w.WriteString("# TYPE ")
+	written += n
+	if err != nil {
+		return
+	}
+	n, err = w.WriteString(name)
+	written += n
+	if err != nil {
+		return
+	}
+	metricType := in.GetType()
+	switch metricType {
+	case dto.MetricType_COUNTER:
+		n, err = w.WriteString(" counter\n")
+	case dto.MetricType_GAUGE:
+		n, err = w.WriteString(" gauge\n")
+	case dto.MetricType_SUMMARY:
+		n, err = w.WriteString(" summary\n")
+	case dto.MetricType_UNTYPED:
+		n, err = w.WriteString(" untyped\n")
+	case dto.MetricType_HISTOGRAM:
+		n, err = w.WriteString(" histogram\n")
+	default:
+		return written, fmt.Errorf("unknown metric type %s", metricType.String())
+	}
+	written += n
+	if err != nil {
+		return
+	}
+
+	// Finally the samples, one line for each.
+	for _, metric := range in.Metric {
+		switch metricType {
+		case dto.MetricType_COUNTER:
+			if metric.Counter == nil {
+				return written, fmt.Errorf(
+					"expected counter in metric %s %s", name, metric,
+				)
+			}
+			n, err = writeSample(
+				w, name, "", metric, "", 0,
+				metric.Counter.GetValue(),
+			)
+		case dto.MetricType_GAUGE:
+			if metric.Gauge == nil {
+				return written, fmt.Errorf(
+					"expected gauge in metric %s %s", name, metric,
+				)
+			}
+			n, err = writeSample(
+				w, name, "", metric, "", 0,
+				metric.Gauge.GetValue(),
+			)
+		case dto.MetricType_UNTYPED:
+			if metric.Untyped == nil {
+				return written, fmt.Errorf(
+					"expected untyped in metric %s %s", name, metric,
+				)
+			}
+			n, err = writeSample(
+				w, name, "", metric, "", 0,
+				metric.Untyped.GetValue(),
+			)
+		case dto.MetricType_SUMMARY:
+			if metric.Summary == nil {
+				return written, fmt.Errorf(
+					"expected summary in metric %s %s", name, metric,
+				)
+			}
+			for _, q := range metric.Summary.Quantile {
+				n, err = writeSample(
+					w, name, "", metric,
+					model.QuantileLabel, q.GetQuantile(),
+					q.GetValue(),
+				)
+				written += n
+				if err != nil {
+					return
+				}
+			}
+			n, err = writeSample(
+				w, name, "_sum", metric, "", 0,
+				metric.Summary.GetSampleSum(),
+			)
+			written += n
+			if err != nil {
+				return
+			}
+			n, err = writeSample(
+				w, name, "_count", metric, "", 0,
+				float64(metric.Summary.GetSampleCount()),
+			)
+		case dto.MetricType_HISTOGRAM:
+			if metric.Histogram == nil {
+				return written, fmt.Errorf(
+					"expected histogram in metric %s %s", name, metric,
+				)
+			}
+			infSeen := false
+			for _, b := range metric.Histogram.Bucket {
+				n, err = writeSample(
+					w, name, "_bucket", metric,
+					model.BucketLabel, b.GetUpperBound(),
+					float64(b.GetCumulativeCount()),
+				)
+				written += n
+				if err != nil {
+					return
+				}
+				if math.IsInf(b.GetUpperBound(), +1) {
+					infSeen = true
+				}
+			}
+			if !infSeen {
+				n, err = writeSample(
+					w, name, "_bucket", metric,
+					model.BucketLabel, math.Inf(+1),
+					float64(metric.Histogram.GetSampleCount()),
+				)
+				written += n
+				if err != nil {
+					return
+				}
+			}
+			n, err = writeSample(
+				w, name, "_sum", metric, "", 0,
+				metric.Histogram.GetSampleSum(),
+			)
+			written += n
+			if err != nil {
+				return
+			}
+			n, err = writeSample(
+				w, name, "_count", metric, "", 0,
+				float64(metric.Histogram.GetSampleCount()),
+			)
+		default:
+			return written, fmt.Errorf(
+				"unexpected type in metric %s %s", name, metric,
+			)
+		}
+		written += n
+		if err != nil {
+			return
+		}
+	}
+	return
+}
+
+// writeSample writes a single sample in text format to w, given the metric
+// name, the metric proto message itself, optionally an additional label name
+// with a float64 value (use empty string as label name if not required), and
+// the value. The function returns the number of bytes written and any error
+// encountered.
+func writeSample(
+	w enhancedWriter,
+	name, suffix string,
+	metric *dto.Metric,
+	additionalLabelName string, additionalLabelValue float64,
+	value float64,
+) (int, error) {
+	var written int
+	n, err := w.WriteString(name)
+	written += n
+	if err != nil {
+		return written, err
+	}
+	if suffix != "" {
+		n, err = w.WriteString(suffix)
+		written += n
+		if err != nil {
+			return written, err
+		}
+	}
+	n, err = writeLabelPairs(
+		w, metric.Label, additionalLabelName, additionalLabelValue,
+	)
+	written += n
+	if err != nil {
+		return written, err
+	}
+	err = w.WriteByte(' ')
+	written++
+	if err != nil {
+		return written, err
+	}
+	n, err = writeFloat(w, value)
+	written += n
+	if err != nil {
+		return written, err
+	}
+	if metric.TimestampMs != nil {
+		err = w.WriteByte(' ')
+		written++
+		if err != nil {
+			return written, err
+		}
+		n, err = writeInt(w, *metric.TimestampMs)
+		written += n
+		if err != nil {
+			return written, err
+		}
+	}
+	err = w.WriteByte('\n')
+	written++
+	if err != nil {
+		return written, err
+	}
+	return written, nil
+}
+
+// writeLabelPairs converts a slice of LabelPair proto messages plus the
+// explicitly given additional label pair into text formatted as required by the
+// text format and writes it to 'w'. An empty slice in combination with an empty
+// string 'additionalLabelName' results in nothing being written. Otherwise, the
+// label pairs are written, escaped as required by the text format, and enclosed
+// in '{...}'. The function returns the number of bytes written and any error
+// encountered.
+func writeLabelPairs(
+	w enhancedWriter,
+	in []*dto.LabelPair,
+	additionalLabelName string, additionalLabelValue float64,
+) (int, error) {
+	if len(in) == 0 && additionalLabelName == "" {
+		return 0, nil
+	}
+	var (
+		written   int
+		separator byte = '{'
+	)
+	for _, lp := range in {
+		err := w.WriteByte(separator)
+		written++
+		if err != nil {
+			return written, err
+		}
+		n, err := w.WriteString(lp.GetName())
+		written += n
+		if err != nil {
+			return written, err
+		}
+		n, err = w.WriteString(`="`)
+		written += n
+		if err != nil {
+			return written, err
+		}
+		n, err = writeEscapedString(w, lp.GetValue(), true)
+		written += n
+		if err != nil {
+			return written, err
+		}
+		err = w.WriteByte('"')
+		written++
+		if err != nil {
+			return written, err
+		}
+		separator = ','
+	}
+	if additionalLabelName != "" {
+		err := w.WriteByte(separator)
+		written++
+		if err != nil {
+			return written, err
+		}
+		n, err := w.WriteString(additionalLabelName)
+		written += n
+		if err != nil {
+			return written, err
+		}
+		n, err = w.WriteString(`="`)
+		written += n
+		if err != nil {
+			return written, err
+		}
+		n, err = writeFloat(w, additionalLabelValue)
+		written += n
+		if err != nil {
+			return written, err
+		}
+		err = w.WriteByte('"')
+		written++
+		if err != nil {
+			return written, err
+		}
+	}
+	err := w.WriteByte('}')
+	written++
+	if err != nil {
+		return written, err
+	}
+	return written, nil
+}
+
+// writeEscapedString replaces '\' by '\\', new line character by '\n', and - if
+// includeDoubleQuote is true - '"' by '\"'.
+var (
+	escaper       = strings.NewReplacer("\\", `\\`, "\n", `\n`)
+	quotedEscaper = strings.NewReplacer("\\", `\\`, "\n", `\n`, "\"", `\"`)
+)
+
+func writeEscapedString(w enhancedWriter, v string, includeDoubleQuote bool) (int, error) {
+	if includeDoubleQuote {
+		return quotedEscaper.WriteString(w, v)
+	} else {
+		return escaper.WriteString(w, v)
+	}
+}
+
+// writeFloat is equivalent to fmt.Fprint with a float64 argument but hardcodes
+// a few common cases for increased efficiency. For non-hardcoded cases, it uses
+// strconv.AppendFloat to avoid allocations, similar to writeInt.
+func writeFloat(w enhancedWriter, f float64) (int, error) {
+	switch {
+	case f == 1:
+		return 1, w.WriteByte('1')
+	case f == 0:
+		return 1, w.WriteByte('0')
+	case f == -1:
+		return w.WriteString("-1")
+	case math.IsNaN(f):
+		return w.WriteString("NaN")
+	case math.IsInf(f, +1):
+		return w.WriteString("+Inf")
+	case math.IsInf(f, -1):
+		return w.WriteString("-Inf")
+	default:
+		bp := numBufPool.Get().(*[]byte)
+		*bp = strconv.AppendFloat((*bp)[:0], f, 'g', -1, 64)
+		written, err := w.Write(*bp)
+		numBufPool.Put(bp)
+		return written, err
+	}
+}
+
+// writeInt is equivalent to fmt.Fprint with an int64 argument but uses
+// strconv.AppendInt with a byte slice taken from a sync.Pool to avoid
+// allocations.
+func writeInt(w enhancedWriter, i int64) (int, error) {
+	bp := numBufPool.Get().(*[]byte)
+	*bp = strconv.AppendInt((*bp)[:0], i, 10)
+	written, err := w.Write(*bp)
+	numBufPool.Put(bp)
+	return written, err
+}
diff --git a/vendor/github.com/prometheus/common/expfmt/text_parse.go b/vendor/github.com/prometheus/common/expfmt/text_parse.go
new file mode 100644
index 0000000..ec3d86b
--- /dev/null
+++ b/vendor/github.com/prometheus/common/expfmt/text_parse.go
@@ -0,0 +1,757 @@
+// Copyright 2014 The Prometheus 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 expfmt
+
+import (
+	"bufio"
+	"bytes"
+	"fmt"
+	"io"
+	"math"
+	"strconv"
+	"strings"
+
+	dto "github.com/prometheus/client_model/go"
+
+	"github.com/golang/protobuf/proto"
+	"github.com/prometheus/common/model"
+)
+
+// A stateFn is a function that represents a state in a state machine. By
+// executing it, the state is progressed to the next state. The stateFn returns
+// another stateFn, which represents the new state. The end state is represented
+// by nil.
+type stateFn func() stateFn
+
+// ParseError signals errors while parsing the simple and flat text-based
+// exchange format.
+type ParseError struct {
+	Line int
+	Msg  string
+}
+
+// Error implements the error interface.
+func (e ParseError) Error() string {
+	return fmt.Sprintf("text format parsing error in line %d: %s", e.Line, e.Msg)
+}
+
+// TextParser is used to parse the simple and flat text-based exchange format. Its
+// zero value is ready to use.
+type TextParser struct {
+	metricFamiliesByName map[string]*dto.MetricFamily
+	buf                  *bufio.Reader // Where the parsed input is read through.
+	err                  error         // Most recent error.
+	lineCount            int           // Tracks the line count for error messages.
+	currentByte          byte          // The most recent byte read.
+	currentToken         bytes.Buffer  // Re-used each time a token has to be gathered from multiple bytes.
+	currentMF            *dto.MetricFamily
+	currentMetric        *dto.Metric
+	currentLabelPair     *dto.LabelPair
+
+	// The remaining member variables are only used for summaries/histograms.
+	currentLabels map[string]string // All labels including '__name__' but excluding 'quantile'/'le'
+	// Summary specific.
+	summaries       map[uint64]*dto.Metric // Key is created with LabelsToSignature.
+	currentQuantile float64
+	// Histogram specific.
+	histograms    map[uint64]*dto.Metric // Key is created with LabelsToSignature.
+	currentBucket float64
+	// These tell us if the currently processed line ends on '_count' or
+	// '_sum' respectively and belong to a summary/histogram, representing the sample
+	// count and sum of that summary/histogram.
+	currentIsSummaryCount, currentIsSummarySum     bool
+	currentIsHistogramCount, currentIsHistogramSum bool
+}
+
+// TextToMetricFamilies reads 'in' as the simple and flat text-based exchange
+// format and creates MetricFamily proto messages. It returns the MetricFamily
+// proto messages in a map where the metric names are the keys, along with any
+// error encountered.
+//
+// If the input contains duplicate metrics (i.e. lines with the same metric name
+// and exactly the same label set), the resulting MetricFamily will contain
+// duplicate Metric proto messages. Similar is true for duplicate label
+// names. Checks for duplicates have to be performed separately, if required.
+// Also note that neither the metrics within each MetricFamily are sorted nor
+// the label pairs within each Metric. Sorting is not required for the most
+// frequent use of this method, which is sample ingestion in the Prometheus
+// server. However, for presentation purposes, you might want to sort the
+// metrics, and in some cases, you must sort the labels, e.g. for consumption by
+// the metric family injection hook of the Prometheus registry.
+//
+// Summaries and histograms are rather special beasts. You would probably not
+// use them in the simple text format anyway. This method can deal with
+// summaries and histograms if they are presented in exactly the way the
+// text.Create function creates them.
+//
+// This method must not be called concurrently. If you want to parse different
+// input concurrently, instantiate a separate Parser for each goroutine.
+func (p *TextParser) TextToMetricFamilies(in io.Reader) (map[string]*dto.MetricFamily, error) {
+	p.reset(in)
+	for nextState := p.startOfLine; nextState != nil; nextState = nextState() {
+		// Magic happens here...
+	}
+	// Get rid of empty metric families.
+	for k, mf := range p.metricFamiliesByName {
+		if len(mf.GetMetric()) == 0 {
+			delete(p.metricFamiliesByName, k)
+		}
+	}
+	// If p.err is io.EOF now, we have run into a premature end of the input
+	// stream. Turn this error into something nicer and more
+	// meaningful. (io.EOF is often used as a signal for the legitimate end
+	// of an input stream.)
+	if p.err == io.EOF {
+		p.parseError("unexpected end of input stream")
+	}
+	return p.metricFamiliesByName, p.err
+}
+
+func (p *TextParser) reset(in io.Reader) {
+	p.metricFamiliesByName = map[string]*dto.MetricFamily{}
+	if p.buf == nil {
+		p.buf = bufio.NewReader(in)
+	} else {
+		p.buf.Reset(in)
+	}
+	p.err = nil
+	p.lineCount = 0
+	if p.summaries == nil || len(p.summaries) > 0 {
+		p.summaries = map[uint64]*dto.Metric{}
+	}
+	if p.histograms == nil || len(p.histograms) > 0 {
+		p.histograms = map[uint64]*dto.Metric{}
+	}
+	p.currentQuantile = math.NaN()
+	p.currentBucket = math.NaN()
+}
+
+// startOfLine represents the state where the next byte read from p.buf is the
+// start of a line (or whitespace leading up to it).
+func (p *TextParser) startOfLine() stateFn {
+	p.lineCount++
+	if p.skipBlankTab(); p.err != nil {
+		// End of input reached. This is the only case where
+		// that is not an error but a signal that we are done.
+		p.err = nil
+		return nil
+	}
+	switch p.currentByte {
+	case '#':
+		return p.startComment
+	case '\n':
+		return p.startOfLine // Empty line, start the next one.
+	}
+	return p.readingMetricName
+}
+
+// startComment represents the state where the next byte read from p.buf is the
+// start of a comment (or whitespace leading up to it).
+func (p *TextParser) startComment() stateFn {
+	if p.skipBlankTab(); p.err != nil {
+		return nil // Unexpected end of input.
+	}
+	if p.currentByte == '\n' {
+		return p.startOfLine
+	}
+	if p.readTokenUntilWhitespace(); p.err != nil {
+		return nil // Unexpected end of input.
+	}
+	// If we have hit the end of line already, there is nothing left
+	// to do. This is not considered a syntax error.
+	if p.currentByte == '\n' {
+		return p.startOfLine
+	}
+	keyword := p.currentToken.String()
+	if keyword != "HELP" && keyword != "TYPE" {
+		// Generic comment, ignore by fast forwarding to end of line.
+		for p.currentByte != '\n' {
+			if p.currentByte, p.err = p.buf.ReadByte(); p.err != nil {
+				return nil // Unexpected end of input.
+			}
+		}
+		return p.startOfLine
+	}
+	// There is something. Next has to be a metric name.
+	if p.skipBlankTab(); p.err != nil {
+		return nil // Unexpected end of input.
+	}
+	if p.readTokenAsMetricName(); p.err != nil {
+		return nil // Unexpected end of input.
+	}
+	if p.currentByte == '\n' {
+		// At the end of the line already.
+		// Again, this is not considered a syntax error.
+		return p.startOfLine
+	}
+	if !isBlankOrTab(p.currentByte) {
+		p.parseError("invalid metric name in comment")
+		return nil
+	}
+	p.setOrCreateCurrentMF()
+	if p.skipBlankTab(); p.err != nil {
+		return nil // Unexpected end of input.
+	}
+	if p.currentByte == '\n' {
+		// At the end of the line already.
+		// Again, this is not considered a syntax error.
+		return p.startOfLine
+	}
+	switch keyword {
+	case "HELP":
+		return p.readingHelp
+	case "TYPE":
+		return p.readingType
+	}
+	panic(fmt.Sprintf("code error: unexpected keyword %q", keyword))
+}
+
+// readingMetricName represents the state where the last byte read (now in
+// p.currentByte) is the first byte of a metric name.
+func (p *TextParser) readingMetricName() stateFn {
+	if p.readTokenAsMetricName(); p.err != nil {
+		return nil
+	}
+	if p.currentToken.Len() == 0 {
+		p.parseError("invalid metric name")
+		return nil
+	}
+	p.setOrCreateCurrentMF()
+	// Now is the time to fix the type if it hasn't happened yet.
+	if p.currentMF.Type == nil {
+		p.currentMF.Type = dto.MetricType_UNTYPED.Enum()
+	}
+	p.currentMetric = &dto.Metric{}
+	// Do not append the newly created currentMetric to
+	// currentMF.Metric right now. First wait if this is a summary,
+	// and the metric exists already, which we can only know after
+	// having read all the labels.
+	if p.skipBlankTabIfCurrentBlankTab(); p.err != nil {
+		return nil // Unexpected end of input.
+	}
+	return p.readingLabels
+}
+
+// readingLabels represents the state where the last byte read (now in
+// p.currentByte) is either the first byte of the label set (i.e. a '{'), or the
+// first byte of the value (otherwise).
+func (p *TextParser) readingLabels() stateFn {
+	// Summaries/histograms are special. We have to reset the
+	// currentLabels map, currentQuantile and currentBucket before starting to
+	// read labels.
+	if p.currentMF.GetType() == dto.MetricType_SUMMARY || p.currentMF.GetType() == dto.MetricType_HISTOGRAM {
+		p.currentLabels = map[string]string{}
+		p.currentLabels[string(model.MetricNameLabel)] = p.currentMF.GetName()
+		p.currentQuantile = math.NaN()
+		p.currentBucket = math.NaN()
+	}
+	if p.currentByte != '{' {
+		return p.readingValue
+	}
+	return p.startLabelName
+}
+
+// startLabelName represents the state where the next byte read from p.buf is
+// the start of a label name (or whitespace leading up to it).
+func (p *TextParser) startLabelName() stateFn {
+	if p.skipBlankTab(); p.err != nil {
+		return nil // Unexpected end of input.
+	}
+	if p.currentByte == '}' {
+		if p.skipBlankTab(); p.err != nil {
+			return nil // Unexpected end of input.
+		}
+		return p.readingValue
+	}
+	if p.readTokenAsLabelName(); p.err != nil {
+		return nil // Unexpected end of input.
+	}
+	if p.currentToken.Len() == 0 {
+		p.parseError(fmt.Sprintf("invalid label name for metric %q", p.currentMF.GetName()))
+		return nil
+	}
+	p.currentLabelPair = &dto.LabelPair{Name: proto.String(p.currentToken.String())}
+	if p.currentLabelPair.GetName() == string(model.MetricNameLabel) {
+		p.parseError(fmt.Sprintf("label name %q is reserved", model.MetricNameLabel))
+		return nil
+	}
+	// Special summary/histogram treatment. Don't add 'quantile' and 'le'
+	// labels to 'real' labels.
+	if !(p.currentMF.GetType() == dto.MetricType_SUMMARY && p.currentLabelPair.GetName() == model.QuantileLabel) &&
+		!(p.currentMF.GetType() == dto.MetricType_HISTOGRAM && p.currentLabelPair.GetName() == model.BucketLabel) {
+		p.currentMetric.Label = append(p.currentMetric.Label, p.currentLabelPair)
+	}
+	if p.skipBlankTabIfCurrentBlankTab(); p.err != nil {
+		return nil // Unexpected end of input.
+	}
+	if p.currentByte != '=' {
+		p.parseError(fmt.Sprintf("expected '=' after label name, found %q", p.currentByte))
+		return nil
+	}
+	return p.startLabelValue
+}
+
+// startLabelValue represents the state where the next byte read from p.buf is
+// the start of a (quoted) label value (or whitespace leading up to it).
+func (p *TextParser) startLabelValue() stateFn {
+	if p.skipBlankTab(); p.err != nil {
+		return nil // Unexpected end of input.
+	}
+	if p.currentByte != '"' {
+		p.parseError(fmt.Sprintf("expected '\"' at start of label value, found %q", p.currentByte))
+		return nil
+	}
+	if p.readTokenAsLabelValue(); p.err != nil {
+		return nil
+	}
+	if !model.LabelValue(p.currentToken.String()).IsValid() {
+		p.parseError(fmt.Sprintf("invalid label value %q", p.currentToken.String()))
+		return nil
+	}
+	p.currentLabelPair.Value = proto.String(p.currentToken.String())
+	// Special treatment of summaries:
+	// - Quantile labels are special, will result in dto.Quantile later.
+	// - Other labels have to be added to currentLabels for signature calculation.
+	if p.currentMF.GetType() == dto.MetricType_SUMMARY {
+		if p.currentLabelPair.GetName() == model.QuantileLabel {
+			if p.currentQuantile, p.err = strconv.ParseFloat(p.currentLabelPair.GetValue(), 64); p.err != nil {
+				// Create a more helpful error message.
+				p.parseError(fmt.Sprintf("expected float as value for 'quantile' label, got %q", p.currentLabelPair.GetValue()))
+				return nil
+			}
+		} else {
+			p.currentLabels[p.currentLabelPair.GetName()] = p.currentLabelPair.GetValue()
+		}
+	}
+	// Similar special treatment of histograms.
+	if p.currentMF.GetType() == dto.MetricType_HISTOGRAM {
+		if p.currentLabelPair.GetName() == model.BucketLabel {
+			if p.currentBucket, p.err = strconv.ParseFloat(p.currentLabelPair.GetValue(), 64); p.err != nil {
+				// Create a more helpful error message.
+				p.parseError(fmt.Sprintf("expected float as value for 'le' label, got %q", p.currentLabelPair.GetValue()))
+				return nil
+			}
+		} else {
+			p.currentLabels[p.currentLabelPair.GetName()] = p.currentLabelPair.GetValue()
+		}
+	}
+	if p.skipBlankTab(); p.err != nil {
+		return nil // Unexpected end of input.
+	}
+	switch p.currentByte {
+	case ',':
+		return p.startLabelName
+
+	case '}':
+		if p.skipBlankTab(); p.err != nil {
+			return nil // Unexpected end of input.
+		}
+		return p.readingValue
+	default:
+		p.parseError(fmt.Sprintf("unexpected end of label value %q", p.currentLabelPair.GetValue()))
+		return nil
+	}
+}
+
+// readingValue represents the state where the last byte read (now in
+// p.currentByte) is the first byte of the sample value (i.e. a float).
+func (p *TextParser) readingValue() stateFn {
+	// When we are here, we have read all the labels, so for the
+	// special case of a summary/histogram, we can finally find out
+	// if the metric already exists.
+	if p.currentMF.GetType() == dto.MetricType_SUMMARY {
+		signature := model.LabelsToSignature(p.currentLabels)
+		if summary := p.summaries[signature]; summary != nil {
+			p.currentMetric = summary
+		} else {
+			p.summaries[signature] = p.currentMetric
+			p.currentMF.Metric = append(p.currentMF.Metric, p.currentMetric)
+		}
+	} else if p.currentMF.GetType() == dto.MetricType_HISTOGRAM {
+		signature := model.LabelsToSignature(p.currentLabels)
+		if histogram := p.histograms[signature]; histogram != nil {
+			p.currentMetric = histogram
+		} else {
+			p.histograms[signature] = p.currentMetric
+			p.currentMF.Metric = append(p.currentMF.Metric, p.currentMetric)
+		}
+	} else {
+		p.currentMF.Metric = append(p.currentMF.Metric, p.currentMetric)
+	}
+	if p.readTokenUntilWhitespace(); p.err != nil {
+		return nil // Unexpected end of input.
+	}
+	value, err := strconv.ParseFloat(p.currentToken.String(), 64)
+	if err != nil {
+		// Create a more helpful error message.
+		p.parseError(fmt.Sprintf("expected float as value, got %q", p.currentToken.String()))
+		return nil
+	}
+	switch p.currentMF.GetType() {
+	case dto.MetricType_COUNTER:
+		p.currentMetric.Counter = &dto.Counter{Value: proto.Float64(value)}
+	case dto.MetricType_GAUGE:
+		p.currentMetric.Gauge = &dto.Gauge{Value: proto.Float64(value)}
+	case dto.MetricType_UNTYPED:
+		p.currentMetric.Untyped = &dto.Untyped{Value: proto.Float64(value)}
+	case dto.MetricType_SUMMARY:
+		// *sigh*
+		if p.currentMetric.Summary == nil {
+			p.currentMetric.Summary = &dto.Summary{}
+		}
+		switch {
+		case p.currentIsSummaryCount:
+			p.currentMetric.Summary.SampleCount = proto.Uint64(uint64(value))
+		case p.currentIsSummarySum:
+			p.currentMetric.Summary.SampleSum = proto.Float64(value)
+		case !math.IsNaN(p.currentQuantile):
+			p.currentMetric.Summary.Quantile = append(
+				p.currentMetric.Summary.Quantile,
+				&dto.Quantile{
+					Quantile: proto.Float64(p.currentQuantile),
+					Value:    proto.Float64(value),
+				},
+			)
+		}
+	case dto.MetricType_HISTOGRAM:
+		// *sigh*
+		if p.currentMetric.Histogram == nil {
+			p.currentMetric.Histogram = &dto.Histogram{}
+		}
+		switch {
+		case p.currentIsHistogramCount:
+			p.currentMetric.Histogram.SampleCount = proto.Uint64(uint64(value))
+		case p.currentIsHistogramSum:
+			p.currentMetric.Histogram.SampleSum = proto.Float64(value)
+		case !math.IsNaN(p.currentBucket):
+			p.currentMetric.Histogram.Bucket = append(
+				p.currentMetric.Histogram.Bucket,
+				&dto.Bucket{
+					UpperBound:      proto.Float64(p.currentBucket),
+					CumulativeCount: proto.Uint64(uint64(value)),
+				},
+			)
+		}
+	default:
+		p.err = fmt.Errorf("unexpected type for metric name %q", p.currentMF.GetName())
+	}
+	if p.currentByte == '\n' {
+		return p.startOfLine
+	}
+	return p.startTimestamp
+}
+
+// startTimestamp represents the state where the next byte read from p.buf is
+// the start of the timestamp (or whitespace leading up to it).
+func (p *TextParser) startTimestamp() stateFn {
+	if p.skipBlankTab(); p.err != nil {
+		return nil // Unexpected end of input.
+	}
+	if p.readTokenUntilWhitespace(); p.err != nil {
+		return nil // Unexpected end of input.
+	}
+	timestamp, err := strconv.ParseInt(p.currentToken.String(), 10, 64)
+	if err != nil {
+		// Create a more helpful error message.
+		p.parseError(fmt.Sprintf("expected integer as timestamp, got %q", p.currentToken.String()))
+		return nil
+	}
+	p.currentMetric.TimestampMs = proto.Int64(timestamp)
+	if p.readTokenUntilNewline(false); p.err != nil {
+		return nil // Unexpected end of input.
+	}
+	if p.currentToken.Len() > 0 {
+		p.parseError(fmt.Sprintf("spurious string after timestamp: %q", p.currentToken.String()))
+		return nil
+	}
+	return p.startOfLine
+}
+
+// readingHelp represents the state where the last byte read (now in
+// p.currentByte) is the first byte of the docstring after 'HELP'.
+func (p *TextParser) readingHelp() stateFn {
+	if p.currentMF.Help != nil {
+		p.parseError(fmt.Sprintf("second HELP line for metric name %q", p.currentMF.GetName()))
+		return nil
+	}
+	// Rest of line is the docstring.
+	if p.readTokenUntilNewline(true); p.err != nil {
+		return nil // Unexpected end of input.
+	}
+	p.currentMF.Help = proto.String(p.currentToken.String())
+	return p.startOfLine
+}
+
+// readingType represents the state where the last byte read (now in
+// p.currentByte) is the first byte of the type hint after 'HELP'.
+func (p *TextParser) readingType() stateFn {
+	if p.currentMF.Type != nil {
+		p.parseError(fmt.Sprintf("second TYPE line for metric name %q, or TYPE reported after samples", p.currentMF.GetName()))
+		return nil
+	}
+	// Rest of line is the type.
+	if p.readTokenUntilNewline(false); p.err != nil {
+		return nil // Unexpected end of input.
+	}
+	metricType, ok := dto.MetricType_value[strings.ToUpper(p.currentToken.String())]
+	if !ok {
+		p.parseError(fmt.Sprintf("unknown metric type %q", p.currentToken.String()))
+		return nil
+	}
+	p.currentMF.Type = dto.MetricType(metricType).Enum()
+	return p.startOfLine
+}
+
+// parseError sets p.err to a ParseError at the current line with the given
+// message.
+func (p *TextParser) parseError(msg string) {
+	p.err = ParseError{
+		Line: p.lineCount,
+		Msg:  msg,
+	}
+}
+
+// skipBlankTab reads (and discards) bytes from p.buf until it encounters a byte
+// that is neither ' ' nor '\t'. That byte is left in p.currentByte.
+func (p *TextParser) skipBlankTab() {
+	for {
+		if p.currentByte, p.err = p.buf.ReadByte(); p.err != nil || !isBlankOrTab(p.currentByte) {
+			return
+		}
+	}
+}
+
+// skipBlankTabIfCurrentBlankTab works exactly as skipBlankTab but doesn't do
+// anything if p.currentByte is neither ' ' nor '\t'.
+func (p *TextParser) skipBlankTabIfCurrentBlankTab() {
+	if isBlankOrTab(p.currentByte) {
+		p.skipBlankTab()
+	}
+}
+
+// readTokenUntilWhitespace copies bytes from p.buf into p.currentToken.  The
+// first byte considered is the byte already read (now in p.currentByte).  The
+// first whitespace byte encountered is still copied into p.currentByte, but not
+// into p.currentToken.
+func (p *TextParser) readTokenUntilWhitespace() {
+	p.currentToken.Reset()
+	for p.err == nil && !isBlankOrTab(p.currentByte) && p.currentByte != '\n' {
+		p.currentToken.WriteByte(p.currentByte)
+		p.currentByte, p.err = p.buf.ReadByte()
+	}
+}
+
+// readTokenUntilNewline copies bytes from p.buf into p.currentToken.  The first
+// byte considered is the byte already read (now in p.currentByte).  The first
+// newline byte encountered is still copied into p.currentByte, but not into
+// p.currentToken. If recognizeEscapeSequence is true, two escape sequences are
+// recognized: '\\' translates into '\', and '\n' into a line-feed character.
+// All other escape sequences are invalid and cause an error.
+func (p *TextParser) readTokenUntilNewline(recognizeEscapeSequence bool) {
+	p.currentToken.Reset()
+	escaped := false
+	for p.err == nil {
+		if recognizeEscapeSequence && escaped {
+			switch p.currentByte {
+			case '\\':
+				p.currentToken.WriteByte(p.currentByte)
+			case 'n':
+				p.currentToken.WriteByte('\n')
+			default:
+				p.parseError(fmt.Sprintf("invalid escape sequence '\\%c'", p.currentByte))
+				return
+			}
+			escaped = false
+		} else {
+			switch p.currentByte {
+			case '\n':
+				return
+			case '\\':
+				escaped = true
+			default:
+				p.currentToken.WriteByte(p.currentByte)
+			}
+		}
+		p.currentByte, p.err = p.buf.ReadByte()
+	}
+}
+
+// readTokenAsMetricName copies a metric name from p.buf into p.currentToken.
+// The first byte considered is the byte already read (now in p.currentByte).
+// The first byte not part of a metric name is still copied into p.currentByte,
+// but not into p.currentToken.
+func (p *TextParser) readTokenAsMetricName() {
+	p.currentToken.Reset()
+	if !isValidMetricNameStart(p.currentByte) {
+		return
+	}
+	for {
+		p.currentToken.WriteByte(p.currentByte)
+		p.currentByte, p.err = p.buf.ReadByte()
+		if p.err != nil || !isValidMetricNameContinuation(p.currentByte) {
+			return
+		}
+	}
+}
+
+// readTokenAsLabelName copies a label name from p.buf into p.currentToken.
+// The first byte considered is the byte already read (now in p.currentByte).
+// The first byte not part of a label name is still copied into p.currentByte,
+// but not into p.currentToken.
+func (p *TextParser) readTokenAsLabelName() {
+	p.currentToken.Reset()
+	if !isValidLabelNameStart(p.currentByte) {
+		return
+	}
+	for {
+		p.currentToken.WriteByte(p.currentByte)
+		p.currentByte, p.err = p.buf.ReadByte()
+		if p.err != nil || !isValidLabelNameContinuation(p.currentByte) {
+			return
+		}
+	}
+}
+
+// readTokenAsLabelValue copies a label value from p.buf into p.currentToken.
+// In contrast to the other 'readTokenAs...' functions, which start with the
+// last read byte in p.currentByte, this method ignores p.currentByte and starts
+// with reading a new byte from p.buf. The first byte not part of a label value
+// is still copied into p.currentByte, but not into p.currentToken.
+func (p *TextParser) readTokenAsLabelValue() {
+	p.currentToken.Reset()
+	escaped := false
+	for {
+		if p.currentByte, p.err = p.buf.ReadByte(); p.err != nil {
+			return
+		}
+		if escaped {
+			switch p.currentByte {
+			case '"', '\\':
+				p.currentToken.WriteByte(p.currentByte)
+			case 'n':
+				p.currentToken.WriteByte('\n')
+			default:
+				p.parseError(fmt.Sprintf("invalid escape sequence '\\%c'", p.currentByte))
+				return
+			}
+			escaped = false
+			continue
+		}
+		switch p.currentByte {
+		case '"':
+			return
+		case '\n':
+			p.parseError(fmt.Sprintf("label value %q contains unescaped new-line", p.currentToken.String()))
+			return
+		case '\\':
+			escaped = true
+		default:
+			p.currentToken.WriteByte(p.currentByte)
+		}
+	}
+}
+
+func (p *TextParser) setOrCreateCurrentMF() {
+	p.currentIsSummaryCount = false
+	p.currentIsSummarySum = false
+	p.currentIsHistogramCount = false
+	p.currentIsHistogramSum = false
+	name := p.currentToken.String()
+	if p.currentMF = p.metricFamiliesByName[name]; p.currentMF != nil {
+		return
+	}
+	// Try out if this is a _sum or _count for a summary/histogram.
+	summaryName := summaryMetricName(name)
+	if p.currentMF = p.metricFamiliesByName[summaryName]; p.currentMF != nil {
+		if p.currentMF.GetType() == dto.MetricType_SUMMARY {
+			if isCount(name) {
+				p.currentIsSummaryCount = true
+			}
+			if isSum(name) {
+				p.currentIsSummarySum = true
+			}
+			return
+		}
+	}
+	histogramName := histogramMetricName(name)
+	if p.currentMF = p.metricFamiliesByName[histogramName]; p.currentMF != nil {
+		if p.currentMF.GetType() == dto.MetricType_HISTOGRAM {
+			if isCount(name) {
+				p.currentIsHistogramCount = true
+			}
+			if isSum(name) {
+				p.currentIsHistogramSum = true
+			}
+			return
+		}
+	}
+	p.currentMF = &dto.MetricFamily{Name: proto.String(name)}
+	p.metricFamiliesByName[name] = p.currentMF
+}
+
+func isValidLabelNameStart(b byte) bool {
+	return (b >= 'a' && b <= 'z') || (b >= 'A' && b <= 'Z') || b == '_'
+}
+
+func isValidLabelNameContinuation(b byte) bool {
+	return isValidLabelNameStart(b) || (b >= '0' && b <= '9')
+}
+
+func isValidMetricNameStart(b byte) bool {
+	return isValidLabelNameStart(b) || b == ':'
+}
+
+func isValidMetricNameContinuation(b byte) bool {
+	return isValidLabelNameContinuation(b) || b == ':'
+}
+
+func isBlankOrTab(b byte) bool {
+	return b == ' ' || b == '\t'
+}
+
+func isCount(name string) bool {
+	return len(name) > 6 && name[len(name)-6:] == "_count"
+}
+
+func isSum(name string) bool {
+	return len(name) > 4 && name[len(name)-4:] == "_sum"
+}
+
+func isBucket(name string) bool {
+	return len(name) > 7 && name[len(name)-7:] == "_bucket"
+}
+
+func summaryMetricName(name string) string {
+	switch {
+	case isCount(name):
+		return name[:len(name)-6]
+	case isSum(name):
+		return name[:len(name)-4]
+	default:
+		return name
+	}
+}
+
+func histogramMetricName(name string) string {
+	switch {
+	case isCount(name):
+		return name[:len(name)-6]
+	case isSum(name):
+		return name[:len(name)-4]
+	case isBucket(name):
+		return name[:len(name)-7]
+	default:
+		return name
+	}
+}
diff --git a/vendor/github.com/prometheus/common/internal/bitbucket.org/ww/goautoneg/README.txt b/vendor/github.com/prometheus/common/internal/bitbucket.org/ww/goautoneg/README.txt
new file mode 100644
index 0000000..7723656
--- /dev/null
+++ b/vendor/github.com/prometheus/common/internal/bitbucket.org/ww/goautoneg/README.txt
@@ -0,0 +1,67 @@
+PACKAGE
+
+package goautoneg
+import "bitbucket.org/ww/goautoneg"
+
+HTTP Content-Type Autonegotiation.
+
+The functions in this package implement the behaviour specified in
+http://www.w3.org/Protocols/rfc2616/rfc2616-sec14.html
+
+Copyright (c) 2011, Open Knowledge Foundation Ltd.
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    Redistributions of source code must retain the above copyright
+    notice, this list of conditions and the following disclaimer.
+
+    Redistributions in binary form must reproduce the above copyright
+    notice, this list of conditions and the following disclaimer in
+    the documentation and/or other materials provided with the
+    distribution.
+
+    Neither the name of the Open Knowledge Foundation Ltd. nor the
+    names of its contributors may be used to endorse or promote
+    products derived from this software without specific prior written
+    permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+FUNCTIONS
+
+func Negotiate(header string, alternatives []string) (content_type string)
+Negotiate the most appropriate content_type given the accept header
+and a list of alternatives.
+
+func ParseAccept(header string) (accept []Accept)
+Parse an Accept Header string returning a sorted list
+of clauses
+
+
+TYPES
+
+type Accept struct {
+    Type, SubType string
+    Q             float32
+    Params        map[string]string
+}
+Structure to represent a clause in an HTTP Accept Header
+
+
+SUBDIRECTORIES
+
+	.hg
diff --git a/vendor/github.com/prometheus/common/internal/bitbucket.org/ww/goautoneg/autoneg.go b/vendor/github.com/prometheus/common/internal/bitbucket.org/ww/goautoneg/autoneg.go
new file mode 100644
index 0000000..26e9228
--- /dev/null
+++ b/vendor/github.com/prometheus/common/internal/bitbucket.org/ww/goautoneg/autoneg.go
@@ -0,0 +1,162 @@
+/*
+Copyright (c) 2011, Open Knowledge Foundation Ltd.
+All rights reserved.
+
+HTTP Content-Type Autonegotiation.
+
+The functions in this package implement the behaviour specified in
+http://www.w3.org/Protocols/rfc2616/rfc2616-sec14.html
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    Redistributions of source code must retain the above copyright
+    notice, this list of conditions and the following disclaimer.
+
+    Redistributions in binary form must reproduce the above copyright
+    notice, this list of conditions and the following disclaimer in
+    the documentation and/or other materials provided with the
+    distribution.
+
+    Neither the name of the Open Knowledge Foundation Ltd. nor the
+    names of its contributors may be used to endorse or promote
+    products derived from this software without specific prior written
+    permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+*/
+package goautoneg
+
+import (
+	"sort"
+	"strconv"
+	"strings"
+)
+
+// Structure to represent a clause in an HTTP Accept Header
+type Accept struct {
+	Type, SubType string
+	Q             float64
+	Params        map[string]string
+}
+
+// For internal use, so that we can use the sort interface
+type accept_slice []Accept
+
+func (accept accept_slice) Len() int {
+	slice := []Accept(accept)
+	return len(slice)
+}
+
+func (accept accept_slice) Less(i, j int) bool {
+	slice := []Accept(accept)
+	ai, aj := slice[i], slice[j]
+	if ai.Q > aj.Q {
+		return true
+	}
+	if ai.Type != "*" && aj.Type == "*" {
+		return true
+	}
+	if ai.SubType != "*" && aj.SubType == "*" {
+		return true
+	}
+	return false
+}
+
+func (accept accept_slice) Swap(i, j int) {
+	slice := []Accept(accept)
+	slice[i], slice[j] = slice[j], slice[i]
+}
+
+// Parse an Accept Header string returning a sorted list
+// of clauses
+func ParseAccept(header string) (accept []Accept) {
+	parts := strings.Split(header, ",")
+	accept = make([]Accept, 0, len(parts))
+	for _, part := range parts {
+		part := strings.Trim(part, " ")
+
+		a := Accept{}
+		a.Params = make(map[string]string)
+		a.Q = 1.0
+
+		mrp := strings.Split(part, ";")
+
+		media_range := mrp[0]
+		sp := strings.Split(media_range, "/")
+		a.Type = strings.Trim(sp[0], " ")
+
+		switch {
+		case len(sp) == 1 && a.Type == "*":
+			a.SubType = "*"
+		case len(sp) == 2:
+			a.SubType = strings.Trim(sp[1], " ")
+		default:
+			continue
+		}
+
+		if len(mrp) == 1 {
+			accept = append(accept, a)
+			continue
+		}
+
+		for _, param := range mrp[1:] {
+			sp := strings.SplitN(param, "=", 2)
+			if len(sp) != 2 {
+				continue
+			}
+			token := strings.Trim(sp[0], " ")
+			if token == "q" {
+				a.Q, _ = strconv.ParseFloat(sp[1], 32)
+			} else {
+				a.Params[token] = strings.Trim(sp[1], " ")
+			}
+		}
+
+		accept = append(accept, a)
+	}
+
+	slice := accept_slice(accept)
+	sort.Sort(slice)
+
+	return
+}
+
+// Negotiate the most appropriate content_type given the accept header
+// and a list of alternatives.
+func Negotiate(header string, alternatives []string) (content_type string) {
+	asp := make([][]string, 0, len(alternatives))
+	for _, ctype := range alternatives {
+		asp = append(asp, strings.SplitN(ctype, "/", 2))
+	}
+	for _, clause := range ParseAccept(header) {
+		for i, ctsp := range asp {
+			if clause.Type == ctsp[0] && clause.SubType == ctsp[1] {
+				content_type = alternatives[i]
+				return
+			}
+			if clause.Type == ctsp[0] && clause.SubType == "*" {
+				content_type = alternatives[i]
+				return
+			}
+			if clause.Type == "*" && clause.SubType == "*" {
+				content_type = alternatives[i]
+				return
+			}
+		}
+	}
+	return
+}
diff --git a/vendor/github.com/prometheus/common/model/alert.go b/vendor/github.com/prometheus/common/model/alert.go
new file mode 100644
index 0000000..35e739c
--- /dev/null
+++ b/vendor/github.com/prometheus/common/model/alert.go
@@ -0,0 +1,136 @@
+// Copyright 2013 The Prometheus 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 model
+
+import (
+	"fmt"
+	"time"
+)
+
+type AlertStatus string
+
+const (
+	AlertFiring   AlertStatus = "firing"
+	AlertResolved AlertStatus = "resolved"
+)
+
+// Alert is a generic representation of an alert in the Prometheus eco-system.
+type Alert struct {
+	// Label value pairs for purpose of aggregation, matching, and disposition
+	// dispatching. This must minimally include an "alertname" label.
+	Labels LabelSet `json:"labels"`
+
+	// Extra key/value information which does not define alert identity.
+	Annotations LabelSet `json:"annotations"`
+
+	// The known time range for this alert. Both ends are optional.
+	StartsAt     time.Time `json:"startsAt,omitempty"`
+	EndsAt       time.Time `json:"endsAt,omitempty"`
+	GeneratorURL string    `json:"generatorURL"`
+}
+
+// Name returns the name of the alert. It is equivalent to the "alertname" label.
+func (a *Alert) Name() string {
+	return string(a.Labels[AlertNameLabel])
+}
+
+// Fingerprint returns a unique hash for the alert. It is equivalent to
+// the fingerprint of the alert's label set.
+func (a *Alert) Fingerprint() Fingerprint {
+	return a.Labels.Fingerprint()
+}
+
+func (a *Alert) String() string {
+	s := fmt.Sprintf("%s[%s]", a.Name(), a.Fingerprint().String()[:7])
+	if a.Resolved() {
+		return s + "[resolved]"
+	}
+	return s + "[active]"
+}
+
+// Resolved returns true iff the activity interval ended in the past.
+func (a *Alert) Resolved() bool {
+	return a.ResolvedAt(time.Now())
+}
+
+// ResolvedAt returns true off the activity interval ended before
+// the given timestamp.
+func (a *Alert) ResolvedAt(ts time.Time) bool {
+	if a.EndsAt.IsZero() {
+		return false
+	}
+	return !a.EndsAt.After(ts)
+}
+
+// Status returns the status of the alert.
+func (a *Alert) Status() AlertStatus {
+	if a.Resolved() {
+		return AlertResolved
+	}
+	return AlertFiring
+}
+
+// Validate checks whether the alert data is inconsistent.
+func (a *Alert) Validate() error {
+	if a.StartsAt.IsZero() {
+		return fmt.Errorf("start time missing")
+	}
+	if !a.EndsAt.IsZero() && a.EndsAt.Before(a.StartsAt) {
+		return fmt.Errorf("start time must be before end time")
+	}
+	if err := a.Labels.Validate(); err != nil {
+		return fmt.Errorf("invalid label set: %s", err)
+	}
+	if len(a.Labels) == 0 {
+		return fmt.Errorf("at least one label pair required")
+	}
+	if err := a.Annotations.Validate(); err != nil {
+		return fmt.Errorf("invalid annotations: %s", err)
+	}
+	return nil
+}
+
+// Alert is a list of alerts that can be sorted in chronological order.
+type Alerts []*Alert
+
+func (as Alerts) Len() int      { return len(as) }
+func (as Alerts) Swap(i, j int) { as[i], as[j] = as[j], as[i] }
+
+func (as Alerts) Less(i, j int) bool {
+	if as[i].StartsAt.Before(as[j].StartsAt) {
+		return true
+	}
+	if as[i].EndsAt.Before(as[j].EndsAt) {
+		return true
+	}
+	return as[i].Fingerprint() < as[j].Fingerprint()
+}
+
+// HasFiring returns true iff one of the alerts is not resolved.
+func (as Alerts) HasFiring() bool {
+	for _, a := range as {
+		if !a.Resolved() {
+			return true
+		}
+	}
+	return false
+}
+
+// Status returns StatusFiring iff at least one of the alerts is firing.
+func (as Alerts) Status() AlertStatus {
+	if as.HasFiring() {
+		return AlertFiring
+	}
+	return AlertResolved
+}
diff --git a/vendor/github.com/prometheus/common/model/fingerprinting.go b/vendor/github.com/prometheus/common/model/fingerprinting.go
new file mode 100644
index 0000000..fc4de41
--- /dev/null
+++ b/vendor/github.com/prometheus/common/model/fingerprinting.go
@@ -0,0 +1,105 @@
+// Copyright 2013 The Prometheus 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 model
+
+import (
+	"fmt"
+	"strconv"
+)
+
+// Fingerprint provides a hash-capable representation of a Metric.
+// For our purposes, FNV-1A 64-bit is used.
+type Fingerprint uint64
+
+// FingerprintFromString transforms a string representation into a Fingerprint.
+func FingerprintFromString(s string) (Fingerprint, error) {
+	num, err := strconv.ParseUint(s, 16, 64)
+	return Fingerprint(num), err
+}
+
+// ParseFingerprint parses the input string into a fingerprint.
+func ParseFingerprint(s string) (Fingerprint, error) {
+	num, err := strconv.ParseUint(s, 16, 64)
+	if err != nil {
+		return 0, err
+	}
+	return Fingerprint(num), nil
+}
+
+func (f Fingerprint) String() string {
+	return fmt.Sprintf("%016x", uint64(f))
+}
+
+// Fingerprints represents a collection of Fingerprint subject to a given
+// natural sorting scheme. It implements sort.Interface.
+type Fingerprints []Fingerprint
+
+// Len implements sort.Interface.
+func (f Fingerprints) Len() int {
+	return len(f)
+}
+
+// Less implements sort.Interface.
+func (f Fingerprints) Less(i, j int) bool {
+	return f[i] < f[j]
+}
+
+// Swap implements sort.Interface.
+func (f Fingerprints) Swap(i, j int) {
+	f[i], f[j] = f[j], f[i]
+}
+
+// FingerprintSet is a set of Fingerprints.
+type FingerprintSet map[Fingerprint]struct{}
+
+// Equal returns true if both sets contain the same elements (and not more).
+func (s FingerprintSet) Equal(o FingerprintSet) bool {
+	if len(s) != len(o) {
+		return false
+	}
+
+	for k := range s {
+		if _, ok := o[k]; !ok {
+			return false
+		}
+	}
+
+	return true
+}
+
+// Intersection returns the elements contained in both sets.
+func (s FingerprintSet) Intersection(o FingerprintSet) FingerprintSet {
+	myLength, otherLength := len(s), len(o)
+	if myLength == 0 || otherLength == 0 {
+		return FingerprintSet{}
+	}
+
+	subSet := s
+	superSet := o
+
+	if otherLength < myLength {
+		subSet = o
+		superSet = s
+	}
+
+	out := FingerprintSet{}
+
+	for k := range subSet {
+		if _, ok := superSet[k]; ok {
+			out[k] = struct{}{}
+		}
+	}
+
+	return out
+}
diff --git a/vendor/github.com/prometheus/common/model/fnv.go b/vendor/github.com/prometheus/common/model/fnv.go
new file mode 100644
index 0000000..038fc1c
--- /dev/null
+++ b/vendor/github.com/prometheus/common/model/fnv.go
@@ -0,0 +1,42 @@
+// Copyright 2015 The Prometheus 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 model
+
+// Inline and byte-free variant of hash/fnv's fnv64a.
+
+const (
+	offset64 = 14695981039346656037
+	prime64  = 1099511628211
+)
+
+// hashNew initializies a new fnv64a hash value.
+func hashNew() uint64 {
+	return offset64
+}
+
+// hashAdd adds a string to a fnv64a hash value, returning the updated hash.
+func hashAdd(h uint64, s string) uint64 {
+	for i := 0; i < len(s); i++ {
+		h ^= uint64(s[i])
+		h *= prime64
+	}
+	return h
+}
+
+// hashAddByte adds a byte to a fnv64a hash value, returning the updated hash.
+func hashAddByte(h uint64, b byte) uint64 {
+	h ^= uint64(b)
+	h *= prime64
+	return h
+}
diff --git a/vendor/github.com/prometheus/common/model/labels.go b/vendor/github.com/prometheus/common/model/labels.go
new file mode 100644
index 0000000..41051a0
--- /dev/null
+++ b/vendor/github.com/prometheus/common/model/labels.go
@@ -0,0 +1,210 @@
+// Copyright 2013 The Prometheus 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 model
+
+import (
+	"encoding/json"
+	"fmt"
+	"regexp"
+	"strings"
+	"unicode/utf8"
+)
+
+const (
+	// AlertNameLabel is the name of the label containing the an alert's name.
+	AlertNameLabel = "alertname"
+
+	// ExportedLabelPrefix is the prefix to prepend to the label names present in
+	// exported metrics if a label of the same name is added by the server.
+	ExportedLabelPrefix = "exported_"
+
+	// MetricNameLabel is the label name indicating the metric name of a
+	// timeseries.
+	MetricNameLabel = "__name__"
+
+	// SchemeLabel is the name of the label that holds the scheme on which to
+	// scrape a target.
+	SchemeLabel = "__scheme__"
+
+	// AddressLabel is the name of the label that holds the address of
+	// a scrape target.
+	AddressLabel = "__address__"
+
+	// MetricsPathLabel is the name of the label that holds the path on which to
+	// scrape a target.
+	MetricsPathLabel = "__metrics_path__"
+
+	// ReservedLabelPrefix is a prefix which is not legal in user-supplied
+	// label names.
+	ReservedLabelPrefix = "__"
+
+	// MetaLabelPrefix is a prefix for labels that provide meta information.
+	// Labels with this prefix are used for intermediate label processing and
+	// will not be attached to time series.
+	MetaLabelPrefix = "__meta_"
+
+	// TmpLabelPrefix is a prefix for temporary labels as part of relabelling.
+	// Labels with this prefix are used for intermediate label processing and
+	// will not be attached to time series. This is reserved for use in
+	// Prometheus configuration files by users.
+	TmpLabelPrefix = "__tmp_"
+
+	// ParamLabelPrefix is a prefix for labels that provide URL parameters
+	// used to scrape a target.
+	ParamLabelPrefix = "__param_"
+
+	// JobLabel is the label name indicating the job from which a timeseries
+	// was scraped.
+	JobLabel = "job"
+
+	// InstanceLabel is the label name used for the instance label.
+	InstanceLabel = "instance"
+
+	// BucketLabel is used for the label that defines the upper bound of a
+	// bucket of a histogram ("le" -> "less or equal").
+	BucketLabel = "le"
+
+	// QuantileLabel is used for the label that defines the quantile in a
+	// summary.
+	QuantileLabel = "quantile"
+)
+
+// LabelNameRE is a regular expression matching valid label names. Note that the
+// IsValid method of LabelName performs the same check but faster than a match
+// with this regular expression.
+var LabelNameRE = regexp.MustCompile("^[a-zA-Z_][a-zA-Z0-9_]*$")
+
+// A LabelName is a key for a LabelSet or Metric.  It has a value associated
+// therewith.
+type LabelName string
+
+// IsValid is true iff the label name matches the pattern of LabelNameRE. This
+// method, however, does not use LabelNameRE for the check but a much faster
+// hardcoded implementation.
+func (ln LabelName) IsValid() bool {
+	if len(ln) == 0 {
+		return false
+	}
+	for i, b := range ln {
+		if !((b >= 'a' && b <= 'z') || (b >= 'A' && b <= 'Z') || b == '_' || (b >= '0' && b <= '9' && i > 0)) {
+			return false
+		}
+	}
+	return true
+}
+
+// UnmarshalYAML implements the yaml.Unmarshaler interface.
+func (ln *LabelName) UnmarshalYAML(unmarshal func(interface{}) error) error {
+	var s string
+	if err := unmarshal(&s); err != nil {
+		return err
+	}
+	if !LabelName(s).IsValid() {
+		return fmt.Errorf("%q is not a valid label name", s)
+	}
+	*ln = LabelName(s)
+	return nil
+}
+
+// UnmarshalJSON implements the json.Unmarshaler interface.
+func (ln *LabelName) UnmarshalJSON(b []byte) error {
+	var s string
+	if err := json.Unmarshal(b, &s); err != nil {
+		return err
+	}
+	if !LabelName(s).IsValid() {
+		return fmt.Errorf("%q is not a valid label name", s)
+	}
+	*ln = LabelName(s)
+	return nil
+}
+
+// LabelNames is a sortable LabelName slice. In implements sort.Interface.
+type LabelNames []LabelName
+
+func (l LabelNames) Len() int {
+	return len(l)
+}
+
+func (l LabelNames) Less(i, j int) bool {
+	return l[i] < l[j]
+}
+
+func (l LabelNames) Swap(i, j int) {
+	l[i], l[j] = l[j], l[i]
+}
+
+func (l LabelNames) String() string {
+	labelStrings := make([]string, 0, len(l))
+	for _, label := range l {
+		labelStrings = append(labelStrings, string(label))
+	}
+	return strings.Join(labelStrings, ", ")
+}
+
+// A LabelValue is an associated value for a LabelName.
+type LabelValue string
+
+// IsValid returns true iff the string is a valid UTF8.
+func (lv LabelValue) IsValid() bool {
+	return utf8.ValidString(string(lv))
+}
+
+// LabelValues is a sortable LabelValue slice. It implements sort.Interface.
+type LabelValues []LabelValue
+
+func (l LabelValues) Len() int {
+	return len(l)
+}
+
+func (l LabelValues) Less(i, j int) bool {
+	return string(l[i]) < string(l[j])
+}
+
+func (l LabelValues) Swap(i, j int) {
+	l[i], l[j] = l[j], l[i]
+}
+
+// LabelPair pairs a name with a value.
+type LabelPair struct {
+	Name  LabelName
+	Value LabelValue
+}
+
+// LabelPairs is a sortable slice of LabelPair pointers. It implements
+// sort.Interface.
+type LabelPairs []*LabelPair
+
+func (l LabelPairs) Len() int {
+	return len(l)
+}
+
+func (l LabelPairs) Less(i, j int) bool {
+	switch {
+	case l[i].Name > l[j].Name:
+		return false
+	case l[i].Name < l[j].Name:
+		return true
+	case l[i].Value > l[j].Value:
+		return false
+	case l[i].Value < l[j].Value:
+		return true
+	default:
+		return false
+	}
+}
+
+func (l LabelPairs) Swap(i, j int) {
+	l[i], l[j] = l[j], l[i]
+}
diff --git a/vendor/github.com/prometheus/common/model/labelset.go b/vendor/github.com/prometheus/common/model/labelset.go
new file mode 100644
index 0000000..6eda08a
--- /dev/null
+++ b/vendor/github.com/prometheus/common/model/labelset.go
@@ -0,0 +1,169 @@
+// Copyright 2013 The Prometheus 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 model
+
+import (
+	"encoding/json"
+	"fmt"
+	"sort"
+	"strings"
+)
+
+// A LabelSet is a collection of LabelName and LabelValue pairs.  The LabelSet
+// may be fully-qualified down to the point where it may resolve to a single
+// Metric in the data store or not.  All operations that occur within the realm
+// of a LabelSet can emit a vector of Metric entities to which the LabelSet may
+// match.
+type LabelSet map[LabelName]LabelValue
+
+// Validate checks whether all names and values in the label set
+// are valid.
+func (ls LabelSet) Validate() error {
+	for ln, lv := range ls {
+		if !ln.IsValid() {
+			return fmt.Errorf("invalid name %q", ln)
+		}
+		if !lv.IsValid() {
+			return fmt.Errorf("invalid value %q", lv)
+		}
+	}
+	return nil
+}
+
+// Equal returns true iff both label sets have exactly the same key/value pairs.
+func (ls LabelSet) Equal(o LabelSet) bool {
+	if len(ls) != len(o) {
+		return false
+	}
+	for ln, lv := range ls {
+		olv, ok := o[ln]
+		if !ok {
+			return false
+		}
+		if olv != lv {
+			return false
+		}
+	}
+	return true
+}
+
+// Before compares the metrics, using the following criteria:
+//
+// If m has fewer labels than o, it is before o. If it has more, it is not.
+//
+// If the number of labels is the same, the superset of all label names is
+// sorted alphanumerically. The first differing label pair found in that order
+// determines the outcome: If the label does not exist at all in m, then m is
+// before o, and vice versa. Otherwise the label value is compared
+// alphanumerically.
+//
+// If m and o are equal, the method returns false.
+func (ls LabelSet) Before(o LabelSet) bool {
+	if len(ls) < len(o) {
+		return true
+	}
+	if len(ls) > len(o) {
+		return false
+	}
+
+	lns := make(LabelNames, 0, len(ls)+len(o))
+	for ln := range ls {
+		lns = append(lns, ln)
+	}
+	for ln := range o {
+		lns = append(lns, ln)
+	}
+	// It's probably not worth it to de-dup lns.
+	sort.Sort(lns)
+	for _, ln := range lns {
+		mlv, ok := ls[ln]
+		if !ok {
+			return true
+		}
+		olv, ok := o[ln]
+		if !ok {
+			return false
+		}
+		if mlv < olv {
+			return true
+		}
+		if mlv > olv {
+			return false
+		}
+	}
+	return false
+}
+
+// Clone returns a copy of the label set.
+func (ls LabelSet) Clone() LabelSet {
+	lsn := make(LabelSet, len(ls))
+	for ln, lv := range ls {
+		lsn[ln] = lv
+	}
+	return lsn
+}
+
+// Merge is a helper function to non-destructively merge two label sets.
+func (l LabelSet) Merge(other LabelSet) LabelSet {
+	result := make(LabelSet, len(l))
+
+	for k, v := range l {
+		result[k] = v
+	}
+
+	for k, v := range other {
+		result[k] = v
+	}
+
+	return result
+}
+
+func (l LabelSet) String() string {
+	lstrs := make([]string, 0, len(l))
+	for l, v := range l {
+		lstrs = append(lstrs, fmt.Sprintf("%s=%q", l, v))
+	}
+
+	sort.Strings(lstrs)
+	return fmt.Sprintf("{%s}", strings.Join(lstrs, ", "))
+}
+
+// Fingerprint returns the LabelSet's fingerprint.
+func (ls LabelSet) Fingerprint() Fingerprint {
+	return labelSetToFingerprint(ls)
+}
+
+// FastFingerprint returns the LabelSet's Fingerprint calculated by a faster hashing
+// algorithm, which is, however, more susceptible to hash collisions.
+func (ls LabelSet) FastFingerprint() Fingerprint {
+	return labelSetToFastFingerprint(ls)
+}
+
+// UnmarshalJSON implements the json.Unmarshaler interface.
+func (l *LabelSet) UnmarshalJSON(b []byte) error {
+	var m map[LabelName]LabelValue
+	if err := json.Unmarshal(b, &m); err != nil {
+		return err
+	}
+	// encoding/json only unmarshals maps of the form map[string]T. It treats
+	// LabelName as a string and does not call its UnmarshalJSON method.
+	// Thus, we have to replicate the behavior here.
+	for ln := range m {
+		if !ln.IsValid() {
+			return fmt.Errorf("%q is not a valid label name", ln)
+		}
+	}
+	*l = LabelSet(m)
+	return nil
+}
diff --git a/vendor/github.com/prometheus/common/model/metric.go b/vendor/github.com/prometheus/common/model/metric.go
new file mode 100644
index 0000000..00804b7
--- /dev/null
+++ b/vendor/github.com/prometheus/common/model/metric.go
@@ -0,0 +1,102 @@
+// Copyright 2013 The Prometheus 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 model
+
+import (
+	"fmt"
+	"regexp"
+	"sort"
+	"strings"
+)
+
+var (
+	// MetricNameRE is a regular expression matching valid metric
+	// names. Note that the IsValidMetricName function performs the same
+	// check but faster than a match with this regular expression.
+	MetricNameRE = regexp.MustCompile(`^[a-zA-Z_:][a-zA-Z0-9_:]*$`)
+)
+
+// A Metric is similar to a LabelSet, but the key difference is that a Metric is
+// a singleton and refers to one and only one stream of samples.
+type Metric LabelSet
+
+// Equal compares the metrics.
+func (m Metric) Equal(o Metric) bool {
+	return LabelSet(m).Equal(LabelSet(o))
+}
+
+// Before compares the metrics' underlying label sets.
+func (m Metric) Before(o Metric) bool {
+	return LabelSet(m).Before(LabelSet(o))
+}
+
+// Clone returns a copy of the Metric.
+func (m Metric) Clone() Metric {
+	clone := make(Metric, len(m))
+	for k, v := range m {
+		clone[k] = v
+	}
+	return clone
+}
+
+func (m Metric) String() string {
+	metricName, hasName := m[MetricNameLabel]
+	numLabels := len(m) - 1
+	if !hasName {
+		numLabels = len(m)
+	}
+	labelStrings := make([]string, 0, numLabels)
+	for label, value := range m {
+		if label != MetricNameLabel {
+			labelStrings = append(labelStrings, fmt.Sprintf("%s=%q", label, value))
+		}
+	}
+
+	switch numLabels {
+	case 0:
+		if hasName {
+			return string(metricName)
+		}
+		return "{}"
+	default:
+		sort.Strings(labelStrings)
+		return fmt.Sprintf("%s{%s}", metricName, strings.Join(labelStrings, ", "))
+	}
+}
+
+// Fingerprint returns a Metric's Fingerprint.
+func (m Metric) Fingerprint() Fingerprint {
+	return LabelSet(m).Fingerprint()
+}
+
+// FastFingerprint returns a Metric's Fingerprint calculated by a faster hashing
+// algorithm, which is, however, more susceptible to hash collisions.
+func (m Metric) FastFingerprint() Fingerprint {
+	return LabelSet(m).FastFingerprint()
+}
+
+// IsValidMetricName returns true iff name matches the pattern of MetricNameRE.
+// This function, however, does not use MetricNameRE for the check but a much
+// faster hardcoded implementation.
+func IsValidMetricName(n LabelValue) bool {
+	if len(n) == 0 {
+		return false
+	}
+	for i, b := range n {
+		if !((b >= 'a' && b <= 'z') || (b >= 'A' && b <= 'Z') || b == '_' || b == ':' || (b >= '0' && b <= '9' && i > 0)) {
+			return false
+		}
+	}
+	return true
+}
diff --git a/vendor/github.com/prometheus/common/model/model.go b/vendor/github.com/prometheus/common/model/model.go
new file mode 100644
index 0000000..a7b9691
--- /dev/null
+++ b/vendor/github.com/prometheus/common/model/model.go
@@ -0,0 +1,16 @@
+// Copyright 2013 The Prometheus 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 model contains common data structures that are shared across
+// Prometheus components and libraries.
+package model
diff --git a/vendor/github.com/prometheus/common/model/signature.go b/vendor/github.com/prometheus/common/model/signature.go
new file mode 100644
index 0000000..8762b13
--- /dev/null
+++ b/vendor/github.com/prometheus/common/model/signature.go
@@ -0,0 +1,144 @@
+// Copyright 2014 The Prometheus 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 model
+
+import (
+	"sort"
+)
+
+// SeparatorByte is a byte that cannot occur in valid UTF-8 sequences and is
+// used to separate label names, label values, and other strings from each other
+// when calculating their combined hash value (aka signature aka fingerprint).
+const SeparatorByte byte = 255
+
+var (
+	// cache the signature of an empty label set.
+	emptyLabelSignature = hashNew()
+)
+
+// LabelsToSignature returns a quasi-unique signature (i.e., fingerprint) for a
+// given label set. (Collisions are possible but unlikely if the number of label
+// sets the function is applied to is small.)
+func LabelsToSignature(labels map[string]string) uint64 {
+	if len(labels) == 0 {
+		return emptyLabelSignature
+	}
+
+	labelNames := make([]string, 0, len(labels))
+	for labelName := range labels {
+		labelNames = append(labelNames, labelName)
+	}
+	sort.Strings(labelNames)
+
+	sum := hashNew()
+	for _, labelName := range labelNames {
+		sum = hashAdd(sum, labelName)
+		sum = hashAddByte(sum, SeparatorByte)
+		sum = hashAdd(sum, labels[labelName])
+		sum = hashAddByte(sum, SeparatorByte)
+	}
+	return sum
+}
+
+// labelSetToFingerprint works exactly as LabelsToSignature but takes a LabelSet as
+// parameter (rather than a label map) and returns a Fingerprint.
+func labelSetToFingerprint(ls LabelSet) Fingerprint {
+	if len(ls) == 0 {
+		return Fingerprint(emptyLabelSignature)
+	}
+
+	labelNames := make(LabelNames, 0, len(ls))
+	for labelName := range ls {
+		labelNames = append(labelNames, labelName)
+	}
+	sort.Sort(labelNames)
+
+	sum := hashNew()
+	for _, labelName := range labelNames {
+		sum = hashAdd(sum, string(labelName))
+		sum = hashAddByte(sum, SeparatorByte)
+		sum = hashAdd(sum, string(ls[labelName]))
+		sum = hashAddByte(sum, SeparatorByte)
+	}
+	return Fingerprint(sum)
+}
+
+// labelSetToFastFingerprint works similar to labelSetToFingerprint but uses a
+// faster and less allocation-heavy hash function, which is more susceptible to
+// create hash collisions. Therefore, collision detection should be applied.
+func labelSetToFastFingerprint(ls LabelSet) Fingerprint {
+	if len(ls) == 0 {
+		return Fingerprint(emptyLabelSignature)
+	}
+
+	var result uint64
+	for labelName, labelValue := range ls {
+		sum := hashNew()
+		sum = hashAdd(sum, string(labelName))
+		sum = hashAddByte(sum, SeparatorByte)
+		sum = hashAdd(sum, string(labelValue))
+		result ^= sum
+	}
+	return Fingerprint(result)
+}
+
+// SignatureForLabels works like LabelsToSignature but takes a Metric as
+// parameter (rather than a label map) and only includes the labels with the
+// specified LabelNames into the signature calculation. The labels passed in
+// will be sorted by this function.
+func SignatureForLabels(m Metric, labels ...LabelName) uint64 {
+	if len(labels) == 0 {
+		return emptyLabelSignature
+	}
+
+	sort.Sort(LabelNames(labels))
+
+	sum := hashNew()
+	for _, label := range labels {
+		sum = hashAdd(sum, string(label))
+		sum = hashAddByte(sum, SeparatorByte)
+		sum = hashAdd(sum, string(m[label]))
+		sum = hashAddByte(sum, SeparatorByte)
+	}
+	return sum
+}
+
+// SignatureWithoutLabels works like LabelsToSignature but takes a Metric as
+// parameter (rather than a label map) and excludes the labels with any of the
+// specified LabelNames from the signature calculation.
+func SignatureWithoutLabels(m Metric, labels map[LabelName]struct{}) uint64 {
+	if len(m) == 0 {
+		return emptyLabelSignature
+	}
+
+	labelNames := make(LabelNames, 0, len(m))
+	for labelName := range m {
+		if _, exclude := labels[labelName]; !exclude {
+			labelNames = append(labelNames, labelName)
+		}
+	}
+	if len(labelNames) == 0 {
+		return emptyLabelSignature
+	}
+	sort.Sort(labelNames)
+
+	sum := hashNew()
+	for _, labelName := range labelNames {
+		sum = hashAdd(sum, string(labelName))
+		sum = hashAddByte(sum, SeparatorByte)
+		sum = hashAdd(sum, string(m[labelName]))
+		sum = hashAddByte(sum, SeparatorByte)
+	}
+	return sum
+}
diff --git a/vendor/github.com/prometheus/common/model/silence.go b/vendor/github.com/prometheus/common/model/silence.go
new file mode 100644
index 0000000..bb99889
--- /dev/null
+++ b/vendor/github.com/prometheus/common/model/silence.go
@@ -0,0 +1,106 @@
+// Copyright 2015 The Prometheus 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 model
+
+import (
+	"encoding/json"
+	"fmt"
+	"regexp"
+	"time"
+)
+
+// Matcher describes a matches the value of a given label.
+type Matcher struct {
+	Name    LabelName `json:"name"`
+	Value   string    `json:"value"`
+	IsRegex bool      `json:"isRegex"`
+}
+
+func (m *Matcher) UnmarshalJSON(b []byte) error {
+	type plain Matcher
+	if err := json.Unmarshal(b, (*plain)(m)); err != nil {
+		return err
+	}
+
+	if len(m.Name) == 0 {
+		return fmt.Errorf("label name in matcher must not be empty")
+	}
+	if m.IsRegex {
+		if _, err := regexp.Compile(m.Value); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+// Validate returns true iff all fields of the matcher have valid values.
+func (m *Matcher) Validate() error {
+	if !m.Name.IsValid() {
+		return fmt.Errorf("invalid name %q", m.Name)
+	}
+	if m.IsRegex {
+		if _, err := regexp.Compile(m.Value); err != nil {
+			return fmt.Errorf("invalid regular expression %q", m.Value)
+		}
+	} else if !LabelValue(m.Value).IsValid() || len(m.Value) == 0 {
+		return fmt.Errorf("invalid value %q", m.Value)
+	}
+	return nil
+}
+
+// Silence defines the representation of a silence definition in the Prometheus
+// eco-system.
+type Silence struct {
+	ID uint64 `json:"id,omitempty"`
+
+	Matchers []*Matcher `json:"matchers"`
+
+	StartsAt time.Time `json:"startsAt"`
+	EndsAt   time.Time `json:"endsAt"`
+
+	CreatedAt time.Time `json:"createdAt,omitempty"`
+	CreatedBy string    `json:"createdBy"`
+	Comment   string    `json:"comment,omitempty"`
+}
+
+// Validate returns true iff all fields of the silence have valid values.
+func (s *Silence) Validate() error {
+	if len(s.Matchers) == 0 {
+		return fmt.Errorf("at least one matcher required")
+	}
+	for _, m := range s.Matchers {
+		if err := m.Validate(); err != nil {
+			return fmt.Errorf("invalid matcher: %s", err)
+		}
+	}
+	if s.StartsAt.IsZero() {
+		return fmt.Errorf("start time missing")
+	}
+	if s.EndsAt.IsZero() {
+		return fmt.Errorf("end time missing")
+	}
+	if s.EndsAt.Before(s.StartsAt) {
+		return fmt.Errorf("start time must be before end time")
+	}
+	if s.CreatedBy == "" {
+		return fmt.Errorf("creator information missing")
+	}
+	if s.Comment == "" {
+		return fmt.Errorf("comment missing")
+	}
+	if s.CreatedAt.IsZero() {
+		return fmt.Errorf("creation timestamp missing")
+	}
+	return nil
+}
diff --git a/vendor/github.com/prometheus/common/model/time.go b/vendor/github.com/prometheus/common/model/time.go
new file mode 100644
index 0000000..7b0064f
--- /dev/null
+++ b/vendor/github.com/prometheus/common/model/time.go
@@ -0,0 +1,270 @@
+// Copyright 2013 The Prometheus 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 model
+
+import (
+	"fmt"
+	"math"
+	"regexp"
+	"strconv"
+	"strings"
+	"time"
+)
+
+const (
+	// MinimumTick is the minimum supported time resolution. This has to be
+	// at least time.Second in order for the code below to work.
+	minimumTick = time.Millisecond
+	// second is the Time duration equivalent to one second.
+	second = int64(time.Second / minimumTick)
+	// The number of nanoseconds per minimum tick.
+	nanosPerTick = int64(minimumTick / time.Nanosecond)
+
+	// Earliest is the earliest Time representable. Handy for
+	// initializing a high watermark.
+	Earliest = Time(math.MinInt64)
+	// Latest is the latest Time representable. Handy for initializing
+	// a low watermark.
+	Latest = Time(math.MaxInt64)
+)
+
+// Time is the number of milliseconds since the epoch
+// (1970-01-01 00:00 UTC) excluding leap seconds.
+type Time int64
+
+// Interval describes an interval between two timestamps.
+type Interval struct {
+	Start, End Time
+}
+
+// Now returns the current time as a Time.
+func Now() Time {
+	return TimeFromUnixNano(time.Now().UnixNano())
+}
+
+// TimeFromUnix returns the Time equivalent to the Unix Time t
+// provided in seconds.
+func TimeFromUnix(t int64) Time {
+	return Time(t * second)
+}
+
+// TimeFromUnixNano returns the Time equivalent to the Unix Time
+// t provided in nanoseconds.
+func TimeFromUnixNano(t int64) Time {
+	return Time(t / nanosPerTick)
+}
+
+// Equal reports whether two Times represent the same instant.
+func (t Time) Equal(o Time) bool {
+	return t == o
+}
+
+// Before reports whether the Time t is before o.
+func (t Time) Before(o Time) bool {
+	return t < o
+}
+
+// After reports whether the Time t is after o.
+func (t Time) After(o Time) bool {
+	return t > o
+}
+
+// Add returns the Time t + d.
+func (t Time) Add(d time.Duration) Time {
+	return t + Time(d/minimumTick)
+}
+
+// Sub returns the Duration t - o.
+func (t Time) Sub(o Time) time.Duration {
+	return time.Duration(t-o) * minimumTick
+}
+
+// Time returns the time.Time representation of t.
+func (t Time) Time() time.Time {
+	return time.Unix(int64(t)/second, (int64(t)%second)*nanosPerTick)
+}
+
+// Unix returns t as a Unix time, the number of seconds elapsed
+// since January 1, 1970 UTC.
+func (t Time) Unix() int64 {
+	return int64(t) / second
+}
+
+// UnixNano returns t as a Unix time, the number of nanoseconds elapsed
+// since January 1, 1970 UTC.
+func (t Time) UnixNano() int64 {
+	return int64(t) * nanosPerTick
+}
+
+// The number of digits after the dot.
+var dotPrecision = int(math.Log10(float64(second)))
+
+// String returns a string representation of the Time.
+func (t Time) String() string {
+	return strconv.FormatFloat(float64(t)/float64(second), 'f', -1, 64)
+}
+
+// MarshalJSON implements the json.Marshaler interface.
+func (t Time) MarshalJSON() ([]byte, error) {
+	return []byte(t.String()), nil
+}
+
+// UnmarshalJSON implements the json.Unmarshaler interface.
+func (t *Time) UnmarshalJSON(b []byte) error {
+	p := strings.Split(string(b), ".")
+	switch len(p) {
+	case 1:
+		v, err := strconv.ParseInt(string(p[0]), 10, 64)
+		if err != nil {
+			return err
+		}
+		*t = Time(v * second)
+
+	case 2:
+		v, err := strconv.ParseInt(string(p[0]), 10, 64)
+		if err != nil {
+			return err
+		}
+		v *= second
+
+		prec := dotPrecision - len(p[1])
+		if prec < 0 {
+			p[1] = p[1][:dotPrecision]
+		} else if prec > 0 {
+			p[1] = p[1] + strings.Repeat("0", prec)
+		}
+
+		va, err := strconv.ParseInt(p[1], 10, 32)
+		if err != nil {
+			return err
+		}
+
+		// If the value was something like -0.1 the negative is lost in the
+		// parsing because of the leading zero, this ensures that we capture it.
+		if len(p[0]) > 0 && p[0][0] == '-' && v+va > 0 {
+			*t = Time(v+va) * -1
+		} else {
+			*t = Time(v + va)
+		}
+
+	default:
+		return fmt.Errorf("invalid time %q", string(b))
+	}
+	return nil
+}
+
+// Duration wraps time.Duration. It is used to parse the custom duration format
+// from YAML.
+// This type should not propagate beyond the scope of input/output processing.
+type Duration time.Duration
+
+// Set implements pflag/flag.Value
+func (d *Duration) Set(s string) error {
+	var err error
+	*d, err = ParseDuration(s)
+	return err
+}
+
+// Type implements pflag.Value
+func (d *Duration) Type() string {
+	return "duration"
+}
+
+var durationRE = regexp.MustCompile("^([0-9]+)(y|w|d|h|m|s|ms)$")
+
+// ParseDuration parses a string into a time.Duration, assuming that a year
+// always has 365d, a week always has 7d, and a day always has 24h.
+func ParseDuration(durationStr string) (Duration, error) {
+	matches := durationRE.FindStringSubmatch(durationStr)
+	if len(matches) != 3 {
+		return 0, fmt.Errorf("not a valid duration string: %q", durationStr)
+	}
+	var (
+		n, _ = strconv.Atoi(matches[1])
+		dur  = time.Duration(n) * time.Millisecond
+	)
+	switch unit := matches[2]; unit {
+	case "y":
+		dur *= 1000 * 60 * 60 * 24 * 365
+	case "w":
+		dur *= 1000 * 60 * 60 * 24 * 7
+	case "d":
+		dur *= 1000 * 60 * 60 * 24
+	case "h":
+		dur *= 1000 * 60 * 60
+	case "m":
+		dur *= 1000 * 60
+	case "s":
+		dur *= 1000
+	case "ms":
+		// Value already correct
+	default:
+		return 0, fmt.Errorf("invalid time unit in duration string: %q", unit)
+	}
+	return Duration(dur), nil
+}
+
+func (d Duration) String() string {
+	var (
+		ms   = int64(time.Duration(d) / time.Millisecond)
+		unit = "ms"
+	)
+	if ms == 0 {
+		return "0s"
+	}
+	factors := map[string]int64{
+		"y":  1000 * 60 * 60 * 24 * 365,
+		"w":  1000 * 60 * 60 * 24 * 7,
+		"d":  1000 * 60 * 60 * 24,
+		"h":  1000 * 60 * 60,
+		"m":  1000 * 60,
+		"s":  1000,
+		"ms": 1,
+	}
+
+	switch int64(0) {
+	case ms % factors["y"]:
+		unit = "y"
+	case ms % factors["w"]:
+		unit = "w"
+	case ms % factors["d"]:
+		unit = "d"
+	case ms % factors["h"]:
+		unit = "h"
+	case ms % factors["m"]:
+		unit = "m"
+	case ms % factors["s"]:
+		unit = "s"
+	}
+	return fmt.Sprintf("%v%v", ms/factors[unit], unit)
+}
+
+// MarshalYAML implements the yaml.Marshaler interface.
+func (d Duration) MarshalYAML() (interface{}, error) {
+	return d.String(), nil
+}
+
+// UnmarshalYAML implements the yaml.Unmarshaler interface.
+func (d *Duration) UnmarshalYAML(unmarshal func(interface{}) error) error {
+	var s string
+	if err := unmarshal(&s); err != nil {
+		return err
+	}
+	dur, err := ParseDuration(s)
+	if err != nil {
+		return err
+	}
+	*d = dur
+	return nil
+}
diff --git a/vendor/github.com/prometheus/common/model/value.go b/vendor/github.com/prometheus/common/model/value.go
new file mode 100644
index 0000000..c9d8fb1
--- /dev/null
+++ b/vendor/github.com/prometheus/common/model/value.go
@@ -0,0 +1,416 @@
+// Copyright 2013 The Prometheus 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 model
+
+import (
+	"encoding/json"
+	"fmt"
+	"math"
+	"sort"
+	"strconv"
+	"strings"
+)
+
+var (
+	// ZeroSamplePair is the pseudo zero-value of SamplePair used to signal a
+	// non-existing sample pair. It is a SamplePair with timestamp Earliest and
+	// value 0.0. Note that the natural zero value of SamplePair has a timestamp
+	// of 0, which is possible to appear in a real SamplePair and thus not
+	// suitable to signal a non-existing SamplePair.
+	ZeroSamplePair = SamplePair{Timestamp: Earliest}
+
+	// ZeroSample is the pseudo zero-value of Sample used to signal a
+	// non-existing sample. It is a Sample with timestamp Earliest, value 0.0,
+	// and metric nil. Note that the natural zero value of Sample has a timestamp
+	// of 0, which is possible to appear in a real Sample and thus not suitable
+	// to signal a non-existing Sample.
+	ZeroSample = Sample{Timestamp: Earliest}
+)
+
+// A SampleValue is a representation of a value for a given sample at a given
+// time.
+type SampleValue float64
+
+// MarshalJSON implements json.Marshaler.
+func (v SampleValue) MarshalJSON() ([]byte, error) {
+	return json.Marshal(v.String())
+}
+
+// UnmarshalJSON implements json.Unmarshaler.
+func (v *SampleValue) UnmarshalJSON(b []byte) error {
+	if len(b) < 2 || b[0] != '"' || b[len(b)-1] != '"' {
+		return fmt.Errorf("sample value must be a quoted string")
+	}
+	f, err := strconv.ParseFloat(string(b[1:len(b)-1]), 64)
+	if err != nil {
+		return err
+	}
+	*v = SampleValue(f)
+	return nil
+}
+
+// Equal returns true if the value of v and o is equal or if both are NaN. Note
+// that v==o is false if both are NaN. If you want the conventional float
+// behavior, use == to compare two SampleValues.
+func (v SampleValue) Equal(o SampleValue) bool {
+	if v == o {
+		return true
+	}
+	return math.IsNaN(float64(v)) && math.IsNaN(float64(o))
+}
+
+func (v SampleValue) String() string {
+	return strconv.FormatFloat(float64(v), 'f', -1, 64)
+}
+
+// SamplePair pairs a SampleValue with a Timestamp.
+type SamplePair struct {
+	Timestamp Time
+	Value     SampleValue
+}
+
+// MarshalJSON implements json.Marshaler.
+func (s SamplePair) MarshalJSON() ([]byte, error) {
+	t, err := json.Marshal(s.Timestamp)
+	if err != nil {
+		return nil, err
+	}
+	v, err := json.Marshal(s.Value)
+	if err != nil {
+		return nil, err
+	}
+	return []byte(fmt.Sprintf("[%s,%s]", t, v)), nil
+}
+
+// UnmarshalJSON implements json.Unmarshaler.
+func (s *SamplePair) UnmarshalJSON(b []byte) error {
+	v := [...]json.Unmarshaler{&s.Timestamp, &s.Value}
+	return json.Unmarshal(b, &v)
+}
+
+// Equal returns true if this SamplePair and o have equal Values and equal
+// Timestamps. The semantics of Value equality is defined by SampleValue.Equal.
+func (s *SamplePair) Equal(o *SamplePair) bool {
+	return s == o || (s.Value.Equal(o.Value) && s.Timestamp.Equal(o.Timestamp))
+}
+
+func (s SamplePair) String() string {
+	return fmt.Sprintf("%s @[%s]", s.Value, s.Timestamp)
+}
+
+// Sample is a sample pair associated with a metric.
+type Sample struct {
+	Metric    Metric      `json:"metric"`
+	Value     SampleValue `json:"value"`
+	Timestamp Time        `json:"timestamp"`
+}
+
+// Equal compares first the metrics, then the timestamp, then the value. The
+// semantics of value equality is defined by SampleValue.Equal.
+func (s *Sample) Equal(o *Sample) bool {
+	if s == o {
+		return true
+	}
+
+	if !s.Metric.Equal(o.Metric) {
+		return false
+	}
+	if !s.Timestamp.Equal(o.Timestamp) {
+		return false
+	}
+
+	return s.Value.Equal(o.Value)
+}
+
+func (s Sample) String() string {
+	return fmt.Sprintf("%s => %s", s.Metric, SamplePair{
+		Timestamp: s.Timestamp,
+		Value:     s.Value,
+	})
+}
+
+// MarshalJSON implements json.Marshaler.
+func (s Sample) MarshalJSON() ([]byte, error) {
+	v := struct {
+		Metric Metric     `json:"metric"`
+		Value  SamplePair `json:"value"`
+	}{
+		Metric: s.Metric,
+		Value: SamplePair{
+			Timestamp: s.Timestamp,
+			Value:     s.Value,
+		},
+	}
+
+	return json.Marshal(&v)
+}
+
+// UnmarshalJSON implements json.Unmarshaler.
+func (s *Sample) UnmarshalJSON(b []byte) error {
+	v := struct {
+		Metric Metric     `json:"metric"`
+		Value  SamplePair `json:"value"`
+	}{
+		Metric: s.Metric,
+		Value: SamplePair{
+			Timestamp: s.Timestamp,
+			Value:     s.Value,
+		},
+	}
+
+	if err := json.Unmarshal(b, &v); err != nil {
+		return err
+	}
+
+	s.Metric = v.Metric
+	s.Timestamp = v.Value.Timestamp
+	s.Value = v.Value.Value
+
+	return nil
+}
+
+// Samples is a sortable Sample slice. It implements sort.Interface.
+type Samples []*Sample
+
+func (s Samples) Len() int {
+	return len(s)
+}
+
+// Less compares first the metrics, then the timestamp.
+func (s Samples) Less(i, j int) bool {
+	switch {
+	case s[i].Metric.Before(s[j].Metric):
+		return true
+	case s[j].Metric.Before(s[i].Metric):
+		return false
+	case s[i].Timestamp.Before(s[j].Timestamp):
+		return true
+	default:
+		return false
+	}
+}
+
+func (s Samples) Swap(i, j int) {
+	s[i], s[j] = s[j], s[i]
+}
+
+// Equal compares two sets of samples and returns true if they are equal.
+func (s Samples) Equal(o Samples) bool {
+	if len(s) != len(o) {
+		return false
+	}
+
+	for i, sample := range s {
+		if !sample.Equal(o[i]) {
+			return false
+		}
+	}
+	return true
+}
+
+// SampleStream is a stream of Values belonging to an attached COWMetric.
+type SampleStream struct {
+	Metric Metric       `json:"metric"`
+	Values []SamplePair `json:"values"`
+}
+
+func (ss SampleStream) String() string {
+	vals := make([]string, len(ss.Values))
+	for i, v := range ss.Values {
+		vals[i] = v.String()
+	}
+	return fmt.Sprintf("%s =>\n%s", ss.Metric, strings.Join(vals, "\n"))
+}
+
+// Value is a generic interface for values resulting from a query evaluation.
+type Value interface {
+	Type() ValueType
+	String() string
+}
+
+func (Matrix) Type() ValueType  { return ValMatrix }
+func (Vector) Type() ValueType  { return ValVector }
+func (*Scalar) Type() ValueType { return ValScalar }
+func (*String) Type() ValueType { return ValString }
+
+type ValueType int
+
+const (
+	ValNone ValueType = iota
+	ValScalar
+	ValVector
+	ValMatrix
+	ValString
+)
+
+// MarshalJSON implements json.Marshaler.
+func (et ValueType) MarshalJSON() ([]byte, error) {
+	return json.Marshal(et.String())
+}
+
+func (et *ValueType) UnmarshalJSON(b []byte) error {
+	var s string
+	if err := json.Unmarshal(b, &s); err != nil {
+		return err
+	}
+	switch s {
+	case "<ValNone>":
+		*et = ValNone
+	case "scalar":
+		*et = ValScalar
+	case "vector":
+		*et = ValVector
+	case "matrix":
+		*et = ValMatrix
+	case "string":
+		*et = ValString
+	default:
+		return fmt.Errorf("unknown value type %q", s)
+	}
+	return nil
+}
+
+func (e ValueType) String() string {
+	switch e {
+	case ValNone:
+		return "<ValNone>"
+	case ValScalar:
+		return "scalar"
+	case ValVector:
+		return "vector"
+	case ValMatrix:
+		return "matrix"
+	case ValString:
+		return "string"
+	}
+	panic("ValueType.String: unhandled value type")
+}
+
+// Scalar is a scalar value evaluated at the set timestamp.
+type Scalar struct {
+	Value     SampleValue `json:"value"`
+	Timestamp Time        `json:"timestamp"`
+}
+
+func (s Scalar) String() string {
+	return fmt.Sprintf("scalar: %v @[%v]", s.Value, s.Timestamp)
+}
+
+// MarshalJSON implements json.Marshaler.
+func (s Scalar) MarshalJSON() ([]byte, error) {
+	v := strconv.FormatFloat(float64(s.Value), 'f', -1, 64)
+	return json.Marshal([...]interface{}{s.Timestamp, string(v)})
+}
+
+// UnmarshalJSON implements json.Unmarshaler.
+func (s *Scalar) UnmarshalJSON(b []byte) error {
+	var f string
+	v := [...]interface{}{&s.Timestamp, &f}
+
+	if err := json.Unmarshal(b, &v); err != nil {
+		return err
+	}
+
+	value, err := strconv.ParseFloat(f, 64)
+	if err != nil {
+		return fmt.Errorf("error parsing sample value: %s", err)
+	}
+	s.Value = SampleValue(value)
+	return nil
+}
+
+// String is a string value evaluated at the set timestamp.
+type String struct {
+	Value     string `json:"value"`
+	Timestamp Time   `json:"timestamp"`
+}
+
+func (s *String) String() string {
+	return s.Value
+}
+
+// MarshalJSON implements json.Marshaler.
+func (s String) MarshalJSON() ([]byte, error) {
+	return json.Marshal([]interface{}{s.Timestamp, s.Value})
+}
+
+// UnmarshalJSON implements json.Unmarshaler.
+func (s *String) UnmarshalJSON(b []byte) error {
+	v := [...]interface{}{&s.Timestamp, &s.Value}
+	return json.Unmarshal(b, &v)
+}
+
+// Vector is basically only an alias for Samples, but the
+// contract is that in a Vector, all Samples have the same timestamp.
+type Vector []*Sample
+
+func (vec Vector) String() string {
+	entries := make([]string, len(vec))
+	for i, s := range vec {
+		entries[i] = s.String()
+	}
+	return strings.Join(entries, "\n")
+}
+
+func (vec Vector) Len() int      { return len(vec) }
+func (vec Vector) Swap(i, j int) { vec[i], vec[j] = vec[j], vec[i] }
+
+// Less compares first the metrics, then the timestamp.
+func (vec Vector) Less(i, j int) bool {
+	switch {
+	case vec[i].Metric.Before(vec[j].Metric):
+		return true
+	case vec[j].Metric.Before(vec[i].Metric):
+		return false
+	case vec[i].Timestamp.Before(vec[j].Timestamp):
+		return true
+	default:
+		return false
+	}
+}
+
+// Equal compares two sets of samples and returns true if they are equal.
+func (vec Vector) Equal(o Vector) bool {
+	if len(vec) != len(o) {
+		return false
+	}
+
+	for i, sample := range vec {
+		if !sample.Equal(o[i]) {
+			return false
+		}
+	}
+	return true
+}
+
+// Matrix is a list of time series.
+type Matrix []*SampleStream
+
+func (m Matrix) Len() int           { return len(m) }
+func (m Matrix) Less(i, j int) bool { return m[i].Metric.Before(m[j].Metric) }
+func (m Matrix) Swap(i, j int)      { m[i], m[j] = m[j], m[i] }
+
+func (mat Matrix) String() string {
+	matCp := make(Matrix, len(mat))
+	copy(matCp, mat)
+	sort.Sort(matCp)
+
+	strs := make([]string, len(matCp))
+
+	for i, ss := range matCp {
+		strs[i] = ss.String()
+	}
+
+	return strings.Join(strs, "\n")
+}
diff --git a/vendor/github.com/prometheus/procfs/.gitignore b/vendor/github.com/prometheus/procfs/.gitignore
new file mode 100644
index 0000000..25e3659
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/.gitignore
@@ -0,0 +1 @@
+/fixtures/
diff --git a/vendor/github.com/prometheus/procfs/.golangci.yml b/vendor/github.com/prometheus/procfs/.golangci.yml
new file mode 100644
index 0000000..438ca92
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/.golangci.yml
@@ -0,0 +1,6 @@
+# Run only staticcheck for now. Additional linters will be enabled one-by-one.
+linters:
+  enable:
+  - staticcheck
+  - govet
+  disable-all: true
diff --git a/vendor/github.com/prometheus/procfs/CONTRIBUTING.md b/vendor/github.com/prometheus/procfs/CONTRIBUTING.md
new file mode 100644
index 0000000..40503ed
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/CONTRIBUTING.md
@@ -0,0 +1,18 @@
+# Contributing
+
+Prometheus uses GitHub to manage reviews of pull requests.
+
+* If you have a trivial fix or improvement, go ahead and create a pull request,
+  addressing (with `@...`) the maintainer of this repository (see
+  [MAINTAINERS.md](MAINTAINERS.md)) in the description of the pull request.
+
+* If you plan to do something more involved, first discuss your ideas
+  on our [mailing list](https://groups.google.com/forum/?fromgroups#!forum/prometheus-developers).
+  This will avoid unnecessary work and surely give you and us a good deal
+  of inspiration.
+
+* Relevant coding style guidelines are the [Go Code Review
+  Comments](https://code.google.com/p/go-wiki/wiki/CodeReviewComments)
+  and the _Formatting and style_ section of Peter Bourgon's [Go: Best
+  Practices for Production
+  Environments](http://peter.bourgon.org/go-in-production/#formatting-and-style).
diff --git a/vendor/github.com/prometheus/procfs/LICENSE b/vendor/github.com/prometheus/procfs/LICENSE
new file mode 100644
index 0000000..261eeb9
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/LICENSE
@@ -0,0 +1,201 @@
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
diff --git a/vendor/github.com/prometheus/procfs/MAINTAINERS.md b/vendor/github.com/prometheus/procfs/MAINTAINERS.md
new file mode 100644
index 0000000..56ba67d
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/MAINTAINERS.md
@@ -0,0 +1,2 @@
+* Johannes 'fish' Ziemke <github@freigeist.org> @discordianfish
+* Paul Gier <pgier@redhat.com> @pgier
diff --git a/vendor/github.com/prometheus/procfs/Makefile b/vendor/github.com/prometheus/procfs/Makefile
new file mode 100644
index 0000000..616a0d2
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/Makefile
@@ -0,0 +1,29 @@
+# Copyright 2018 The Prometheus 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.
+
+include Makefile.common
+
+%/.unpacked: %.ttar
+	@echo ">> extracting fixtures"
+	./ttar -C $(dir $*) -x -f $*.ttar
+	touch $@
+
+update_fixtures:
+	rm -vf fixtures/.unpacked
+	./ttar -c -f fixtures.ttar fixtures/
+
+.PHONY: build
+build:
+
+.PHONY: test
+test: fixtures/.unpacked common-test
diff --git a/vendor/github.com/prometheus/procfs/Makefile.common b/vendor/github.com/prometheus/procfs/Makefile.common
new file mode 100644
index 0000000..c7f9ea6
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/Makefile.common
@@ -0,0 +1,272 @@
+# Copyright 2018 The Prometheus 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.
+
+
+# A common Makefile that includes rules to be reused in different prometheus projects.
+# !!! Open PRs only against the prometheus/prometheus/Makefile.common repository!
+
+# Example usage :
+# Create the main Makefile in the root project directory.
+# include Makefile.common
+# customTarget:
+# 	@echo ">> Running customTarget"
+#
+
+# Ensure GOBIN is not set during build so that promu is installed to the correct path
+unexport GOBIN
+
+GO           ?= go
+GOFMT        ?= $(GO)fmt
+FIRST_GOPATH := $(firstword $(subst :, ,$(shell $(GO) env GOPATH)))
+GOOPTS       ?=
+GOHOSTOS     ?= $(shell $(GO) env GOHOSTOS)
+GOHOSTARCH   ?= $(shell $(GO) env GOHOSTARCH)
+
+GO_VERSION        ?= $(shell $(GO) version)
+GO_VERSION_NUMBER ?= $(word 3, $(GO_VERSION))
+PRE_GO_111        ?= $(shell echo $(GO_VERSION_NUMBER) | grep -E 'go1\.(10|[0-9])\.')
+
+GOVENDOR :=
+GO111MODULE :=
+ifeq (, $(PRE_GO_111))
+	ifneq (,$(wildcard go.mod))
+		# Enforce Go modules support just in case the directory is inside GOPATH (and for Travis CI).
+		GO111MODULE := on
+
+		ifneq (,$(wildcard vendor))
+			# Always use the local vendor/ directory to satisfy the dependencies.
+			GOOPTS := $(GOOPTS) -mod=vendor
+		endif
+	endif
+else
+	ifneq (,$(wildcard go.mod))
+		ifneq (,$(wildcard vendor))
+$(warning This repository requires Go >= 1.11 because of Go modules)
+$(warning Some recipes may not work as expected as the current Go runtime is '$(GO_VERSION_NUMBER)')
+		endif
+	else
+		# This repository isn't using Go modules (yet).
+		GOVENDOR := $(FIRST_GOPATH)/bin/govendor
+	endif
+endif
+PROMU        := $(FIRST_GOPATH)/bin/promu
+pkgs          = ./...
+
+ifeq (arm, $(GOHOSTARCH))
+	GOHOSTARM ?= $(shell GOARM= $(GO) env GOARM)
+	GO_BUILD_PLATFORM ?= $(GOHOSTOS)-$(GOHOSTARCH)v$(GOHOSTARM)
+else
+	GO_BUILD_PLATFORM ?= $(GOHOSTOS)-$(GOHOSTARCH)
+endif
+
+PROMU_VERSION ?= 0.4.0
+PROMU_URL     := https://github.com/prometheus/promu/releases/download/v$(PROMU_VERSION)/promu-$(PROMU_VERSION).$(GO_BUILD_PLATFORM).tar.gz
+
+GOLANGCI_LINT :=
+GOLANGCI_LINT_OPTS ?=
+GOLANGCI_LINT_VERSION ?= v1.16.0
+# golangci-lint only supports linux, darwin and windows platforms on i386/amd64.
+# windows isn't included here because of the path separator being different.
+ifeq ($(GOHOSTOS),$(filter $(GOHOSTOS),linux darwin))
+	ifeq ($(GOHOSTARCH),$(filter $(GOHOSTARCH),amd64 i386))
+		GOLANGCI_LINT := $(FIRST_GOPATH)/bin/golangci-lint
+	endif
+endif
+
+PREFIX                  ?= $(shell pwd)
+BIN_DIR                 ?= $(shell pwd)
+DOCKER_IMAGE_TAG        ?= $(subst /,-,$(shell git rev-parse --abbrev-ref HEAD))
+DOCKER_REPO             ?= prom
+
+DOCKER_ARCHS            ?= amd64
+
+BUILD_DOCKER_ARCHS = $(addprefix common-docker-,$(DOCKER_ARCHS))
+PUBLISH_DOCKER_ARCHS = $(addprefix common-docker-publish-,$(DOCKER_ARCHS))
+TAG_DOCKER_ARCHS = $(addprefix common-docker-tag-latest-,$(DOCKER_ARCHS))
+
+ifeq ($(GOHOSTARCH),amd64)
+        ifeq ($(GOHOSTOS),$(filter $(GOHOSTOS),linux freebsd darwin windows))
+                # Only supported on amd64
+                test-flags := -race
+        endif
+endif
+
+# This rule is used to forward a target like "build" to "common-build".  This
+# allows a new "build" target to be defined in a Makefile which includes this
+# one and override "common-build" without override warnings.
+%: common-% ;
+
+.PHONY: common-all
+common-all: precheck style check_license lint unused build test
+
+.PHONY: common-style
+common-style:
+	@echo ">> checking code style"
+	@fmtRes=$$($(GOFMT) -d $$(find . -path ./vendor -prune -o -name '*.go' -print)); \
+	if [ -n "$${fmtRes}" ]; then \
+		echo "gofmt checking failed!"; echo "$${fmtRes}"; echo; \
+		echo "Please ensure you are using $$($(GO) version) for formatting code."; \
+		exit 1; \
+	fi
+
+.PHONY: common-check_license
+common-check_license:
+	@echo ">> checking license header"
+	@licRes=$$(for file in $$(find . -type f -iname '*.go' ! -path './vendor/*') ; do \
+               awk 'NR<=3' $$file | grep -Eq "(Copyright|generated|GENERATED)" || echo $$file; \
+       done); \
+       if [ -n "$${licRes}" ]; then \
+               echo "license header checking failed:"; echo "$${licRes}"; \
+               exit 1; \
+       fi
+
+.PHONY: common-deps
+common-deps:
+	@echo ">> getting dependencies"
+ifdef GO111MODULE
+	GO111MODULE=$(GO111MODULE) $(GO) mod download
+else
+	$(GO) get $(GOOPTS) -t ./...
+endif
+
+.PHONY: common-test-short
+common-test-short:
+	@echo ">> running short tests"
+	GO111MODULE=$(GO111MODULE) $(GO) test -short $(GOOPTS) $(pkgs)
+
+.PHONY: common-test
+common-test:
+	@echo ">> running all tests"
+	GO111MODULE=$(GO111MODULE) $(GO) test $(test-flags) $(GOOPTS) $(pkgs)
+
+.PHONY: common-format
+common-format:
+	@echo ">> formatting code"
+	GO111MODULE=$(GO111MODULE) $(GO) fmt $(pkgs)
+
+.PHONY: common-vet
+common-vet:
+	@echo ">> vetting code"
+	GO111MODULE=$(GO111MODULE) $(GO) vet $(GOOPTS) $(pkgs)
+
+.PHONY: common-lint
+common-lint: $(GOLANGCI_LINT)
+ifdef GOLANGCI_LINT
+	@echo ">> running golangci-lint"
+ifdef GO111MODULE
+# 'go list' needs to be executed before staticcheck to prepopulate the modules cache.
+# Otherwise staticcheck might fail randomly for some reason not yet explained.
+	GO111MODULE=$(GO111MODULE) $(GO) list -e -compiled -test=true -export=false -deps=true -find=false -tags= -- ./... > /dev/null
+	GO111MODULE=$(GO111MODULE) $(GOLANGCI_LINT) run $(GOLANGCI_LINT_OPTS) $(pkgs)
+else
+	$(GOLANGCI_LINT) run $(pkgs)
+endif
+endif
+
+# For backward-compatibility.
+.PHONY: common-staticcheck
+common-staticcheck: lint
+
+.PHONY: common-unused
+common-unused: $(GOVENDOR)
+ifdef GOVENDOR
+	@echo ">> running check for unused packages"
+	@$(GOVENDOR) list +unused | grep . && exit 1 || echo 'No unused packages'
+else
+ifdef GO111MODULE
+	@echo ">> running check for unused/missing packages in go.mod"
+	GO111MODULE=$(GO111MODULE) $(GO) mod tidy
+ifeq (,$(wildcard vendor))
+	@git diff --exit-code -- go.sum go.mod
+else
+	@echo ">> running check for unused packages in vendor/"
+	GO111MODULE=$(GO111MODULE) $(GO) mod vendor
+	@git diff --exit-code -- go.sum go.mod vendor/
+endif
+endif
+endif
+
+.PHONY: common-build
+common-build: promu
+	@echo ">> building binaries"
+	GO111MODULE=$(GO111MODULE) $(PROMU) build --prefix $(PREFIX)
+
+.PHONY: common-tarball
+common-tarball: promu
+	@echo ">> building release tarball"
+	$(PROMU) tarball --prefix $(PREFIX) $(BIN_DIR)
+
+.PHONY: common-docker $(BUILD_DOCKER_ARCHS)
+common-docker: $(BUILD_DOCKER_ARCHS)
+$(BUILD_DOCKER_ARCHS): common-docker-%:
+	docker build -t "$(DOCKER_REPO)/$(DOCKER_IMAGE_NAME)-linux-$*:$(DOCKER_IMAGE_TAG)" \
+		--build-arg ARCH="$*" \
+		--build-arg OS="linux" \
+		.
+
+.PHONY: common-docker-publish $(PUBLISH_DOCKER_ARCHS)
+common-docker-publish: $(PUBLISH_DOCKER_ARCHS)
+$(PUBLISH_DOCKER_ARCHS): common-docker-publish-%:
+	docker push "$(DOCKER_REPO)/$(DOCKER_IMAGE_NAME)-linux-$*:$(DOCKER_IMAGE_TAG)"
+
+.PHONY: common-docker-tag-latest $(TAG_DOCKER_ARCHS)
+common-docker-tag-latest: $(TAG_DOCKER_ARCHS)
+$(TAG_DOCKER_ARCHS): common-docker-tag-latest-%:
+	docker tag "$(DOCKER_REPO)/$(DOCKER_IMAGE_NAME)-linux-$*:$(DOCKER_IMAGE_TAG)" "$(DOCKER_REPO)/$(DOCKER_IMAGE_NAME)-linux-$*:latest"
+
+.PHONY: common-docker-manifest
+common-docker-manifest:
+	DOCKER_CLI_EXPERIMENTAL=enabled docker manifest create -a "$(DOCKER_REPO)/$(DOCKER_IMAGE_NAME):$(DOCKER_IMAGE_TAG)" $(foreach ARCH,$(DOCKER_ARCHS),$(DOCKER_REPO)/$(DOCKER_IMAGE_NAME)-linux-$(ARCH):$(DOCKER_IMAGE_TAG))
+	DOCKER_CLI_EXPERIMENTAL=enabled docker manifest push "$(DOCKER_REPO)/$(DOCKER_IMAGE_NAME):$(DOCKER_IMAGE_TAG)"
+
+.PHONY: promu
+promu: $(PROMU)
+
+$(PROMU):
+	$(eval PROMU_TMP := $(shell mktemp -d))
+	curl -s -L $(PROMU_URL) | tar -xvzf - -C $(PROMU_TMP)
+	mkdir -p $(FIRST_GOPATH)/bin
+	cp $(PROMU_TMP)/promu-$(PROMU_VERSION).$(GO_BUILD_PLATFORM)/promu $(FIRST_GOPATH)/bin/promu
+	rm -r $(PROMU_TMP)
+
+.PHONY: proto
+proto:
+	@echo ">> generating code from proto files"
+	@./scripts/genproto.sh
+
+ifdef GOLANGCI_LINT
+$(GOLANGCI_LINT):
+	mkdir -p $(FIRST_GOPATH)/bin
+	curl -sfL https://install.goreleaser.com/github.com/golangci/golangci-lint.sh | sh -s -- -b $(FIRST_GOPATH)/bin $(GOLANGCI_LINT_VERSION)
+endif
+
+ifdef GOVENDOR
+.PHONY: $(GOVENDOR)
+$(GOVENDOR):
+	GOOS= GOARCH= $(GO) get -u github.com/kardianos/govendor
+endif
+
+.PHONY: precheck
+precheck::
+
+define PRECHECK_COMMAND_template =
+precheck:: $(1)_precheck
+
+PRECHECK_COMMAND_$(1) ?= $(1) $$(strip $$(PRECHECK_OPTIONS_$(1)))
+.PHONY: $(1)_precheck
+$(1)_precheck:
+	@if ! $$(PRECHECK_COMMAND_$(1)) 1>/dev/null 2>&1; then \
+		echo "Execution of '$$(PRECHECK_COMMAND_$(1))' command failed. Is $(1) installed?"; \
+		exit 1; \
+	fi
+endef
diff --git a/vendor/github.com/prometheus/procfs/NOTICE b/vendor/github.com/prometheus/procfs/NOTICE
new file mode 100644
index 0000000..53c5e9a
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/NOTICE
@@ -0,0 +1,7 @@
+procfs provides functions to retrieve system, kernel and process
+metrics from the pseudo-filesystem proc.
+
+Copyright 2014-2015 The Prometheus Authors
+
+This product includes software developed at
+SoundCloud Ltd. (http://soundcloud.com/).
diff --git a/vendor/github.com/prometheus/procfs/README.md b/vendor/github.com/prometheus/procfs/README.md
new file mode 100644
index 0000000..6f8850f
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/README.md
@@ -0,0 +1,53 @@
+# procfs
+
+This procfs package provides functions to retrieve system, kernel and process
+metrics from the pseudo-filesystems /proc and /sys.
+
+*WARNING*: This package is a work in progress. Its API may still break in
+backwards-incompatible ways without warnings. Use it at your own risk.
+
+[![GoDoc](https://godoc.org/github.com/prometheus/procfs?status.png)](https://godoc.org/github.com/prometheus/procfs)
+[![Build Status](https://travis-ci.org/prometheus/procfs.svg?branch=master)](https://travis-ci.org/prometheus/procfs)
+[![Go Report Card](https://goreportcard.com/badge/github.com/prometheus/procfs)](https://goreportcard.com/report/github.com/prometheus/procfs)
+
+## Usage
+
+The procfs library is organized by packages based on whether the gathered data is coming from
+/proc, /sys, or both.  Each package contains an `FS` type which represents the path to either /proc, /sys, or both.  For example, current cpu statistics are gathered from
+`/proc/stat` and are available via the root procfs package.  First, the proc filesystem mount
+point is initialized, and then the stat information is read.
+
+```go
+fs, err := procfs.NewFS("/proc")
+stats, err := fs.Stat()
+```
+
+Some sub-packages such as `blockdevice`, require access to both the proc and sys filesystems.
+
+```go
+    fs, err := blockdevice.NewFS("/proc", "/sys")
+    stats, err := fs.ProcDiskstats()
+```
+
+## Building and Testing
+
+The procfs library is normally built as part of another application.  However, when making
+changes to the library, the `make test` command can be used to run the API test suite.
+
+### Updating Test Fixtures
+
+The procfs library includes a set of test fixtures which include many example files from
+the `/proc` and `/sys` filesystems.  These fixtures are included as a [ttar](https://github.com/ideaship/ttar) file
+which is extracted automatically during testing.  To add/update the test fixtures, first
+ensure the `fixtures` directory is up to date by removing the existing directory and then
+extracting the ttar file using `make fixtures/.unpacked` or just `make test`.
+
+```bash
+rm -rf fixtures
+make test
+```
+
+Next, make the required changes to the extracted files in the `fixtures` directory.  When
+the changes are complete, run `make update_fixtures` to create a new `fixtures.ttar` file
+based on the updated `fixtures` directory.  And finally, verify the changes using
+`git diff fixtures.ttar`.
diff --git a/vendor/github.com/prometheus/procfs/buddyinfo.go b/vendor/github.com/prometheus/procfs/buddyinfo.go
new file mode 100644
index 0000000..63d4229
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/buddyinfo.go
@@ -0,0 +1,85 @@
+// Copyright 2017 The Prometheus 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 procfs
+
+import (
+	"bufio"
+	"fmt"
+	"io"
+	"os"
+	"strconv"
+	"strings"
+)
+
+// A BuddyInfo is the details parsed from /proc/buddyinfo.
+// The data is comprised of an array of free fragments of each size.
+// The sizes are 2^n*PAGE_SIZE, where n is the array index.
+type BuddyInfo struct {
+	Node  string
+	Zone  string
+	Sizes []float64
+}
+
+// NewBuddyInfo reads the buddyinfo statistics from the specified `proc` filesystem.
+func (fs FS) BuddyInfo() ([]BuddyInfo, error) {
+	file, err := os.Open(fs.proc.Path("buddyinfo"))
+	if err != nil {
+		return nil, err
+	}
+	defer file.Close()
+
+	return parseBuddyInfo(file)
+}
+
+func parseBuddyInfo(r io.Reader) ([]BuddyInfo, error) {
+	var (
+		buddyInfo   = []BuddyInfo{}
+		scanner     = bufio.NewScanner(r)
+		bucketCount = -1
+	)
+
+	for scanner.Scan() {
+		var err error
+		line := scanner.Text()
+		parts := strings.Fields(line)
+
+		if len(parts) < 4 {
+			return nil, fmt.Errorf("invalid number of fields when parsing buddyinfo")
+		}
+
+		node := strings.TrimRight(parts[1], ",")
+		zone := strings.TrimRight(parts[3], ",")
+		arraySize := len(parts[4:])
+
+		if bucketCount == -1 {
+			bucketCount = arraySize
+		} else {
+			if bucketCount != arraySize {
+				return nil, fmt.Errorf("mismatch in number of buddyinfo buckets, previous count %d, new count %d", bucketCount, arraySize)
+			}
+		}
+
+		sizes := make([]float64, arraySize)
+		for i := 0; i < arraySize; i++ {
+			sizes[i], err = strconv.ParseFloat(parts[i+4], 64)
+			if err != nil {
+				return nil, fmt.Errorf("invalid value in buddyinfo: %s", err)
+			}
+		}
+
+		buddyInfo = append(buddyInfo, BuddyInfo{node, zone, sizes})
+	}
+
+	return buddyInfo, scanner.Err()
+}
diff --git a/vendor/github.com/prometheus/procfs/doc.go b/vendor/github.com/prometheus/procfs/doc.go
new file mode 100644
index 0000000..e2acd6d
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/doc.go
@@ -0,0 +1,45 @@
+// Copyright 2014 Prometheus Team
+// 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 procfs provides functions to retrieve system, kernel and process
+// metrics from the pseudo-filesystem proc.
+//
+// Example:
+//
+//    package main
+//
+//    import (
+//    	"fmt"
+//    	"log"
+//
+//    	"github.com/prometheus/procfs"
+//    )
+//
+//    func main() {
+//    	p, err := procfs.Self()
+//    	if err != nil {
+//    		log.Fatalf("could not get process: %s", err)
+//    	}
+//
+//    	stat, err := p.NewStat()
+//    	if err != nil {
+//    		log.Fatalf("could not get process stat: %s", err)
+//    	}
+//
+//    	fmt.Printf("command:  %s\n", stat.Comm)
+//    	fmt.Printf("cpu time: %fs\n", stat.CPUTime())
+//    	fmt.Printf("vsize:    %dB\n", stat.VirtualMemory())
+//    	fmt.Printf("rss:      %dB\n", stat.ResidentMemory())
+//    }
+//
+package procfs
diff --git a/vendor/github.com/prometheus/procfs/fixtures.ttar b/vendor/github.com/prometheus/procfs/fixtures.ttar
new file mode 100644
index 0000000..951d909
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/fixtures.ttar
@@ -0,0 +1,1808 @@
+# Archive created by ttar -c -f fixtures.ttar fixtures/
+Directory: fixtures
+Mode: 775
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/proc
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/proc/26231
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26231/cmdline
+Lines: 1
+vimNULLBYTEtest.goNULLBYTE+10NULLBYTEEOF
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26231/comm
+Lines: 1
+vim
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26231/cwd
+SymlinkTo: /usr/bin
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26231/exe
+SymlinkTo: /usr/bin/vim
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/proc/26231/fd
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26231/fd/0
+SymlinkTo: ../../symlinktargets/abc
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26231/fd/1
+SymlinkTo: ../../symlinktargets/def
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26231/fd/10
+SymlinkTo: ../../symlinktargets/xyz
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26231/fd/2
+SymlinkTo: ../../symlinktargets/ghi
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26231/fd/3
+SymlinkTo: ../../symlinktargets/uvw
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26231/io
+Lines: 7
+rchar: 750339
+wchar: 818609
+syscr: 7405
+syscw: 5245
+read_bytes: 1024
+write_bytes: 2048
+cancelled_write_bytes: -1024
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26231/limits
+Lines: 17
+Limit                     Soft Limit           Hard Limit           Units
+Max cpu time              unlimited            unlimited            seconds
+Max file size             unlimited            unlimited            bytes
+Max data size             unlimited            unlimited            bytes
+Max stack size            8388608              unlimited            bytes
+Max core file size        0                    unlimited            bytes
+Max resident set          unlimited            unlimited            bytes
+Max processes             62898                62898                processes
+Max open files            2048                 4096                 files
+Max locked memory         65536                65536                bytes
+Max address space         8589934592           unlimited            bytes
+Max file locks            unlimited            unlimited            locks
+Max pending signals       62898                62898                signals
+Max msgqueue size         819200               819200               bytes
+Max nice priority         0                    0
+Max realtime priority     0                    0
+Max realtime timeout      unlimited            unlimited            us
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26231/mountstats
+Lines: 20
+device rootfs mounted on / with fstype rootfs
+device sysfs mounted on /sys with fstype sysfs
+device proc mounted on /proc with fstype proc
+device /dev/sda1 mounted on / with fstype ext4
+device 192.168.1.1:/srv/test mounted on /mnt/nfs/test with fstype nfs4 statvers=1.1
+	opts:	rw,vers=4.0,rsize=1048576,wsize=1048576,namlen=255,acregmin=3,acregmax=60,acdirmin=30,acdirmax=60,hard,proto=tcp,port=0,timeo=600,retrans=2,sec=sys,mountaddr=192.168.1.1,clientaddr=192.168.1.5,local_lock=none
+	age:	13968
+	caps:	caps=0xfff7,wtmult=512,dtsize=32768,bsize=0,namlen=255
+	nfsv4:	bm0=0xfdffafff,bm1=0xf9be3e,bm2=0x0,acl=0x0,pnfs=not configured
+	sec:	flavor=1,pseudoflavor=1
+	events:	52 226 0 0 1 13 398 0 0 331 0 47 0 0 77 0 0 77 0 0 0 0 0 0 0 0 0
+	bytes:	1207640230 0 0 0 1210214218 0 295483 0
+	RPC iostats version: 1.0  p/v: 100003/4 (nfs)
+	xprt:	tcp 832 0 1 0 11 6428 6428 0 12154 0 24 26 5726
+	per-op statistics
+	        NULL: 0 0 0 0 0 0 0 0
+	        READ: 1298 1298 0 207680 1210292152 6 79386 79407
+	       WRITE: 0 0 0 0 0 0 0 0
+	      ACCESS: 2927395007 2927394995 0 526931094212 362996810236 18446743919241604546 1667369447 1953587717
+
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/proc/26231/net
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26231/net/dev
+Lines: 4
+Inter-|   Receive                                                |  Transmit
+ face |bytes    packets errs drop fifo frame compressed multicast|bytes    packets errs drop fifo colls carrier compressed
+    lo:       0       0    0    0    0     0          0         0        0       0    0    0    0     0       0          0
+  eth0:     438       5    0    0    0     0          0         0      648       8    0    0    0     0       0          0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/proc/26231/ns
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26231/ns/mnt
+SymlinkTo: mnt:[4026531840]
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26231/ns/net
+SymlinkTo: net:[4026531993]
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26231/root
+SymlinkTo: /
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26231/stat
+Lines: 1
+26231 (vim) R 5392 7446 5392 34835 7446 4218880 32533 309516 26 82 1677 44 158 99 20 0 1 0 82375 56274944 1981 18446744073709551615 4194304 6294284 140736914091744 140736914087944 139965136429984 0 0 12288 1870679807 0 0 0 17 0 0 0 31 0 0 8391624 8481048 16420864 140736914093252 140736914093279 140736914093279 140736914096107 0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26231/status
+Lines: 53
+
+Name:	prometheus
+Umask:	0022
+State:	S (sleeping)
+Tgid:	1
+Ngid:	0
+Pid:	1
+PPid:	0
+TracerPid:	0
+Uid:	0	0	0	0
+Gid:	0	0	0	0
+FDSize:	128
+Groups:
+NStgid:	1
+NSpid:	1
+NSpgid:	1
+NSsid:	1
+VmPeak:	   58472 kB
+VmSize:	   58440 kB
+VmLck:	       0 kB
+VmPin:	       0 kB
+VmHWM:	    8028 kB
+VmRSS:	    6716 kB
+RssAnon:	    2092 kB
+RssFile:	    4624 kB
+RssShmem:	       0 kB
+VmData:	    2580 kB
+VmStk:	     136 kB
+VmExe:	     948 kB
+VmLib:	    6816 kB
+VmPTE:	     128 kB
+VmPMD:	      12 kB
+VmSwap:	     660 kB
+HugetlbPages:	       0 kB
+Threads:	1
+SigQ:	8/63965
+SigPnd:	0000000000000000
+ShdPnd:	0000000000000000
+SigBlk:	7be3c0fe28014a03
+SigIgn:	0000000000001000
+SigCgt:	00000001800004ec
+CapInh:	0000000000000000
+CapPrm:	0000003fffffffff
+CapEff:	0000003fffffffff
+CapBnd:	0000003fffffffff
+CapAmb:	0000000000000000
+Seccomp:	0
+Cpus_allowed:	ff
+Cpus_allowed_list:	0-7
+Mems_allowed:	00000000,00000001
+Mems_allowed_list:	0
+voluntary_ctxt_switches:	4742839
+nonvoluntary_ctxt_switches:	1727500
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/proc/26232
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26232/cmdline
+Lines: 0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26232/comm
+Lines: 1
+ata_sff
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26232/cwd
+SymlinkTo: /does/not/exist
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/proc/26232/fd
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26232/fd/0
+SymlinkTo: ../../symlinktargets/abc
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26232/fd/1
+SymlinkTo: ../../symlinktargets/def
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26232/fd/2
+SymlinkTo: ../../symlinktargets/ghi
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26232/fd/3
+SymlinkTo: ../../symlinktargets/uvw
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26232/fd/4
+SymlinkTo: ../../symlinktargets/xyz
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26232/limits
+Lines: 17
+Limit                     Soft Limit           Hard Limit           Units
+Max cpu time              unlimited            unlimited            seconds
+Max file size             unlimited            unlimited            bytes
+Max data size             unlimited            unlimited            bytes
+Max stack size            8388608              unlimited            bytes
+Max core file size        0                    unlimited            bytes
+Max resident set          unlimited            unlimited            bytes
+Max processes             29436                29436                processes
+Max open files            1024                 4096                 files
+Max locked memory         65536                65536                bytes
+Max address space         unlimited            unlimited            bytes
+Max file locks            unlimited            unlimited            locks
+Max pending signals       29436                29436                signals
+Max msgqueue size         819200               819200               bytes
+Max nice priority         0                    0
+Max realtime priority     0                    0
+Max realtime timeout      unlimited            unlimited            us
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26232/root
+SymlinkTo: /does/not/exist
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26232/stat
+Lines: 1
+33 (ata_sff) S 2 0 0 0 -1 69238880 0 0 0 0 0 0 0 0 0 -20 1 0 5 0 0 18446744073709551615 0 0 0 0 0 0 0 2147483647 0 18446744073709551615 0 0 17 1 0 0 0 0 0 0 0 0 0 0 0 0 0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/proc/26233
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/26233/cmdline
+Lines: 1
+com.github.uiautomatorNULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTENULLBYTEEOF
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/proc/584
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/584/stat
+Lines: 2
+1020 ((a b ) ( c d) ) R 28378 1020 28378 34842 1020 4218880 286 0 0 0 0 0 0 0 20 0 1 0 10839175 10395648 155 18446744073709551615 4194304 4238788 140736466511168 140736466511168 140609271124624 0 0 0 0 0 0 0 17 5 0 0 0 0 0 6336016 6337300 25579520 140736466515030 140736466515061 140736466515061 140736466518002 0
+#!/bin/cat /proc/self/stat
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/buddyinfo
+Lines: 3
+Node 0, zone      DMA      1      0      1      0      2      1      1      0      1      1      3
+Node 0, zone    DMA32    759    572    791    475    194     45     12      0      0      0      0
+Node 0, zone   Normal   4381   1093    185   1530    567    102      4      0      0      0      0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/diskstats
+Lines: 49
+   1       0 ram0 0 0 0 0 0 0 0 0 0 0 0
+   1       1 ram1 0 0 0 0 0 0 0 0 0 0 0
+   1       2 ram2 0 0 0 0 0 0 0 0 0 0 0
+   1       3 ram3 0 0 0 0 0 0 0 0 0 0 0
+   1       4 ram4 0 0 0 0 0 0 0 0 0 0 0
+   1       5 ram5 0 0 0 0 0 0 0 0 0 0 0
+   1       6 ram6 0 0 0 0 0 0 0 0 0 0 0
+   1       7 ram7 0 0 0 0 0 0 0 0 0 0 0
+   1       8 ram8 0 0 0 0 0 0 0 0 0 0 0
+   1       9 ram9 0 0 0 0 0 0 0 0 0 0 0
+   1      10 ram10 0 0 0 0 0 0 0 0 0 0 0
+   1      11 ram11 0 0 0 0 0 0 0 0 0 0 0
+   1      12 ram12 0 0 0 0 0 0 0 0 0 0 0
+   1      13 ram13 0 0 0 0 0 0 0 0 0 0 0
+   1      14 ram14 0 0 0 0 0 0 0 0 0 0 0
+   1      15 ram15 0 0 0 0 0 0 0 0 0 0 0
+   7       0 loop0 0 0 0 0 0 0 0 0 0 0 0
+   7       1 loop1 0 0 0 0 0 0 0 0 0 0 0
+   7       2 loop2 0 0 0 0 0 0 0 0 0 0 0
+   7       3 loop3 0 0 0 0 0 0 0 0 0 0 0
+   7       4 loop4 0 0 0 0 0 0 0 0 0 0 0
+   7       5 loop5 0 0 0 0 0 0 0 0 0 0 0
+   7       6 loop6 0 0 0 0 0 0 0 0 0 0 0
+   7       7 loop7 0 0 0 0 0 0 0 0 0 0 0
+   8       0 sda 25354637 34367663 1003346126 18492372 28444756 11134226 505697032 63877960 0 9653880 82621804
+   8       1 sda1 250 0 2000 36 0 0 0 0 0 36 36
+   8       2 sda2 246 0 1968 32 0 0 0 0 0 32 32
+   8       3 sda3 340 13 2818 52 11 8 152 8 0 56 60
+   8       4 sda4 25353629 34367650 1003337964 18492232 27448755 11134218 505696880 61593380 0 7576432 80332428
+ 252       0 dm-0 59910002 0 1003337218 46229572 39231014 0 505696880 1158557800 0 11325968 1206301256
+ 252       1 dm-1 388 0 3104 84 74 0 592 0 0 76 84
+ 252       2 dm-2 11571 0 308350 6536 153522 0 5093416 122884 0 65400 129416
+ 252       3 dm-3 3870 0 3870 104 0 0 0 0 0 16 104
+ 252       4 dm-4 392 0 1034 28 38 0 137 16 0 24 44
+ 252       5 dm-5 3729 0 84279 924 98918 0 1151688 104684 0 58848 105632
+ 179       0 mmcblk0 192 3 1560 156 0 0 0 0 0 136 156
+ 179       1 mmcblk0p1 17 3 160 24 0 0 0 0 0 24 24
+ 179       2 mmcblk0p2 95 0 760 68 0 0 0 0 0 68 68
+   2       0 fd0 2 0 16 80 0 0 0 0 0 80 80
+ 254       0 vda 1775784 15386 32670882 8655768 6038856 20711856 213637440 2069221364 0 41614592 2077872228
+ 254       1 vda1 668 85 5984 956 207 4266 35784 32772 0 8808 33720
+ 254       2 vda2 1774936 15266 32663262 8654692 5991028 20707590 213601656 2069152216 0 41607628 2077801992
+  11       0 sr0 0 0 0 0 0 0 0 0 0 0 0
+ 259       0 nvme0n1 47114 4 4643973 21650 1078320 43950 39451633 1011053 0 222766 1032546
+ 259       1 nvme0n1p1 1140 0 9370 16 1 0 1 0 0 16 16
+ 259       2 nvme0n1p2 45914 4 4631243 21626 1036885 43950 39451632 919480 0 131580 940970
+   8       0 sdb 326552 841 9657779 84 41822 2895 1972905 5007 0 60730 67070 68851 0 1925173784 11130
+   8       1 sdb1 231 3 34466 4 24 23 106 0 0 64 64 0 0 0 0
+   8       2 sdb2 326310 838 9622281 67 40726 2872 1972799 4924 0 58250 64567 68851 0 1925173784 11130
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/proc/fs
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/proc/fs/xfs
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/fs/xfs/stat
+Lines: 23
+extent_alloc 92447 97589 92448 93751
+abt 0 0 0 0
+blk_map 1767055 188820 184891 92447 92448 2140766 0
+bmbt 0 0 0 0
+dir 185039 92447 92444 136422
+trans 706 944304 0
+ig 185045 58807 0 126238 0 33637 22
+log 2883 113448 9 17360 739
+push_ail 945014 0 134260 15483 0 3940 464 159985 0 40
+xstrat 92447 0
+rw 107739 94045
+attr 4 0 0 0
+icluster 8677 7849 135802
+vnodes 92601 0 0 0 92444 92444 92444 0
+buf 2666287 7122 2659202 3599 2 7085 0 10297 7085
+abtb2 184941 1277345 13257 13278 0 0 0 0 0 0 0 0 0 0 2746147
+abtc2 345295 2416764 172637 172658 0 0 0 0 0 0 0 0 0 0 21406023
+bmbt2 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
+ibt2 343004 1358467 0 0 0 0 0 0 0 0 0 0 0 0 0
+fibt2 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
+qm 0 0 0 0 0 0 0 0
+xpc 399724544 92823103 86219234
+debug 0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/mdstat
+Lines: 26
+Personalities : [linear] [multipath] [raid0] [raid1] [raid6] [raid5] [raid4] [raid10]
+md3 : active raid6 sda1[8] sdh1[7] sdg1[6] sdf1[5] sde1[11] sdd1[3] sdc1[10] sdb1[9]
+      5853468288 blocks super 1.2 level 6, 64k chunk, algorithm 2 [8/8] [UUUUUUUU]
+
+md127 : active raid1 sdi2[0] sdj2[1]
+      312319552 blocks [2/2] [UU]
+
+md0 : active raid1 sdk[2](S) sdi1[0] sdj1[1]
+      248896 blocks [2/2] [UU]
+
+md4 : inactive raid1 sda3[0] sdb3[1]
+      4883648 blocks [2/2] [UU]
+
+md6 : active raid1 sdb2[2] sda2[0]
+      195310144 blocks [2/1] [U_]
+      [=>...................]  recovery =  8.5% (16775552/195310144) finish=17.0min speed=259783K/sec
+
+md8 : active raid1 sdb1[1] sda1[0]
+      195310144 blocks [2/2] [UU]
+      [=>...................]  resync =  8.5% (16775552/195310144) finish=17.0min speed=259783K/sec
+
+md7 : active raid6 sdb1[0] sde1[3] sdd1[2] sdc1[1]
+      7813735424 blocks super 1.2 level 6, 512k chunk, algorithm 2 [4/3] [U_UU]
+      bitmap: 0/30 pages [0KB], 65536KB chunk
+
+unused devices: <none>
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/proc/net
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/net/dev
+Lines: 6
+Inter-|   Receive                                                |  Transmit
+ face |bytes    packets errs drop fifo frame compressed multicast|bytes    packets errs drop fifo colls carrier compressed
+vethf345468:     648       8    0    0    0     0          0         0      438       5    0    0    0     0       0          0
+    lo: 1664039048 1566805    0    0    0     0          0         0 1664039048 1566805    0    0    0     0       0          0
+docker0:    2568      38    0    0    0     0          0         0      438       5    0    0    0     0       0          0
+  eth0: 874354587 1036395    0    0    0     0          0         0 563352563  732147    0    0    0     0       0          0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/net/ip_vs
+Lines: 21
+IP Virtual Server version 1.2.1 (size=4096)
+Prot LocalAddress:Port Scheduler Flags
+  -> RemoteAddress:Port Forward Weight ActiveConn InActConn
+TCP  C0A80016:0CEA wlc
+  -> C0A85216:0CEA      Tunnel  100    248        2
+  -> C0A85318:0CEA      Tunnel  100    248        2
+  -> C0A85315:0CEA      Tunnel  100    248        1
+TCP  C0A80039:0CEA wlc
+  -> C0A85416:0CEA      Tunnel  0      0          0
+  -> C0A85215:0CEA      Tunnel  100    1499       0
+  -> C0A83215:0CEA      Tunnel  100    1498       0
+TCP  C0A80037:0CEA wlc
+  -> C0A8321A:0CEA      Tunnel  0      0          0
+  -> C0A83120:0CEA      Tunnel  100    0          0
+TCP  [2620:0000:0000:0000:0000:0000:0000:0001]:0050 sh
+  -> [2620:0000:0000:0000:0000:0000:0000:0002]:0050      Route   1      0          0
+  -> [2620:0000:0000:0000:0000:0000:0000:0003]:0050      Route   1      0          0
+  -> [2620:0000:0000:0000:0000:0000:0000:0004]:0050      Route   1      1          1
+FWM  10001000 wlc
+  -> C0A8321A:0CEA      Route   0      0          1
+  -> C0A83215:0CEA      Route   0      0          2
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/net/ip_vs_stats
+Lines: 6
+   Total Incoming Outgoing         Incoming         Outgoing
+   Conns  Packets  Packets            Bytes            Bytes
+ 16AA370 E33656E5        0     51D8C8883AB3                0
+
+ Conns/s   Pkts/s   Pkts/s          Bytes/s          Bytes/s
+       4    1FB3C        0          1282A8F                0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/proc/net/rpc
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/net/rpc/nfs
+Lines: 5
+net 18628 0 18628 6
+rpc 4329785 0 4338291
+proc2 18 2 69 0 0 4410 0 0 0 0 0 0 0 0 0 0 0 99 2
+proc3 22 1 4084749 29200 94754 32580 186 47747 7981 8639 0 6356 0 6962 0 7958 0 0 241 4 4 2 39
+proc4 61 1 0 0 0 0 0 0 0 0 0 0 0 1 1 0 0 0 0 0 0 0 2 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/net/rpc/nfsd
+Lines: 11
+rc 0 6 18622
+fh 0 0 0 0 0
+io 157286400 0
+th 8 0 0.000 0.000 0.000 0.000 0.000 0.000 0.000 0.000 0.000 0.000
+ra 32 0 0 0 0 0 0 0 0 0 0 0
+net 18628 0 18628 6
+rpc 18628 0 0 0 0
+proc2 18 2 69 0 0 4410 0 0 0 0 0 0 0 0 0 0 0 99 2
+proc3 22 2 112 0 2719 111 0 0 0 0 0 0 0 0 0 0 0 27 216 0 2 1 0
+proc4 2 2 10853
+proc4ops 72 0 0 0 1098 2 0 0 0 0 8179 5896 0 0 0 0 5900 0 0 2 0 2 0 9609 0 2 150 1272 0 0 0 1236 0 0 0 0 3 3 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/net/unix
+Lines: 6
+Num       RefCount Protocol Flags    Type St Inode Path
+0000000000000000: 00000002 00000000 00010000 0001 01 3442596 /var/run/postgresql/.s.PGSQL.5432
+0000000000000000: 0000000a 00000000 00010000 0005 01 10061 /run/udev/control
+0000000000000000: 00000007 00000000 00000000 0002 01 12392 /dev/log
+0000000000000000: 00000003 00000000 00000000 0001 03 4787297 /var/run/postgresql/.s.PGSQL.5432
+0000000000000000: 00000003 00000000 00000000 0001 03 5091797
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/net/unix_without_inode
+Lines: 6
+Num       RefCount Protocol Flags    Type St Path
+0000000000000000: 00000002 00000000 00010000 0001 01 /var/run/postgresql/.s.PGSQL.5432
+0000000000000000: 0000000a 00000000 00010000 0005 01 /run/udev/control
+0000000000000000: 00000007 00000000 00000000 0002 01 /dev/log
+0000000000000000: 00000003 00000000 00000000 0001 03 /var/run/postgresql/.s.PGSQL.5432
+0000000000000000: 00000003 00000000 00000000 0001 03
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/net/xfrm_stat
+Lines: 28
+XfrmInError                     1
+XfrmInBufferError               2
+XfrmInHdrError                  4
+XfrmInNoStates                  3
+XfrmInStateProtoError           40
+XfrmInStateModeError            100
+XfrmInStateSeqError             6000
+XfrmInStateExpired              4
+XfrmInStateMismatch             23451
+XfrmInStateInvalid              55555
+XfrmInTmplMismatch              51
+XfrmInNoPols                    65432
+XfrmInPolBlock                  100
+XfrmInPolError                  10000
+XfrmOutError                    1000000
+XfrmOutBundleGenError           43321
+XfrmOutBundleCheckError         555
+XfrmOutNoStates                 869
+XfrmOutStateProtoError          4542
+XfrmOutStateModeError           4
+XfrmOutStateSeqError            543
+XfrmOutStateExpired             565
+XfrmOutPolBlock                 43456
+XfrmOutPolDead                  7656
+XfrmOutPolError                 1454
+XfrmFwdHdrError                 6654
+XfrmOutStateInvalid             28765
+XfrmAcquireError                24532
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/proc/pressure
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/pressure/cpu
+Lines: 1
+some avg10=0.10 avg60=2.00 avg300=3.85 total=15
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/pressure/io
+Lines: 2
+some avg10=0.10 avg60=2.00 avg300=3.85 total=15
+full avg10=0.20 avg60=3.00 avg300=4.95 total=25
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/pressure/memory
+Lines: 2
+some avg10=0.10 avg60=2.00 avg300=3.85 total=15
+full avg10=0.20 avg60=3.00 avg300=4.95 total=25
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/self
+SymlinkTo: 26231
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/stat
+Lines: 16
+cpu  301854 612 111922 8979004 3552 2 3944 0 0 0
+cpu0 44490 19 21045 1087069 220 1 3410 0 0 0
+cpu1 47869 23 16474 1110787 591 0 46 0 0 0
+cpu2 46504 36 15916 1112321 441 0 326 0 0 0
+cpu3 47054 102 15683 1113230 533 0 60 0 0 0
+cpu4 28413 25 10776 1140321 217 0 8 0 0 0
+cpu5 29271 101 11586 1136270 672 0 30 0 0 0
+cpu6 29152 36 10276 1139721 319 0 29 0 0 0
+cpu7 29098 268 10164 1139282 555 0 31 0 0 0
+intr 8885917 17 0 0 0 0 0 0 0 1 79281 0 0 0 0 0 0 0 231237 0 0 0 0 250586 103 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 223424 190745 13 906 1283803 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
+ctxt 38014093
+btime 1418183276
+processes 26442
+procs_running 2
+procs_blocked 1
+softirq 5057579 250191 1481983 1647 211099 186066 0 1783454 622196 12499 508444
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/proc/symlinktargets
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/symlinktargets/README
+Lines: 2
+This directory contains some empty files that are the symlinks the files in the "fd" directory point to.
+They are otherwise ignored by the tests
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/symlinktargets/abc
+Lines: 0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/symlinktargets/def
+Lines: 0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/symlinktargets/ghi
+Lines: 0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/symlinktargets/uvw
+Lines: 0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/proc/symlinktargets/xyz
+Lines: 0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/block
+Mode: 775
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/block/dm-0
+Mode: 775
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/block/dm-0/stat
+Lines: 1
+6447303        0 710266738  1529043   953216        0 31201176  4557464        0   796160  6088971
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/block/sda
+Mode: 775
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/block/sda/stat
+Lines: 1
+9652963   396792 759304206   412943  8422549  6731723 286915323 13947418        0  5658367 19174573 1 2 3 12
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/class
+Mode: 775
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/class/net
+Mode: 775
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/class/net/eth0
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/addr_assign_type
+Lines: 1
+3
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/addr_len
+Lines: 1
+6
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/address
+Lines: 1
+01:01:01:01:01:01
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/broadcast
+Lines: 1
+ff:ff:ff:ff:ff:ff
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/carrier
+Lines: 1
+1
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/carrier_changes
+Lines: 1
+2
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/carrier_down_count
+Lines: 1
+1
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/carrier_up_count
+Lines: 1
+1
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/dev_id
+Lines: 1
+0x20
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/dormant
+Lines: 1
+1
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/duplex
+Lines: 1
+full
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/flags
+Lines: 1
+0x1303
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/ifalias
+Lines: 0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/ifindex
+Lines: 1
+2
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/iflink
+Lines: 1
+2
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/link_mode
+Lines: 1
+1
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/mtu
+Lines: 1
+1500
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/name_assign_type
+Lines: 1
+2
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/netdev_group
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/operstate
+Lines: 1
+up
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/phys_port_id
+Lines: 0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/phys_port_name
+Lines: 0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/phys_switch_id
+Lines: 0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/speed
+Lines: 1
+1000
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/tx_queue_len
+Lines: 1
+1000
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/net/eth0/type
+Lines: 1
+1
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/class/power_supply
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/class/power_supply/AC
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/AC/online
+Lines: 1
+0
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/AC/type
+Lines: 1
+Mains
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/AC/uevent
+Lines: 2
+POWER_SUPPLY_NAME=AC
+POWER_SUPPLY_ONLINE=0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/class/power_supply/BAT0
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/alarm
+Lines: 1
+2503000
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/capacity
+Lines: 1
+98
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/capacity_level
+Lines: 1
+Normal
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/charge_start_threshold
+Lines: 1
+95
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/charge_stop_threshold
+Lines: 1
+100
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/cycle_count
+Lines: 1
+0
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/energy_full
+Lines: 1
+50060000
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/energy_full_design
+Lines: 1
+47520000
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/energy_now
+Lines: 1
+49450000
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/manufacturer
+Lines: 1
+LGC
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/model_name
+Lines: 1
+LNV-45N1
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/power_now
+Lines: 1
+4830000
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/present
+Lines: 1
+1
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/serial_number
+Lines: 1
+38109
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/status
+Lines: 1
+Discharging
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/technology
+Lines: 1
+Li-ion
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/type
+Lines: 1
+Battery
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/uevent
+Lines: 16
+POWER_SUPPLY_NAME=BAT0
+POWER_SUPPLY_STATUS=Discharging
+POWER_SUPPLY_PRESENT=1
+POWER_SUPPLY_TECHNOLOGY=Li-ion
+POWER_SUPPLY_CYCLE_COUNT=0
+POWER_SUPPLY_VOLTAGE_MIN_DESIGN=10800000
+POWER_SUPPLY_VOLTAGE_NOW=12229000
+POWER_SUPPLY_POWER_NOW=4830000
+POWER_SUPPLY_ENERGY_FULL_DESIGN=47520000
+POWER_SUPPLY_ENERGY_FULL=50060000
+POWER_SUPPLY_ENERGY_NOW=49450000
+POWER_SUPPLY_CAPACITY=98
+POWER_SUPPLY_CAPACITY_LEVEL=Normal
+POWER_SUPPLY_MODEL_NAME=LNV-45N1
+POWER_SUPPLY_MANUFACTURER=LGC
+POWER_SUPPLY_SERIAL_NUMBER=38109
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/voltage_min_design
+Lines: 1
+10800000
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/power_supply/BAT0/voltage_now
+Lines: 1
+12229000
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/class/thermal
+Mode: 775
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/class/thermal/thermal_zone0
+Mode: 775
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/thermal/thermal_zone0/policy
+Lines: 1
+step_wise
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/thermal/thermal_zone0/temp
+Lines: 1
+49925
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/thermal/thermal_zone0/type
+Lines: 1
+bcm2835_thermal
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/class/thermal/thermal_zone1
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/thermal/thermal_zone1/mode
+Lines: 1
+enabled
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/thermal/thermal_zone1/passive
+Lines: 1
+0
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/thermal/thermal_zone1/policy
+Lines: 1
+step_wise
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/thermal/thermal_zone1/temp
+Lines: 1
+44000
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/class/thermal/thermal_zone1/type
+Lines: 1
+acpitz
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/dirty_data
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_day
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_day/bypassed
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_day/cache_bypass_hits
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_day/cache_bypass_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_day/cache_hit_ratio
+Lines: 1
+100
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_day/cache_hits
+Lines: 1
+289
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_day/cache_miss_collisions
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_day/cache_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_day/cache_readaheads
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_five_minute
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_five_minute/bypassed
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_five_minute/cache_bypass_hits
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_five_minute/cache_bypass_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_five_minute/cache_hit_ratio
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_five_minute/cache_hits
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_five_minute/cache_miss_collisions
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_five_minute/cache_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_five_minute/cache_readaheads
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_hour
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_hour/bypassed
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_hour/cache_bypass_hits
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_hour/cache_bypass_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_hour/cache_hit_ratio
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_hour/cache_hits
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_hour/cache_miss_collisions
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_hour/cache_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_hour/cache_readaheads
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_total
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_total/bypassed
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_total/cache_bypass_hits
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_total/cache_bypass_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_total/cache_hit_ratio
+Lines: 1
+100
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_total/cache_hits
+Lines: 1
+546
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_total/cache_miss_collisions
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_total/cache_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata4/host3/target3:0:0/3:0:0:0/block/sdb/bcache/stats_total/cache_readaheads
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata5
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata5/host4
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata5/host4/target4:0:0
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata5/host4/target4:0:0/4:0:0:0
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata5/host4/target4:0:0/4:0:0:0/block
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata5/host4/target4:0:0/4:0:0:0/block/sdc
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata5/host4/target4:0:0/4:0:0:0/block/sdc/bcache
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata5/host4/target4:0:0/4:0:0:0/block/sdc/bcache/io_errors
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata5/host4/target4:0:0/4:0:0:0/block/sdc/bcache/metadata_written
+Lines: 1
+512
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata5/host4/target4:0:0/4:0:0:0/block/sdc/bcache/priority_stats
+Lines: 5
+Unused:		99%
+Metadata:	0%
+Average:	10473
+Sectors per Q:	64
+Quantiles:	[0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 20946 20946 20946 20946 20946 20946 20946 20946 20946 20946 20946 20946 20946 20946 20946 20946]
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/pci0000:00/0000:00:0d.0/ata5/host4/target4:0:0/4:0:0:0/block/sdc/bcache/written
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/system
+Mode: 775
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/system/clocksource
+Mode: 775
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/system/clocksource/clocksource0
+Mode: 775
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/clocksource/clocksource0/available_clocksource
+Lines: 1
+tsc hpet acpi_pm 
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/clocksource/clocksource0/current_clocksource
+Lines: 1
+tsc
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/system/cpu
+Mode: 775
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/system/cpu/cpu0
+Mode: 775
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpu0/cpufreq
+SymlinkTo: ../cpufreq/policy0
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/system/cpu/cpu1
+Mode: 775
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/system/cpu/cpu1/cpufreq
+Mode: 775
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpu1/cpufreq/cpuinfo_cur_freq
+Lines: 1
+1200195
+Mode: 400
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpu1/cpufreq/cpuinfo_max_freq
+Lines: 1
+3300000
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpu1/cpufreq/cpuinfo_min_freq
+Lines: 1
+1200000
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpu1/cpufreq/cpuinfo_transition_latency
+Lines: 1
+4294967295
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpu1/cpufreq/related_cpus
+Lines: 1
+1
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpu1/cpufreq/scaling_available_governors
+Lines: 1
+performance powersave
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpu1/cpufreq/scaling_driver
+Lines: 1
+intel_pstate
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpu1/cpufreq/scaling_governor
+Lines: 1
+powersave
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpu1/cpufreq/scaling_max_freq
+Lines: 1
+3300000
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpu1/cpufreq/scaling_min_freq
+Lines: 1
+1200000
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpu1/cpufreq/scaling_setspeed
+Lines: 1
+<unsupported>
+Mode: 664
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/system/cpu/cpufreq
+Mode: 775
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/system/cpu/cpufreq/policy0
+Mode: 775
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpufreq/policy0/affected_cpus
+Lines: 1
+0
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpufreq/policy0/cpuinfo_max_freq
+Lines: 1
+2400000
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpufreq/policy0/cpuinfo_min_freq
+Lines: 1
+800000
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpufreq/policy0/cpuinfo_transition_latency
+Lines: 1
+0
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpufreq/policy0/related_cpus
+Lines: 1
+0
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpufreq/policy0/scaling_available_governors
+Lines: 1
+performance powersave
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpufreq/policy0/scaling_cur_freq
+Lines: 1
+1219917
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpufreq/policy0/scaling_driver
+Lines: 1
+intel_pstate
+Mode: 444
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpufreq/policy0/scaling_governor
+Lines: 1
+powersave
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpufreq/policy0/scaling_max_freq
+Lines: 1
+2400000
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpufreq/policy0/scaling_min_freq
+Lines: 1
+800000
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/devices/system/cpu/cpufreq/policy0/scaling_setspeed
+Lines: 1
+<unsupported>
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/devices/system/cpu/cpufreq/policy1
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs/bcache
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/average_key_size
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0
+Mode: 777
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/dirty_data
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_day
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_day/bypassed
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_day/cache_bypass_hits
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_day/cache_bypass_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_day/cache_hit_ratio
+Lines: 1
+100
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_day/cache_hits
+Lines: 1
+289
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_day/cache_miss_collisions
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_day/cache_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_day/cache_readaheads
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_five_minute
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_five_minute/bypassed
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_five_minute/cache_bypass_hits
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_five_minute/cache_bypass_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_five_minute/cache_hit_ratio
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_five_minute/cache_hits
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_five_minute/cache_miss_collisions
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_five_minute/cache_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_five_minute/cache_readaheads
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_hour
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_hour/bypassed
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_hour/cache_bypass_hits
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_hour/cache_bypass_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_hour/cache_hit_ratio
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_hour/cache_hits
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_hour/cache_miss_collisions
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_hour/cache_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_hour/cache_readaheads
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_total
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_total/bypassed
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_total/cache_bypass_hits
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_total/cache_bypass_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_total/cache_hit_ratio
+Lines: 1
+100
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_total/cache_hits
+Lines: 1
+546
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_total/cache_miss_collisions
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_total/cache_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/bdev0/stats_total/cache_readaheads
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/btree_cache_size
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/cache0
+Mode: 777
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/cache0/io_errors
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/cache0/metadata_written
+Lines: 1
+512
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/cache0/priority_stats
+Lines: 5
+Unused:		99%
+Metadata:	0%
+Average:	10473
+Sectors per Q:	64
+Quantiles:	[0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 20946 20946 20946 20946 20946 20946 20946 20946 20946 20946 20946 20946 20946 20946 20946 20946]
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/cache0/written
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/cache_available_percent
+Lines: 1
+100
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/congested
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/internal
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/internal/active_journal_entries
+Lines: 1
+1
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/internal/btree_nodes
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/internal/btree_read_average_duration_us
+Lines: 1
+1305
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/internal/cache_read_races
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/root_usage_percent
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_day
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_day/bypassed
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_day/cache_bypass_hits
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_day/cache_bypass_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_day/cache_hit_ratio
+Lines: 1
+100
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_day/cache_hits
+Lines: 1
+289
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_day/cache_miss_collisions
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_day/cache_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_day/cache_readaheads
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_five_minute
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_five_minute/bypassed
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_five_minute/cache_bypass_hits
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_five_minute/cache_bypass_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_five_minute/cache_hit_ratio
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_five_minute/cache_hits
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_five_minute/cache_miss_collisions
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_five_minute/cache_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_five_minute/cache_readaheads
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_hour
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_hour/bypassed
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_hour/cache_bypass_hits
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_hour/cache_bypass_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_hour/cache_hit_ratio
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_hour/cache_hits
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_hour/cache_miss_collisions
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_hour/cache_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_hour/cache_readaheads
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_total
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_total/bypassed
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_total/cache_bypass_hits
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_total/cache_bypass_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_total/cache_hit_ratio
+Lines: 1
+100
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_total/cache_hits
+Lines: 1
+546
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_total/cache_miss_collisions
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_total/cache_misses
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/stats_total/cache_readaheads
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/bcache/deaddd54-c735-46d5-868e-f331c5fd7c74/tree_depth
+Lines: 1
+0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs/xfs
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs/xfs/sda1
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs/xfs/sda1/stats
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/xfs/sda1/stats/stats
+Lines: 1
+extent_alloc 1 0 0 0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs/xfs/sdb1
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Directory: fixtures/sys/fs/xfs/sdb1/stats
+Mode: 755
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
+Path: fixtures/sys/fs/xfs/sdb1/stats/stats
+Lines: 1
+extent_alloc 2 0 0 0
+Mode: 644
+# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
diff --git a/vendor/github.com/prometheus/procfs/fs.go b/vendor/github.com/prometheus/procfs/fs.go
new file mode 100644
index 0000000..0102ab0
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/fs.go
@@ -0,0 +1,43 @@
+// Copyright 2018 The Prometheus 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 procfs
+
+import (
+	"github.com/prometheus/procfs/internal/fs"
+)
+
+// FS represents the pseudo-filesystem sys, which provides an interface to
+// kernel data structures.
+type FS struct {
+	proc fs.FS
+}
+
+// DefaultMountPoint is the common mount point of the proc filesystem.
+const DefaultMountPoint = fs.DefaultProcMountPoint
+
+// NewDefaultFS returns a new proc FS mounted under the default proc mountPoint.
+// It will error if the mount point directory can't be read or is a file.
+func NewDefaultFS() (FS, error) {
+	return NewFS(DefaultMountPoint)
+}
+
+// NewFS returns a new proc FS mounted under the given proc mountPoint. It will error
+// if the mount point directory can't be read or is a file.
+func NewFS(mountPoint string) (FS, error) {
+	fs, err := fs.NewFS(mountPoint)
+	if err != nil {
+		return FS{}, err
+	}
+	return FS{fs}, nil
+}
diff --git a/vendor/github.com/prometheus/procfs/go.mod b/vendor/github.com/prometheus/procfs/go.mod
new file mode 100644
index 0000000..8a1b839
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/go.mod
@@ -0,0 +1,3 @@
+module github.com/prometheus/procfs
+
+require golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4
diff --git a/vendor/github.com/prometheus/procfs/go.sum b/vendor/github.com/prometheus/procfs/go.sum
new file mode 100644
index 0000000..7827dd3
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/go.sum
@@ -0,0 +1,2 @@
+golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4 h1:YUO/7uOKsKeq9UokNS62b8FYywz3ker1l1vDZRCRefw=
+golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
diff --git a/vendor/github.com/prometheus/procfs/internal/fs/fs.go b/vendor/github.com/prometheus/procfs/internal/fs/fs.go
new file mode 100644
index 0000000..c66a1cf
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/internal/fs/fs.go
@@ -0,0 +1,52 @@
+// Copyright 2019 The Prometheus 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 fs
+
+import (
+	"fmt"
+	"os"
+	"path/filepath"
+)
+
+const (
+	// DefaultProcMountPoint is the common mount point of the proc filesystem.
+	DefaultProcMountPoint = "/proc"
+
+	// DefaultSysMountPoint is the common mount point of the sys filesystem.
+	DefaultSysMountPoint = "/sys"
+)
+
+// FS represents a pseudo-filesystem, normally /proc or /sys, which provides an
+// interface to kernel data structures.
+type FS string
+
+// NewFS returns a new FS mounted under the given mountPoint. It will error
+// if the mount point can't be read.
+func NewFS(mountPoint string) (FS, error) {
+	info, err := os.Stat(mountPoint)
+	if err != nil {
+		return "", fmt.Errorf("could not read %s: %s", mountPoint, err)
+	}
+	if !info.IsDir() {
+		return "", fmt.Errorf("mount point %s is not a directory", mountPoint)
+	}
+
+	return FS(mountPoint), nil
+}
+
+// Path appends the given path elements to the filesystem path, adding separators
+// as necessary.
+func (fs FS) Path(p ...string) string {
+	return filepath.Join(append([]string{string(fs)}, p...)...)
+}
diff --git a/vendor/github.com/prometheus/procfs/ipvs.go b/vendor/github.com/prometheus/procfs/ipvs.go
new file mode 100644
index 0000000..2d6cb8d
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/ipvs.go
@@ -0,0 +1,239 @@
+// Copyright 2018 The Prometheus 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 procfs
+
+import (
+	"bufio"
+	"encoding/hex"
+	"errors"
+	"fmt"
+	"io"
+	"io/ioutil"
+	"net"
+	"os"
+	"strconv"
+	"strings"
+)
+
+// IPVSStats holds IPVS statistics, as exposed by the kernel in `/proc/net/ip_vs_stats`.
+type IPVSStats struct {
+	// Total count of connections.
+	Connections uint64
+	// Total incoming packages processed.
+	IncomingPackets uint64
+	// Total outgoing packages processed.
+	OutgoingPackets uint64
+	// Total incoming traffic.
+	IncomingBytes uint64
+	// Total outgoing traffic.
+	OutgoingBytes uint64
+}
+
+// IPVSBackendStatus holds current metrics of one virtual / real address pair.
+type IPVSBackendStatus struct {
+	// The local (virtual) IP address.
+	LocalAddress net.IP
+	// The remote (real) IP address.
+	RemoteAddress net.IP
+	// The local (virtual) port.
+	LocalPort uint16
+	// The remote (real) port.
+	RemotePort uint16
+	// The local firewall mark
+	LocalMark string
+	// The transport protocol (TCP, UDP).
+	Proto string
+	// The current number of active connections for this virtual/real address pair.
+	ActiveConn uint64
+	// The current number of inactive connections for this virtual/real address pair.
+	InactConn uint64
+	// The current weight of this virtual/real address pair.
+	Weight uint64
+}
+
+// IPVSStats reads the IPVS statistics from the specified `proc` filesystem.
+func (fs FS) IPVSStats() (IPVSStats, error) {
+	file, err := os.Open(fs.proc.Path("net/ip_vs_stats"))
+	if err != nil {
+		return IPVSStats{}, err
+	}
+	defer file.Close()
+
+	return parseIPVSStats(file)
+}
+
+// parseIPVSStats performs the actual parsing of `ip_vs_stats`.
+func parseIPVSStats(file io.Reader) (IPVSStats, error) {
+	var (
+		statContent []byte
+		statLines   []string
+		statFields  []string
+		stats       IPVSStats
+	)
+
+	statContent, err := ioutil.ReadAll(file)
+	if err != nil {
+		return IPVSStats{}, err
+	}
+
+	statLines = strings.SplitN(string(statContent), "\n", 4)
+	if len(statLines) != 4 {
+		return IPVSStats{}, errors.New("ip_vs_stats corrupt: too short")
+	}
+
+	statFields = strings.Fields(statLines[2])
+	if len(statFields) != 5 {
+		return IPVSStats{}, errors.New("ip_vs_stats corrupt: unexpected number of fields")
+	}
+
+	stats.Connections, err = strconv.ParseUint(statFields[0], 16, 64)
+	if err != nil {
+		return IPVSStats{}, err
+	}
+	stats.IncomingPackets, err = strconv.ParseUint(statFields[1], 16, 64)
+	if err != nil {
+		return IPVSStats{}, err
+	}
+	stats.OutgoingPackets, err = strconv.ParseUint(statFields[2], 16, 64)
+	if err != nil {
+		return IPVSStats{}, err
+	}
+	stats.IncomingBytes, err = strconv.ParseUint(statFields[3], 16, 64)
+	if err != nil {
+		return IPVSStats{}, err
+	}
+	stats.OutgoingBytes, err = strconv.ParseUint(statFields[4], 16, 64)
+	if err != nil {
+		return IPVSStats{}, err
+	}
+
+	return stats, nil
+}
+
+// IPVSBackendStatus reads and returns the status of all (virtual,real) server pairs from the specified `proc` filesystem.
+func (fs FS) IPVSBackendStatus() ([]IPVSBackendStatus, error) {
+	file, err := os.Open(fs.proc.Path("net/ip_vs"))
+	if err != nil {
+		return nil, err
+	}
+	defer file.Close()
+
+	return parseIPVSBackendStatus(file)
+}
+
+func parseIPVSBackendStatus(file io.Reader) ([]IPVSBackendStatus, error) {
+	var (
+		status       []IPVSBackendStatus
+		scanner      = bufio.NewScanner(file)
+		proto        string
+		localMark    string
+		localAddress net.IP
+		localPort    uint16
+		err          error
+	)
+
+	for scanner.Scan() {
+		fields := strings.Fields(scanner.Text())
+		if len(fields) == 0 {
+			continue
+		}
+		switch {
+		case fields[0] == "IP" || fields[0] == "Prot" || fields[1] == "RemoteAddress:Port":
+			continue
+		case fields[0] == "TCP" || fields[0] == "UDP":
+			if len(fields) < 2 {
+				continue
+			}
+			proto = fields[0]
+			localMark = ""
+			localAddress, localPort, err = parseIPPort(fields[1])
+			if err != nil {
+				return nil, err
+			}
+		case fields[0] == "FWM":
+			if len(fields) < 2 {
+				continue
+			}
+			proto = fields[0]
+			localMark = fields[1]
+			localAddress = nil
+			localPort = 0
+		case fields[0] == "->":
+			if len(fields) < 6 {
+				continue
+			}
+			remoteAddress, remotePort, err := parseIPPort(fields[1])
+			if err != nil {
+				return nil, err
+			}
+			weight, err := strconv.ParseUint(fields[3], 10, 64)
+			if err != nil {
+				return nil, err
+			}
+			activeConn, err := strconv.ParseUint(fields[4], 10, 64)
+			if err != nil {
+				return nil, err
+			}
+			inactConn, err := strconv.ParseUint(fields[5], 10, 64)
+			if err != nil {
+				return nil, err
+			}
+			status = append(status, IPVSBackendStatus{
+				LocalAddress:  localAddress,
+				LocalPort:     localPort,
+				LocalMark:     localMark,
+				RemoteAddress: remoteAddress,
+				RemotePort:    remotePort,
+				Proto:         proto,
+				Weight:        weight,
+				ActiveConn:    activeConn,
+				InactConn:     inactConn,
+			})
+		}
+	}
+	return status, nil
+}
+
+func parseIPPort(s string) (net.IP, uint16, error) {
+	var (
+		ip  net.IP
+		err error
+	)
+
+	switch len(s) {
+	case 13:
+		ip, err = hex.DecodeString(s[0:8])
+		if err != nil {
+			return nil, 0, err
+		}
+	case 46:
+		ip = net.ParseIP(s[1:40])
+		if ip == nil {
+			return nil, 0, fmt.Errorf("invalid IPv6 address: %s", s[1:40])
+		}
+	default:
+		return nil, 0, fmt.Errorf("unexpected IP:Port: %s", s)
+	}
+
+	portString := s[len(s)-4:]
+	if len(portString) != 4 {
+		return nil, 0, fmt.Errorf("unexpected port string format: %s", portString)
+	}
+	port, err := strconv.ParseUint(portString, 16, 16)
+	if err != nil {
+		return nil, 0, err
+	}
+
+	return ip, uint16(port), nil
+}
diff --git a/vendor/github.com/prometheus/procfs/mdstat.go b/vendor/github.com/prometheus/procfs/mdstat.go
new file mode 100644
index 0000000..71c1067
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/mdstat.go
@@ -0,0 +1,151 @@
+// Copyright 2018 The Prometheus 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 procfs
+
+import (
+	"fmt"
+	"io/ioutil"
+	"regexp"
+	"strconv"
+	"strings"
+)
+
+var (
+	statuslineRE = regexp.MustCompile(`(\d+) blocks .*\[(\d+)/(\d+)\] \[[U_]+\]`)
+	buildlineRE  = regexp.MustCompile(`\((\d+)/\d+\)`)
+)
+
+// MDStat holds info parsed from /proc/mdstat.
+type MDStat struct {
+	// Name of the device.
+	Name string
+	// activity-state of the device.
+	ActivityState string
+	// Number of active disks.
+	DisksActive int64
+	// Total number of disks the device consists of.
+	DisksTotal int64
+	// Number of blocks the device holds.
+	BlocksTotal int64
+	// Number of blocks on the device that are in sync.
+	BlocksSynced int64
+}
+
+// MDStat parses an mdstat-file (/proc/mdstat) and returns a slice of
+// structs containing the relevant info.  More information available here:
+// https://raid.wiki.kernel.org/index.php/Mdstat
+func (fs FS) MDStat() ([]MDStat, error) {
+	data, err := ioutil.ReadFile(fs.proc.Path("mdstat"))
+	if err != nil {
+		return nil, fmt.Errorf("error parsing mdstat %s: %s", fs.proc.Path("mdstat"), err)
+	}
+	mdstat, err := parseMDStat(data)
+	if err != nil {
+		return nil, fmt.Errorf("error parsing mdstat %s: %s", fs.proc.Path("mdstat"), err)
+	}
+	return mdstat, nil
+}
+
+// parseMDStat parses data from mdstat file (/proc/mdstat) and returns a slice of
+// structs containing the relevant info.
+func parseMDStat(mdstatData []byte) ([]MDStat, error) {
+	mdStats := []MDStat{}
+	lines := strings.Split(string(mdstatData), "\n")
+	for i, l := range lines {
+		if strings.TrimSpace(l) == "" || l[0] == ' ' ||
+			strings.HasPrefix(l, "Personalities") || strings.HasPrefix(l, "unused") {
+			continue
+		}
+
+		deviceFields := strings.Fields(l)
+		if len(deviceFields) < 3 {
+			return nil, fmt.Errorf("not enough fields in mdline (expected at least 3): %s", l)
+		}
+		mdName := deviceFields[0]
+		activityState := deviceFields[2]
+
+		if len(lines) <= i+3 {
+			return mdStats, fmt.Errorf("missing lines for md device %s", mdName)
+		}
+
+		active, total, size, err := evalStatusLine(lines[i+1])
+		if err != nil {
+			return nil, err
+		}
+
+		syncLineIdx := i + 2
+		if strings.Contains(lines[i+2], "bitmap") { // skip bitmap line
+			syncLineIdx++
+		}
+
+		// If device is recovering/syncing at the moment, get the number of currently
+		// synced bytes, otherwise that number equals the size of the device.
+		syncedBlocks := size
+		if strings.Contains(lines[syncLineIdx], "recovery") || strings.Contains(lines[syncLineIdx], "resync") {
+			syncedBlocks, err = evalRecoveryLine(lines[syncLineIdx])
+			if err != nil {
+				return nil, err
+			}
+		}
+
+		mdStats = append(mdStats, MDStat{
+			Name:          mdName,
+			ActivityState: activityState,
+			DisksActive:   active,
+			DisksTotal:    total,
+			BlocksTotal:   size,
+			BlocksSynced:  syncedBlocks,
+		})
+	}
+
+	return mdStats, nil
+}
+
+func evalStatusLine(statusline string) (active, total, size int64, err error) {
+	matches := statuslineRE.FindStringSubmatch(statusline)
+	if len(matches) != 4 {
+		return 0, 0, 0, fmt.Errorf("unexpected statusline: %s", statusline)
+	}
+
+	size, err = strconv.ParseInt(matches[1], 10, 64)
+	if err != nil {
+		return 0, 0, 0, fmt.Errorf("unexpected statusline %s: %s", statusline, err)
+	}
+
+	total, err = strconv.ParseInt(matches[2], 10, 64)
+	if err != nil {
+		return 0, 0, 0, fmt.Errorf("unexpected statusline %s: %s", statusline, err)
+	}
+
+	active, err = strconv.ParseInt(matches[3], 10, 64)
+	if err != nil {
+		return 0, 0, 0, fmt.Errorf("unexpected statusline %s: %s", statusline, err)
+	}
+
+	return active, total, size, nil
+}
+
+func evalRecoveryLine(buildline string) (syncedBlocks int64, err error) {
+	matches := buildlineRE.FindStringSubmatch(buildline)
+	if len(matches) != 2 {
+		return 0, fmt.Errorf("unexpected buildline: %s", buildline)
+	}
+
+	syncedBlocks, err = strconv.ParseInt(matches[1], 10, 64)
+	if err != nil {
+		return 0, fmt.Errorf("%s in buildline: %s", err, buildline)
+	}
+
+	return syncedBlocks, nil
+}
diff --git a/vendor/github.com/prometheus/procfs/mountstats.go b/vendor/github.com/prometheus/procfs/mountstats.go
new file mode 100644
index 0000000..35b2ef3
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/mountstats.go
@@ -0,0 +1,621 @@
+// Copyright 2018 The Prometheus 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 procfs
+
+// While implementing parsing of /proc/[pid]/mountstats, this blog was used
+// heavily as a reference:
+//   https://utcc.utoronto.ca/~cks/space/blog/linux/NFSMountstatsIndex
+//
+// Special thanks to Chris Siebenmann for all of his posts explaining the
+// various statistics available for NFS.
+
+import (
+	"bufio"
+	"fmt"
+	"io"
+	"strconv"
+	"strings"
+	"time"
+)
+
+// Constants shared between multiple functions.
+const (
+	deviceEntryLen = 8
+
+	fieldBytesLen  = 8
+	fieldEventsLen = 27
+
+	statVersion10 = "1.0"
+	statVersion11 = "1.1"
+
+	fieldTransport10TCPLen = 10
+	fieldTransport10UDPLen = 7
+
+	fieldTransport11TCPLen = 13
+	fieldTransport11UDPLen = 10
+)
+
+// A Mount is a device mount parsed from /proc/[pid]/mountstats.
+type Mount struct {
+	// Name of the device.
+	Device string
+	// The mount point of the device.
+	Mount string
+	// The filesystem type used by the device.
+	Type string
+	// If available additional statistics related to this Mount.
+	// Use a type assertion to determine if additional statistics are available.
+	Stats MountStats
+}
+
+// A MountStats is a type which contains detailed statistics for a specific
+// type of Mount.
+type MountStats interface {
+	mountStats()
+}
+
+// A MountStatsNFS is a MountStats implementation for NFSv3 and v4 mounts.
+type MountStatsNFS struct {
+	// The version of statistics provided.
+	StatVersion string
+	// The mount options of the NFS mount.
+	Opts map[string]string
+	// The age of the NFS mount.
+	Age time.Duration
+	// Statistics related to byte counters for various operations.
+	Bytes NFSBytesStats
+	// Statistics related to various NFS event occurrences.
+	Events NFSEventsStats
+	// Statistics broken down by filesystem operation.
+	Operations []NFSOperationStats
+	// Statistics about the NFS RPC transport.
+	Transport NFSTransportStats
+}
+
+// mountStats implements MountStats.
+func (m MountStatsNFS) mountStats() {}
+
+// A NFSBytesStats contains statistics about the number of bytes read and written
+// by an NFS client to and from an NFS server.
+type NFSBytesStats struct {
+	// Number of bytes read using the read() syscall.
+	Read uint64
+	// Number of bytes written using the write() syscall.
+	Write uint64
+	// Number of bytes read using the read() syscall in O_DIRECT mode.
+	DirectRead uint64
+	// Number of bytes written using the write() syscall in O_DIRECT mode.
+	DirectWrite uint64
+	// Number of bytes read from the NFS server, in total.
+	ReadTotal uint64
+	// Number of bytes written to the NFS server, in total.
+	WriteTotal uint64
+	// Number of pages read directly via mmap()'d files.
+	ReadPages uint64
+	// Number of pages written directly via mmap()'d files.
+	WritePages uint64
+}
+
+// A NFSEventsStats contains statistics about NFS event occurrences.
+type NFSEventsStats struct {
+	// Number of times cached inode attributes are re-validated from the server.
+	InodeRevalidate uint64
+	// Number of times cached dentry nodes are re-validated from the server.
+	DnodeRevalidate uint64
+	// Number of times an inode cache is cleared.
+	DataInvalidate uint64
+	// Number of times cached inode attributes are invalidated.
+	AttributeInvalidate uint64
+	// Number of times files or directories have been open()'d.
+	VFSOpen uint64
+	// Number of times a directory lookup has occurred.
+	VFSLookup uint64
+	// Number of times permissions have been checked.
+	VFSAccess uint64
+	// Number of updates (and potential writes) to pages.
+	VFSUpdatePage uint64
+	// Number of pages read directly via mmap()'d files.
+	VFSReadPage uint64
+	// Number of times a group of pages have been read.
+	VFSReadPages uint64
+	// Number of pages written directly via mmap()'d files.
+	VFSWritePage uint64
+	// Number of times a group of pages have been written.
+	VFSWritePages uint64
+	// Number of times directory entries have been read with getdents().
+	VFSGetdents uint64
+	// Number of times attributes have been set on inodes.
+	VFSSetattr uint64
+	// Number of pending writes that have been forcefully flushed to the server.
+	VFSFlush uint64
+	// Number of times fsync() has been called on directories and files.
+	VFSFsync uint64
+	// Number of times locking has been attempted on a file.
+	VFSLock uint64
+	// Number of times files have been closed and released.
+	VFSFileRelease uint64
+	// Unknown.  Possibly unused.
+	CongestionWait uint64
+	// Number of times files have been truncated.
+	Truncation uint64
+	// Number of times a file has been grown due to writes beyond its existing end.
+	WriteExtension uint64
+	// Number of times a file was removed while still open by another process.
+	SillyRename uint64
+	// Number of times the NFS server gave less data than expected while reading.
+	ShortRead uint64
+	// Number of times the NFS server wrote less data than expected while writing.
+	ShortWrite uint64
+	// Number of times the NFS server indicated EJUKEBOX; retrieving data from
+	// offline storage.
+	JukeboxDelay uint64
+	// Number of NFS v4.1+ pNFS reads.
+	PNFSRead uint64
+	// Number of NFS v4.1+ pNFS writes.
+	PNFSWrite uint64
+}
+
+// A NFSOperationStats contains statistics for a single operation.
+type NFSOperationStats struct {
+	// The name of the operation.
+	Operation string
+	// Number of requests performed for this operation.
+	Requests uint64
+	// Number of times an actual RPC request has been transmitted for this operation.
+	Transmissions uint64
+	// Number of times a request has had a major timeout.
+	MajorTimeouts uint64
+	// Number of bytes sent for this operation, including RPC headers and payload.
+	BytesSent uint64
+	// Number of bytes received for this operation, including RPC headers and payload.
+	BytesReceived uint64
+	// Duration all requests spent queued for transmission before they were sent.
+	CumulativeQueueMilliseconds uint64
+	// Duration it took to get a reply back after the request was transmitted.
+	CumulativeTotalResponseMilliseconds uint64
+	// Duration from when a request was enqueued to when it was completely handled.
+	CumulativeTotalRequestMilliseconds uint64
+}
+
+// A NFSTransportStats contains statistics for the NFS mount RPC requests and
+// responses.
+type NFSTransportStats struct {
+	// The transport protocol used for the NFS mount.
+	Protocol string
+	// The local port used for the NFS mount.
+	Port uint64
+	// Number of times the client has had to establish a connection from scratch
+	// to the NFS server.
+	Bind uint64
+	// Number of times the client has made a TCP connection to the NFS server.
+	Connect uint64
+	// Duration (in jiffies, a kernel internal unit of time) the NFS mount has
+	// spent waiting for connections to the server to be established.
+	ConnectIdleTime uint64
+	// Duration since the NFS mount last saw any RPC traffic.
+	IdleTimeSeconds uint64
+	// Number of RPC requests for this mount sent to the NFS server.
+	Sends uint64
+	// Number of RPC responses for this mount received from the NFS server.
+	Receives uint64
+	// Number of times the NFS server sent a response with a transaction ID
+	// unknown to this client.
+	BadTransactionIDs uint64
+	// A running counter, incremented on each request as the current difference
+	// ebetween sends and receives.
+	CumulativeActiveRequests uint64
+	// A running counter, incremented on each request by the current backlog
+	// queue size.
+	CumulativeBacklog uint64
+
+	// Stats below only available with stat version 1.1.
+
+	// Maximum number of simultaneously active RPC requests ever used.
+	MaximumRPCSlotsUsed uint64
+	// A running counter, incremented on each request as the current size of the
+	// sending queue.
+	CumulativeSendingQueue uint64
+	// A running counter, incremented on each request as the current size of the
+	// pending queue.
+	CumulativePendingQueue uint64
+}
+
+// parseMountStats parses a /proc/[pid]/mountstats file and returns a slice
+// of Mount structures containing detailed information about each mount.
+// If available, statistics for each mount are parsed as well.
+func parseMountStats(r io.Reader) ([]*Mount, error) {
+	const (
+		device            = "device"
+		statVersionPrefix = "statvers="
+
+		nfs3Type = "nfs"
+		nfs4Type = "nfs4"
+	)
+
+	var mounts []*Mount
+
+	s := bufio.NewScanner(r)
+	for s.Scan() {
+		// Only look for device entries in this function
+		ss := strings.Fields(string(s.Bytes()))
+		if len(ss) == 0 || ss[0] != device {
+			continue
+		}
+
+		m, err := parseMount(ss)
+		if err != nil {
+			return nil, err
+		}
+
+		// Does this mount also possess statistics information?
+		if len(ss) > deviceEntryLen {
+			// Only NFSv3 and v4 are supported for parsing statistics
+			if m.Type != nfs3Type && m.Type != nfs4Type {
+				return nil, fmt.Errorf("cannot parse MountStats for fstype %q", m.Type)
+			}
+
+			statVersion := strings.TrimPrefix(ss[8], statVersionPrefix)
+
+			stats, err := parseMountStatsNFS(s, statVersion)
+			if err != nil {
+				return nil, err
+			}
+
+			m.Stats = stats
+		}
+
+		mounts = append(mounts, m)
+	}
+
+	return mounts, s.Err()
+}
+
+// parseMount parses an entry in /proc/[pid]/mountstats in the format:
+//   device [device] mounted on [mount] with fstype [type]
+func parseMount(ss []string) (*Mount, error) {
+	if len(ss) < deviceEntryLen {
+		return nil, fmt.Errorf("invalid device entry: %v", ss)
+	}
+
+	// Check for specific words appearing at specific indices to ensure
+	// the format is consistent with what we expect
+	format := []struct {
+		i int
+		s string
+	}{
+		{i: 0, s: "device"},
+		{i: 2, s: "mounted"},
+		{i: 3, s: "on"},
+		{i: 5, s: "with"},
+		{i: 6, s: "fstype"},
+	}
+
+	for _, f := range format {
+		if ss[f.i] != f.s {
+			return nil, fmt.Errorf("invalid device entry: %v", ss)
+		}
+	}
+
+	return &Mount{
+		Device: ss[1],
+		Mount:  ss[4],
+		Type:   ss[7],
+	}, nil
+}
+
+// parseMountStatsNFS parses a MountStatsNFS by scanning additional information
+// related to NFS statistics.
+func parseMountStatsNFS(s *bufio.Scanner, statVersion string) (*MountStatsNFS, error) {
+	// Field indicators for parsing specific types of data
+	const (
+		fieldOpts       = "opts:"
+		fieldAge        = "age:"
+		fieldBytes      = "bytes:"
+		fieldEvents     = "events:"
+		fieldPerOpStats = "per-op"
+		fieldTransport  = "xprt:"
+	)
+
+	stats := &MountStatsNFS{
+		StatVersion: statVersion,
+	}
+
+	for s.Scan() {
+		ss := strings.Fields(string(s.Bytes()))
+		if len(ss) == 0 {
+			break
+		}
+		if len(ss) < 2 {
+			return nil, fmt.Errorf("not enough information for NFS stats: %v", ss)
+		}
+
+		switch ss[0] {
+		case fieldOpts:
+			if stats.Opts == nil {
+				stats.Opts = map[string]string{}
+			}
+			for _, opt := range strings.Split(ss[1], ",") {
+				split := strings.Split(opt, "=")
+				if len(split) == 2 {
+					stats.Opts[split[0]] = split[1]
+				} else {
+					stats.Opts[opt] = ""
+				}
+			}
+		case fieldAge:
+			// Age integer is in seconds
+			d, err := time.ParseDuration(ss[1] + "s")
+			if err != nil {
+				return nil, err
+			}
+
+			stats.Age = d
+		case fieldBytes:
+			bstats, err := parseNFSBytesStats(ss[1:])
+			if err != nil {
+				return nil, err
+			}
+
+			stats.Bytes = *bstats
+		case fieldEvents:
+			estats, err := parseNFSEventsStats(ss[1:])
+			if err != nil {
+				return nil, err
+			}
+
+			stats.Events = *estats
+		case fieldTransport:
+			if len(ss) < 3 {
+				return nil, fmt.Errorf("not enough information for NFS transport stats: %v", ss)
+			}
+
+			tstats, err := parseNFSTransportStats(ss[1:], statVersion)
+			if err != nil {
+				return nil, err
+			}
+
+			stats.Transport = *tstats
+		}
+
+		// When encountering "per-operation statistics", we must break this
+		// loop and parse them separately to ensure we can terminate parsing
+		// before reaching another device entry; hence why this 'if' statement
+		// is not just another switch case
+		if ss[0] == fieldPerOpStats {
+			break
+		}
+	}
+
+	if err := s.Err(); err != nil {
+		return nil, err
+	}
+
+	// NFS per-operation stats appear last before the next device entry
+	perOpStats, err := parseNFSOperationStats(s)
+	if err != nil {
+		return nil, err
+	}
+
+	stats.Operations = perOpStats
+
+	return stats, nil
+}
+
+// parseNFSBytesStats parses a NFSBytesStats line using an input set of
+// integer fields.
+func parseNFSBytesStats(ss []string) (*NFSBytesStats, error) {
+	if len(ss) != fieldBytesLen {
+		return nil, fmt.Errorf("invalid NFS bytes stats: %v", ss)
+	}
+
+	ns := make([]uint64, 0, fieldBytesLen)
+	for _, s := range ss {
+		n, err := strconv.ParseUint(s, 10, 64)
+		if err != nil {
+			return nil, err
+		}
+
+		ns = append(ns, n)
+	}
+
+	return &NFSBytesStats{
+		Read:        ns[0],
+		Write:       ns[1],
+		DirectRead:  ns[2],
+		DirectWrite: ns[3],
+		ReadTotal:   ns[4],
+		WriteTotal:  ns[5],
+		ReadPages:   ns[6],
+		WritePages:  ns[7],
+	}, nil
+}
+
+// parseNFSEventsStats parses a NFSEventsStats line using an input set of
+// integer fields.
+func parseNFSEventsStats(ss []string) (*NFSEventsStats, error) {
+	if len(ss) != fieldEventsLen {
+		return nil, fmt.Errorf("invalid NFS events stats: %v", ss)
+	}
+
+	ns := make([]uint64, 0, fieldEventsLen)
+	for _, s := range ss {
+		n, err := strconv.ParseUint(s, 10, 64)
+		if err != nil {
+			return nil, err
+		}
+
+		ns = append(ns, n)
+	}
+
+	return &NFSEventsStats{
+		InodeRevalidate:     ns[0],
+		DnodeRevalidate:     ns[1],
+		DataInvalidate:      ns[2],
+		AttributeInvalidate: ns[3],
+		VFSOpen:             ns[4],
+		VFSLookup:           ns[5],
+		VFSAccess:           ns[6],
+		VFSUpdatePage:       ns[7],
+		VFSReadPage:         ns[8],
+		VFSReadPages:        ns[9],
+		VFSWritePage:        ns[10],
+		VFSWritePages:       ns[11],
+		VFSGetdents:         ns[12],
+		VFSSetattr:          ns[13],
+		VFSFlush:            ns[14],
+		VFSFsync:            ns[15],
+		VFSLock:             ns[16],
+		VFSFileRelease:      ns[17],
+		CongestionWait:      ns[18],
+		Truncation:          ns[19],
+		WriteExtension:      ns[20],
+		SillyRename:         ns[21],
+		ShortRead:           ns[22],
+		ShortWrite:          ns[23],
+		JukeboxDelay:        ns[24],
+		PNFSRead:            ns[25],
+		PNFSWrite:           ns[26],
+	}, nil
+}
+
+// parseNFSOperationStats parses a slice of NFSOperationStats by scanning
+// additional information about per-operation statistics until an empty
+// line is reached.
+func parseNFSOperationStats(s *bufio.Scanner) ([]NFSOperationStats, error) {
+	const (
+		// Number of expected fields in each per-operation statistics set
+		numFields = 9
+	)
+
+	var ops []NFSOperationStats
+
+	for s.Scan() {
+		ss := strings.Fields(string(s.Bytes()))
+		if len(ss) == 0 {
+			// Must break when reading a blank line after per-operation stats to
+			// enable top-level function to parse the next device entry
+			break
+		}
+
+		if len(ss) != numFields {
+			return nil, fmt.Errorf("invalid NFS per-operations stats: %v", ss)
+		}
+
+		// Skip string operation name for integers
+		ns := make([]uint64, 0, numFields-1)
+		for _, st := range ss[1:] {
+			n, err := strconv.ParseUint(st, 10, 64)
+			if err != nil {
+				return nil, err
+			}
+
+			ns = append(ns, n)
+		}
+
+		ops = append(ops, NFSOperationStats{
+			Operation:                           strings.TrimSuffix(ss[0], ":"),
+			Requests:                            ns[0],
+			Transmissions:                       ns[1],
+			MajorTimeouts:                       ns[2],
+			BytesSent:                           ns[3],
+			BytesReceived:                       ns[4],
+			CumulativeQueueMilliseconds:         ns[5],
+			CumulativeTotalResponseMilliseconds: ns[6],
+			CumulativeTotalRequestMilliseconds:  ns[7],
+		})
+	}
+
+	return ops, s.Err()
+}
+
+// parseNFSTransportStats parses a NFSTransportStats line using an input set of
+// integer fields matched to a specific stats version.
+func parseNFSTransportStats(ss []string, statVersion string) (*NFSTransportStats, error) {
+	// Extract the protocol field. It is the only string value in the line
+	protocol := ss[0]
+	ss = ss[1:]
+
+	switch statVersion {
+	case statVersion10:
+		var expectedLength int
+		if protocol == "tcp" {
+			expectedLength = fieldTransport10TCPLen
+		} else if protocol == "udp" {
+			expectedLength = fieldTransport10UDPLen
+		} else {
+			return nil, fmt.Errorf("invalid NFS protocol \"%s\" in stats 1.0 statement: %v", protocol, ss)
+		}
+		if len(ss) != expectedLength {
+			return nil, fmt.Errorf("invalid NFS transport stats 1.0 statement: %v", ss)
+		}
+	case statVersion11:
+		var expectedLength int
+		if protocol == "tcp" {
+			expectedLength = fieldTransport11TCPLen
+		} else if protocol == "udp" {
+			expectedLength = fieldTransport11UDPLen
+		} else {
+			return nil, fmt.Errorf("invalid NFS protocol \"%s\" in stats 1.1 statement: %v", protocol, ss)
+		}
+		if len(ss) != expectedLength {
+			return nil, fmt.Errorf("invalid NFS transport stats 1.1 statement: %v", ss)
+		}
+	default:
+		return nil, fmt.Errorf("unrecognized NFS transport stats version: %q", statVersion)
+	}
+
+	// Allocate enough for v1.1 stats since zero value for v1.1 stats will be okay
+	// in a v1.0 response. Since the stat length is bigger for TCP stats, we use
+	// the TCP length here.
+	//
+	// Note: slice length must be set to length of v1.1 stats to avoid a panic when
+	// only v1.0 stats are present.
+	// See: https://github.com/prometheus/node_exporter/issues/571.
+	ns := make([]uint64, fieldTransport11TCPLen)
+	for i, s := range ss {
+		n, err := strconv.ParseUint(s, 10, 64)
+		if err != nil {
+			return nil, err
+		}
+
+		ns[i] = n
+	}
+
+	// The fields differ depending on the transport protocol (TCP or UDP)
+	// From https://utcc.utoronto.ca/%7Ecks/space/blog/linux/NFSMountstatsXprt
+	//
+	// For the udp RPC transport there is no connection count, connect idle time,
+	// or idle time (fields #3, #4, and #5); all other fields are the same. So
+	// we set them to 0 here.
+	if protocol == "udp" {
+		ns = append(ns[:2], append(make([]uint64, 3), ns[2:]...)...)
+	}
+
+	return &NFSTransportStats{
+		Protocol:                 protocol,
+		Port:                     ns[0],
+		Bind:                     ns[1],
+		Connect:                  ns[2],
+		ConnectIdleTime:          ns[3],
+		IdleTimeSeconds:          ns[4],
+		Sends:                    ns[5],
+		Receives:                 ns[6],
+		BadTransactionIDs:        ns[7],
+		CumulativeActiveRequests: ns[8],
+		CumulativeBacklog:        ns[9],
+		MaximumRPCSlotsUsed:      ns[10],
+		CumulativeSendingQueue:   ns[11],
+		CumulativePendingQueue:   ns[12],
+	}, nil
+}
diff --git a/vendor/github.com/prometheus/procfs/net_dev.go b/vendor/github.com/prometheus/procfs/net_dev.go
new file mode 100644
index 0000000..a0b7a01
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/net_dev.go
@@ -0,0 +1,206 @@
+// Copyright 2018 The Prometheus 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 procfs
+
+import (
+	"bufio"
+	"errors"
+	"os"
+	"sort"
+	"strconv"
+	"strings"
+)
+
+// NetDevLine is single line parsed from /proc/net/dev or /proc/[pid]/net/dev.
+type NetDevLine struct {
+	Name         string `json:"name"`          // The name of the interface.
+	RxBytes      uint64 `json:"rx_bytes"`      // Cumulative count of bytes received.
+	RxPackets    uint64 `json:"rx_packets"`    // Cumulative count of packets received.
+	RxErrors     uint64 `json:"rx_errors"`     // Cumulative count of receive errors encountered.
+	RxDropped    uint64 `json:"rx_dropped"`    // Cumulative count of packets dropped while receiving.
+	RxFIFO       uint64 `json:"rx_fifo"`       // Cumulative count of FIFO buffer errors.
+	RxFrame      uint64 `json:"rx_frame"`      // Cumulative count of packet framing errors.
+	RxCompressed uint64 `json:"rx_compressed"` // Cumulative count of compressed packets received by the device driver.
+	RxMulticast  uint64 `json:"rx_multicast"`  // Cumulative count of multicast frames received by the device driver.
+	TxBytes      uint64 `json:"tx_bytes"`      // Cumulative count of bytes transmitted.
+	TxPackets    uint64 `json:"tx_packets"`    // Cumulative count of packets transmitted.
+	TxErrors     uint64 `json:"tx_errors"`     // Cumulative count of transmit errors encountered.
+	TxDropped    uint64 `json:"tx_dropped"`    // Cumulative count of packets dropped while transmitting.
+	TxFIFO       uint64 `json:"tx_fifo"`       // Cumulative count of FIFO buffer errors.
+	TxCollisions uint64 `json:"tx_collisions"` // Cumulative count of collisions detected on the interface.
+	TxCarrier    uint64 `json:"tx_carrier"`    // Cumulative count of carrier losses detected by the device driver.
+	TxCompressed uint64 `json:"tx_compressed"` // Cumulative count of compressed packets transmitted by the device driver.
+}
+
+// NetDev is parsed from /proc/net/dev or /proc/[pid]/net/dev. The map keys
+// are interface names.
+type NetDev map[string]NetDevLine
+
+// NetDev returns kernel/system statistics read from /proc/net/dev.
+func (fs FS) NetDev() (NetDev, error) {
+	return newNetDev(fs.proc.Path("net/dev"))
+}
+
+// NetDev returns kernel/system statistics read from /proc/[pid]/net/dev.
+func (p Proc) NetDev() (NetDev, error) {
+	return newNetDev(p.path("net/dev"))
+}
+
+// newNetDev creates a new NetDev from the contents of the given file.
+func newNetDev(file string) (NetDev, error) {
+	f, err := os.Open(file)
+	if err != nil {
+		return NetDev{}, err
+	}
+	defer f.Close()
+
+	netDev := NetDev{}
+	s := bufio.NewScanner(f)
+	for n := 0; s.Scan(); n++ {
+		// Skip the 2 header lines.
+		if n < 2 {
+			continue
+		}
+
+		line, err := netDev.parseLine(s.Text())
+		if err != nil {
+			return netDev, err
+		}
+
+		netDev[line.Name] = *line
+	}
+
+	return netDev, s.Err()
+}
+
+// parseLine parses a single line from the /proc/net/dev file. Header lines
+// must be filtered prior to calling this method.
+func (netDev NetDev) parseLine(rawLine string) (*NetDevLine, error) {
+	parts := strings.SplitN(rawLine, ":", 2)
+	if len(parts) != 2 {
+		return nil, errors.New("invalid net/dev line, missing colon")
+	}
+	fields := strings.Fields(strings.TrimSpace(parts[1]))
+
+	var err error
+	line := &NetDevLine{}
+
+	// Interface Name
+	line.Name = strings.TrimSpace(parts[0])
+	if line.Name == "" {
+		return nil, errors.New("invalid net/dev line, empty interface name")
+	}
+
+	// RX
+	line.RxBytes, err = strconv.ParseUint(fields[0], 10, 64)
+	if err != nil {
+		return nil, err
+	}
+	line.RxPackets, err = strconv.ParseUint(fields[1], 10, 64)
+	if err != nil {
+		return nil, err
+	}
+	line.RxErrors, err = strconv.ParseUint(fields[2], 10, 64)
+	if err != nil {
+		return nil, err
+	}
+	line.RxDropped, err = strconv.ParseUint(fields[3], 10, 64)
+	if err != nil {
+		return nil, err
+	}
+	line.RxFIFO, err = strconv.ParseUint(fields[4], 10, 64)
+	if err != nil {
+		return nil, err
+	}
+	line.RxFrame, err = strconv.ParseUint(fields[5], 10, 64)
+	if err != nil {
+		return nil, err
+	}
+	line.RxCompressed, err = strconv.ParseUint(fields[6], 10, 64)
+	if err != nil {
+		return nil, err
+	}
+	line.RxMulticast, err = strconv.ParseUint(fields[7], 10, 64)
+	if err != nil {
+		return nil, err
+	}
+
+	// TX
+	line.TxBytes, err = strconv.ParseUint(fields[8], 10, 64)
+	if err != nil {
+		return nil, err
+	}
+	line.TxPackets, err = strconv.ParseUint(fields[9], 10, 64)
+	if err != nil {
+		return nil, err
+	}
+	line.TxErrors, err = strconv.ParseUint(fields[10], 10, 64)
+	if err != nil {
+		return nil, err
+	}
+	line.TxDropped, err = strconv.ParseUint(fields[11], 10, 64)
+	if err != nil {
+		return nil, err
+	}
+	line.TxFIFO, err = strconv.ParseUint(fields[12], 10, 64)
+	if err != nil {
+		return nil, err
+	}
+	line.TxCollisions, err = strconv.ParseUint(fields[13], 10, 64)
+	if err != nil {
+		return nil, err
+	}
+	line.TxCarrier, err = strconv.ParseUint(fields[14], 10, 64)
+	if err != nil {
+		return nil, err
+	}
+	line.TxCompressed, err = strconv.ParseUint(fields[15], 10, 64)
+	if err != nil {
+		return nil, err
+	}
+
+	return line, nil
+}
+
+// Total aggregates the values across interfaces and returns a new NetDevLine.
+// The Name field will be a sorted comma separated list of interface names.
+func (netDev NetDev) Total() NetDevLine {
+	total := NetDevLine{}
+
+	names := make([]string, 0, len(netDev))
+	for _, ifc := range netDev {
+		names = append(names, ifc.Name)
+		total.RxBytes += ifc.RxBytes
+		total.RxPackets += ifc.RxPackets
+		total.RxPackets += ifc.RxPackets
+		total.RxErrors += ifc.RxErrors
+		total.RxDropped += ifc.RxDropped
+		total.RxFIFO += ifc.RxFIFO
+		total.RxFrame += ifc.RxFrame
+		total.RxCompressed += ifc.RxCompressed
+		total.RxMulticast += ifc.RxMulticast
+		total.TxBytes += ifc.TxBytes
+		total.TxPackets += ifc.TxPackets
+		total.TxErrors += ifc.TxErrors
+		total.TxDropped += ifc.TxDropped
+		total.TxFIFO += ifc.TxFIFO
+		total.TxCollisions += ifc.TxCollisions
+		total.TxCarrier += ifc.TxCarrier
+		total.TxCompressed += ifc.TxCompressed
+	}
+	sort.Strings(names)
+	total.Name = strings.Join(names, ", ")
+
+	return total
+}
diff --git a/vendor/github.com/prometheus/procfs/net_unix.go b/vendor/github.com/prometheus/procfs/net_unix.go
new file mode 100644
index 0000000..240340a
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/net_unix.go
@@ -0,0 +1,275 @@
+// Copyright 2018 The Prometheus 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 procfs
+
+import (
+	"bufio"
+	"errors"
+	"fmt"
+	"io"
+	"os"
+	"strconv"
+	"strings"
+)
+
+// For the proc file format details,
+// see https://elixir.bootlin.com/linux/v4.17/source/net/unix/af_unix.c#L2815
+// and https://elixir.bootlin.com/linux/latest/source/include/uapi/linux/net.h#L48.
+
+const (
+	netUnixKernelPtrIdx = iota
+	netUnixRefCountIdx
+	_
+	netUnixFlagsIdx
+	netUnixTypeIdx
+	netUnixStateIdx
+	netUnixInodeIdx
+
+	// Inode and Path are optional.
+	netUnixStaticFieldsCnt = 6
+)
+
+const (
+	netUnixTypeStream    = 1
+	netUnixTypeDgram     = 2
+	netUnixTypeSeqpacket = 5
+
+	netUnixFlagListen = 1 << 16
+
+	netUnixStateUnconnected  = 1
+	netUnixStateConnecting   = 2
+	netUnixStateConnected    = 3
+	netUnixStateDisconnected = 4
+)
+
+var errInvalidKernelPtrFmt = errors.New("Invalid Num(the kernel table slot number) format")
+
+// NetUnixType is the type of the type field.
+type NetUnixType uint64
+
+// NetUnixFlags is the type of the flags field.
+type NetUnixFlags uint64
+
+// NetUnixState is the type of the state field.
+type NetUnixState uint64
+
+// NetUnixLine represents a line of /proc/net/unix.
+type NetUnixLine struct {
+	KernelPtr string
+	RefCount  uint64
+	Protocol  uint64
+	Flags     NetUnixFlags
+	Type      NetUnixType
+	State     NetUnixState
+	Inode     uint64
+	Path      string
+}
+
+// NetUnix holds the data read from /proc/net/unix.
+type NetUnix struct {
+	Rows []*NetUnixLine
+}
+
+// NewNetUnix returns data read from /proc/net/unix.
+func NewNetUnix() (*NetUnix, error) {
+	fs, err := NewFS(DefaultMountPoint)
+	if err != nil {
+		return nil, err
+	}
+
+	return fs.NewNetUnix()
+}
+
+// NewNetUnix returns data read from /proc/net/unix.
+func (fs FS) NewNetUnix() (*NetUnix, error) {
+	return NewNetUnixByPath(fs.proc.Path("net/unix"))
+}
+
+// NewNetUnixByPath returns data read from /proc/net/unix by file path.
+// It might returns an error with partial parsed data, if an error occur after some data parsed.
+func NewNetUnixByPath(path string) (*NetUnix, error) {
+	f, err := os.Open(path)
+	if err != nil {
+		return nil, err
+	}
+	defer f.Close()
+	return NewNetUnixByReader(f)
+}
+
+// NewNetUnixByReader returns data read from /proc/net/unix by a reader.
+// It might returns an error with partial parsed data, if an error occur after some data parsed.
+func NewNetUnixByReader(reader io.Reader) (*NetUnix, error) {
+	nu := &NetUnix{
+		Rows: make([]*NetUnixLine, 0, 32),
+	}
+	scanner := bufio.NewScanner(reader)
+	// Omit the header line.
+	scanner.Scan()
+	header := scanner.Text()
+	// From the man page of proc(5), it does not contain an Inode field,
+	// but in actually it exists.
+	// This code works for both cases.
+	hasInode := strings.Contains(header, "Inode")
+
+	minFieldsCnt := netUnixStaticFieldsCnt
+	if hasInode {
+		minFieldsCnt++
+	}
+	for scanner.Scan() {
+		line := scanner.Text()
+		item, err := nu.parseLine(line, hasInode, minFieldsCnt)
+		if err != nil {
+			return nu, err
+		}
+		nu.Rows = append(nu.Rows, item)
+	}
+
+	return nu, scanner.Err()
+}
+
+func (u *NetUnix) parseLine(line string, hasInode bool, minFieldsCnt int) (*NetUnixLine, error) {
+	fields := strings.Fields(line)
+	fieldsLen := len(fields)
+	if fieldsLen < minFieldsCnt {
+		return nil, fmt.Errorf(
+			"Parse Unix domain failed: expect at least %d fields but got %d",
+			minFieldsCnt, fieldsLen)
+	}
+	kernelPtr, err := u.parseKernelPtr(fields[netUnixKernelPtrIdx])
+	if err != nil {
+		return nil, fmt.Errorf("Parse Unix domain num(%s) failed: %s", fields[netUnixKernelPtrIdx], err)
+	}
+	users, err := u.parseUsers(fields[netUnixRefCountIdx])
+	if err != nil {
+		return nil, fmt.Errorf("Parse Unix domain ref count(%s) failed: %s", fields[netUnixRefCountIdx], err)
+	}
+	flags, err := u.parseFlags(fields[netUnixFlagsIdx])
+	if err != nil {
+		return nil, fmt.Errorf("Parse Unix domain flags(%s) failed: %s", fields[netUnixFlagsIdx], err)
+	}
+	typ, err := u.parseType(fields[netUnixTypeIdx])
+	if err != nil {
+		return nil, fmt.Errorf("Parse Unix domain type(%s) failed: %s", fields[netUnixTypeIdx], err)
+	}
+	state, err := u.parseState(fields[netUnixStateIdx])
+	if err != nil {
+		return nil, fmt.Errorf("Parse Unix domain state(%s) failed: %s", fields[netUnixStateIdx], err)
+	}
+	var inode uint64
+	if hasInode {
+		inodeStr := fields[netUnixInodeIdx]
+		inode, err = u.parseInode(inodeStr)
+		if err != nil {
+			return nil, fmt.Errorf("Parse Unix domain inode(%s) failed: %s", inodeStr, err)
+		}
+	}
+
+	nuLine := &NetUnixLine{
+		KernelPtr: kernelPtr,
+		RefCount:  users,
+		Type:      typ,
+		Flags:     flags,
+		State:     state,
+		Inode:     inode,
+	}
+
+	// Path field is optional.
+	if fieldsLen > minFieldsCnt {
+		pathIdx := netUnixInodeIdx + 1
+		if !hasInode {
+			pathIdx--
+		}
+		nuLine.Path = fields[pathIdx]
+	}
+
+	return nuLine, nil
+}
+
+func (u NetUnix) parseKernelPtr(str string) (string, error) {
+	if !strings.HasSuffix(str, ":") {
+		return "", errInvalidKernelPtrFmt
+	}
+	return str[:len(str)-1], nil
+}
+
+func (u NetUnix) parseUsers(hexStr string) (uint64, error) {
+	return strconv.ParseUint(hexStr, 16, 32)
+}
+
+func (u NetUnix) parseProtocol(hexStr string) (uint64, error) {
+	return strconv.ParseUint(hexStr, 16, 32)
+}
+
+func (u NetUnix) parseType(hexStr string) (NetUnixType, error) {
+	typ, err := strconv.ParseUint(hexStr, 16, 16)
+	if err != nil {
+		return 0, err
+	}
+	return NetUnixType(typ), nil
+}
+
+func (u NetUnix) parseFlags(hexStr string) (NetUnixFlags, error) {
+	flags, err := strconv.ParseUint(hexStr, 16, 32)
+	if err != nil {
+		return 0, err
+	}
+	return NetUnixFlags(flags), nil
+}
+
+func (u NetUnix) parseState(hexStr string) (NetUnixState, error) {
+	st, err := strconv.ParseInt(hexStr, 16, 8)
+	if err != nil {
+		return 0, err
+	}
+	return NetUnixState(st), nil
+}
+
+func (u NetUnix) parseInode(inodeStr string) (uint64, error) {
+	return strconv.ParseUint(inodeStr, 10, 64)
+}
+
+func (t NetUnixType) String() string {
+	switch t {
+	case netUnixTypeStream:
+		return "stream"
+	case netUnixTypeDgram:
+		return "dgram"
+	case netUnixTypeSeqpacket:
+		return "seqpacket"
+	}
+	return "unknown"
+}
+
+func (f NetUnixFlags) String() string {
+	switch f {
+	case netUnixFlagListen:
+		return "listen"
+	default:
+		return "default"
+	}
+}
+
+func (s NetUnixState) String() string {
+	switch s {
+	case netUnixStateUnconnected:
+		return "unconnected"
+	case netUnixStateConnecting:
+		return "connecting"
+	case netUnixStateConnected:
+		return "connected"
+	case netUnixStateDisconnected:
+		return "disconnected"
+	}
+	return "unknown"
+}
diff --git a/vendor/github.com/prometheus/procfs/proc.go b/vendor/github.com/prometheus/procfs/proc.go
new file mode 100644
index 0000000..8a84301
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/proc.go
@@ -0,0 +1,267 @@
+// Copyright 2018 The Prometheus 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 procfs
+
+import (
+	"bytes"
+	"fmt"
+	"io/ioutil"
+	"os"
+	"strconv"
+	"strings"
+
+	"github.com/prometheus/procfs/internal/fs"
+)
+
+// Proc provides information about a running process.
+type Proc struct {
+	// The process ID.
+	PID int
+
+	fs fs.FS
+}
+
+// Procs represents a list of Proc structs.
+type Procs []Proc
+
+func (p Procs) Len() int           { return len(p) }
+func (p Procs) Swap(i, j int)      { p[i], p[j] = p[j], p[i] }
+func (p Procs) Less(i, j int) bool { return p[i].PID < p[j].PID }
+
+// Self returns a process for the current process read via /proc/self.
+func Self() (Proc, error) {
+	fs, err := NewFS(DefaultMountPoint)
+	if err != nil {
+		return Proc{}, err
+	}
+	return fs.Self()
+}
+
+// NewProc returns a process for the given pid under /proc.
+func NewProc(pid int) (Proc, error) {
+	fs, err := NewFS(DefaultMountPoint)
+	if err != nil {
+		return Proc{}, err
+	}
+	return fs.Proc(pid)
+}
+
+// AllProcs returns a list of all currently available processes under /proc.
+func AllProcs() (Procs, error) {
+	fs, err := NewFS(DefaultMountPoint)
+	if err != nil {
+		return Procs{}, err
+	}
+	return fs.AllProcs()
+}
+
+// Self returns a process for the current process.
+func (fs FS) Self() (Proc, error) {
+	p, err := os.Readlink(fs.proc.Path("self"))
+	if err != nil {
+		return Proc{}, err
+	}
+	pid, err := strconv.Atoi(strings.Replace(p, string(fs.proc), "", -1))
+	if err != nil {
+		return Proc{}, err
+	}
+	return fs.Proc(pid)
+}
+
+// NewProc returns a process for the given pid.
+//
+// Deprecated: use fs.Proc() instead
+func (fs FS) NewProc(pid int) (Proc, error) {
+	return fs.Proc(pid)
+}
+
+// Proc returns a process for the given pid.
+func (fs FS) Proc(pid int) (Proc, error) {
+	if _, err := os.Stat(fs.proc.Path(strconv.Itoa(pid))); err != nil {
+		return Proc{}, err
+	}
+	return Proc{PID: pid, fs: fs.proc}, nil
+}
+
+// AllProcs returns a list of all currently available processes.
+func (fs FS) AllProcs() (Procs, error) {
+	d, err := os.Open(fs.proc.Path())
+	if err != nil {
+		return Procs{}, err
+	}
+	defer d.Close()
+
+	names, err := d.Readdirnames(-1)
+	if err != nil {
+		return Procs{}, fmt.Errorf("could not read %s: %s", d.Name(), err)
+	}
+
+	p := Procs{}
+	for _, n := range names {
+		pid, err := strconv.ParseInt(n, 10, 64)
+		if err != nil {
+			continue
+		}
+		p = append(p, Proc{PID: int(pid), fs: fs.proc})
+	}
+
+	return p, nil
+}
+
+// CmdLine returns the command line of a process.
+func (p Proc) CmdLine() ([]string, error) {
+	f, err := os.Open(p.path("cmdline"))
+	if err != nil {
+		return nil, err
+	}
+	defer f.Close()
+
+	data, err := ioutil.ReadAll(f)
+	if err != nil {
+		return nil, err
+	}
+
+	if len(data) < 1 {
+		return []string{}, nil
+	}
+
+	return strings.Split(string(bytes.TrimRight(data, string("\x00"))), string(byte(0))), nil
+}
+
+// Comm returns the command name of a process.
+func (p Proc) Comm() (string, error) {
+	f, err := os.Open(p.path("comm"))
+	if err != nil {
+		return "", err
+	}
+	defer f.Close()
+
+	data, err := ioutil.ReadAll(f)
+	if err != nil {
+		return "", err
+	}
+
+	return strings.TrimSpace(string(data)), nil
+}
+
+// Executable returns the absolute path of the executable command of a process.
+func (p Proc) Executable() (string, error) {
+	exe, err := os.Readlink(p.path("exe"))
+	if os.IsNotExist(err) {
+		return "", nil
+	}
+
+	return exe, err
+}
+
+// Cwd returns the absolute path to the current working directory of the process.
+func (p Proc) Cwd() (string, error) {
+	wd, err := os.Readlink(p.path("cwd"))
+	if os.IsNotExist(err) {
+		return "", nil
+	}
+
+	return wd, err
+}
+
+// RootDir returns the absolute path to the process's root directory (as set by chroot)
+func (p Proc) RootDir() (string, error) {
+	rdir, err := os.Readlink(p.path("root"))
+	if os.IsNotExist(err) {
+		return "", nil
+	}
+
+	return rdir, err
+}
+
+// FileDescriptors returns the currently open file descriptors of a process.
+func (p Proc) FileDescriptors() ([]uintptr, error) {
+	names, err := p.fileDescriptors()
+	if err != nil {
+		return nil, err
+	}
+
+	fds := make([]uintptr, len(names))
+	for i, n := range names {
+		fd, err := strconv.ParseInt(n, 10, 32)
+		if err != nil {
+			return nil, fmt.Errorf("could not parse fd %s: %s", n, err)
+		}
+		fds[i] = uintptr(fd)
+	}
+
+	return fds, nil
+}
+
+// FileDescriptorTargets returns the targets of all file descriptors of a process.
+// If a file descriptor is not a symlink to a file (like a socket), that value will be the empty string.
+func (p Proc) FileDescriptorTargets() ([]string, error) {
+	names, err := p.fileDescriptors()
+	if err != nil {
+		return nil, err
+	}
+
+	targets := make([]string, len(names))
+
+	for i, name := range names {
+		target, err := os.Readlink(p.path("fd", name))
+		if err == nil {
+			targets[i] = target
+		}
+	}
+
+	return targets, nil
+}
+
+// FileDescriptorsLen returns the number of currently open file descriptors of
+// a process.
+func (p Proc) FileDescriptorsLen() (int, error) {
+	fds, err := p.fileDescriptors()
+	if err != nil {
+		return 0, err
+	}
+
+	return len(fds), nil
+}
+
+// MountStats retrieves statistics and configuration for mount points in a
+// process's namespace.
+func (p Proc) MountStats() ([]*Mount, error) {
+	f, err := os.Open(p.path("mountstats"))
+	if err != nil {
+		return nil, err
+	}
+	defer f.Close()
+
+	return parseMountStats(f)
+}
+
+func (p Proc) fileDescriptors() ([]string, error) {
+	d, err := os.Open(p.path("fd"))
+	if err != nil {
+		return nil, err
+	}
+	defer d.Close()
+
+	names, err := d.Readdirnames(-1)
+	if err != nil {
+		return nil, fmt.Errorf("could not read %s: %s", d.Name(), err)
+	}
+
+	return names, nil
+}
+
+func (p Proc) path(pa ...string) string {
+	return p.fs.Path(append([]string{strconv.Itoa(p.PID)}, pa...)...)
+}
diff --git a/vendor/github.com/prometheus/procfs/proc_io.go b/vendor/github.com/prometheus/procfs/proc_io.go
new file mode 100644
index 0000000..0ff89b1
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/proc_io.go
@@ -0,0 +1,65 @@
+// Copyright 2018 The Prometheus 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 procfs
+
+import (
+	"fmt"
+	"io/ioutil"
+	"os"
+)
+
+// ProcIO models the content of /proc/<pid>/io.
+type ProcIO struct {
+	// Chars read.
+	RChar uint64
+	// Chars written.
+	WChar uint64
+	// Read syscalls.
+	SyscR uint64
+	// Write syscalls.
+	SyscW uint64
+	// Bytes read.
+	ReadBytes uint64
+	// Bytes written.
+	WriteBytes uint64
+	// Bytes written, but taking into account truncation. See
+	// Documentation/filesystems/proc.txt in the kernel sources for
+	// detailed explanation.
+	CancelledWriteBytes int64
+}
+
+// IO creates a new ProcIO instance from a given Proc instance.
+func (p Proc) IO() (ProcIO, error) {
+	pio := ProcIO{}
+
+	f, err := os.Open(p.path("io"))
+	if err != nil {
+		return pio, err
+	}
+	defer f.Close()
+
+	data, err := ioutil.ReadAll(f)
+	if err != nil {
+		return pio, err
+	}
+
+	ioFormat := "rchar: %d\nwchar: %d\nsyscr: %d\nsyscw: %d\n" +
+		"read_bytes: %d\nwrite_bytes: %d\n" +
+		"cancelled_write_bytes: %d\n"
+
+	_, err = fmt.Sscanf(string(data), ioFormat, &pio.RChar, &pio.WChar, &pio.SyscR,
+		&pio.SyscW, &pio.ReadBytes, &pio.WriteBytes, &pio.CancelledWriteBytes)
+
+	return pio, err
+}
diff --git a/vendor/github.com/prometheus/procfs/proc_limits.go b/vendor/github.com/prometheus/procfs/proc_limits.go
new file mode 100644
index 0000000..91ee24d
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/proc_limits.go
@@ -0,0 +1,157 @@
+// Copyright 2018 The Prometheus 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 procfs
+
+import (
+	"bufio"
+	"fmt"
+	"os"
+	"regexp"
+	"strconv"
+)
+
+// ProcLimits represents the soft limits for each of the process's resource
+// limits. For more information see getrlimit(2):
+// http://man7.org/linux/man-pages/man2/getrlimit.2.html.
+type ProcLimits struct {
+	// CPU time limit in seconds.
+	CPUTime int64
+	// Maximum size of files that the process may create.
+	FileSize int64
+	// Maximum size of the process's data segment (initialized data,
+	// uninitialized data, and heap).
+	DataSize int64
+	// Maximum size of the process stack in bytes.
+	StackSize int64
+	// Maximum size of a core file.
+	CoreFileSize int64
+	// Limit of the process's resident set in pages.
+	ResidentSet int64
+	// Maximum number of processes that can be created for the real user ID of
+	// the calling process.
+	Processes int64
+	// Value one greater than the maximum file descriptor number that can be
+	// opened by this process.
+	OpenFiles int64
+	// Maximum number of bytes of memory that may be locked into RAM.
+	LockedMemory int64
+	// Maximum size of the process's virtual memory address space in bytes.
+	AddressSpace int64
+	// Limit on the combined number of flock(2) locks and fcntl(2) leases that
+	// this process may establish.
+	FileLocks int64
+	// Limit of signals that may be queued for the real user ID of the calling
+	// process.
+	PendingSignals int64
+	// Limit on the number of bytes that can be allocated for POSIX message
+	// queues for the real user ID of the calling process.
+	MsqqueueSize int64
+	// Limit of the nice priority set using setpriority(2) or nice(2).
+	NicePriority int64
+	// Limit of the real-time priority set using sched_setscheduler(2) or
+	// sched_setparam(2).
+	RealtimePriority int64
+	// Limit (in microseconds) on the amount of CPU time that a process
+	// scheduled under a real-time scheduling policy may consume without making
+	// a blocking system call.
+	RealtimeTimeout int64
+}
+
+const (
+	limitsFields    = 3
+	limitsUnlimited = "unlimited"
+)
+
+var (
+	limitsDelimiter = regexp.MustCompile("  +")
+)
+
+// NewLimits returns the current soft limits of the process.
+//
+// Deprecated: use p.Limits() instead
+func (p Proc) NewLimits() (ProcLimits, error) {
+	return p.Limits()
+}
+
+// Limits returns the current soft limits of the process.
+func (p Proc) Limits() (ProcLimits, error) {
+	f, err := os.Open(p.path("limits"))
+	if err != nil {
+		return ProcLimits{}, err
+	}
+	defer f.Close()
+
+	var (
+		l = ProcLimits{}
+		s = bufio.NewScanner(f)
+	)
+	for s.Scan() {
+		fields := limitsDelimiter.Split(s.Text(), limitsFields)
+		if len(fields) != limitsFields {
+			return ProcLimits{}, fmt.Errorf(
+				"couldn't parse %s line %s", f.Name(), s.Text())
+		}
+
+		switch fields[0] {
+		case "Max cpu time":
+			l.CPUTime, err = parseInt(fields[1])
+		case "Max file size":
+			l.FileSize, err = parseInt(fields[1])
+		case "Max data size":
+			l.DataSize, err = parseInt(fields[1])
+		case "Max stack size":
+			l.StackSize, err = parseInt(fields[1])
+		case "Max core file size":
+			l.CoreFileSize, err = parseInt(fields[1])
+		case "Max resident set":
+			l.ResidentSet, err = parseInt(fields[1])
+		case "Max processes":
+			l.Processes, err = parseInt(fields[1])
+		case "Max open files":
+			l.OpenFiles, err = parseInt(fields[1])
+		case "Max locked memory":
+			l.LockedMemory, err = parseInt(fields[1])
+		case "Max address space":
+			l.AddressSpace, err = parseInt(fields[1])
+		case "Max file locks":
+			l.FileLocks, err = parseInt(fields[1])
+		case "Max pending signals":
+			l.PendingSignals, err = parseInt(fields[1])
+		case "Max msgqueue size":
+			l.MsqqueueSize, err = parseInt(fields[1])
+		case "Max nice priority":
+			l.NicePriority, err = parseInt(fields[1])
+		case "Max realtime priority":
+			l.RealtimePriority, err = parseInt(fields[1])
+		case "Max realtime timeout":
+			l.RealtimeTimeout, err = parseInt(fields[1])
+		}
+		if err != nil {
+			return ProcLimits{}, err
+		}
+	}
+
+	return l, s.Err()
+}
+
+func parseInt(s string) (int64, error) {
+	if s == limitsUnlimited {
+		return -1, nil
+	}
+	i, err := strconv.ParseInt(s, 10, 64)
+	if err != nil {
+		return 0, fmt.Errorf("couldn't parse value %s: %s", s, err)
+	}
+	return i, nil
+}
diff --git a/vendor/github.com/prometheus/procfs/proc_ns.go b/vendor/github.com/prometheus/procfs/proc_ns.go
new file mode 100644
index 0000000..c66740f
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/proc_ns.go
@@ -0,0 +1,68 @@
+// Copyright 2018 The Prometheus 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 procfs
+
+import (
+	"fmt"
+	"os"
+	"strconv"
+	"strings"
+)
+
+// Namespace represents a single namespace of a process.
+type Namespace struct {
+	Type  string // Namespace type.
+	Inode uint32 // Inode number of the namespace. If two processes are in the same namespace their inodes will match.
+}
+
+// Namespaces contains all of the namespaces that the process is contained in.
+type Namespaces map[string]Namespace
+
+// Namespaces reads from /proc/<pid>/ns/* to get the namespaces of which the
+// process is a member.
+func (p Proc) Namespaces() (Namespaces, error) {
+	d, err := os.Open(p.path("ns"))
+	if err != nil {
+		return nil, err
+	}
+	defer d.Close()
+
+	names, err := d.Readdirnames(-1)
+	if err != nil {
+		return nil, fmt.Errorf("failed to read contents of ns dir: %v", err)
+	}
+
+	ns := make(Namespaces, len(names))
+	for _, name := range names {
+		target, err := os.Readlink(p.path("ns", name))
+		if err != nil {
+			return nil, err
+		}
+
+		fields := strings.SplitN(target, ":", 2)
+		if len(fields) != 2 {
+			return nil, fmt.Errorf("failed to parse namespace type and inode from '%v'", target)
+		}
+
+		typ := fields[0]
+		inode, err := strconv.ParseUint(strings.Trim(fields[1], "[]"), 10, 32)
+		if err != nil {
+			return nil, fmt.Errorf("failed to parse inode from '%v': %v", fields[1], err)
+		}
+
+		ns[name] = Namespace{typ, uint32(inode)}
+	}
+
+	return ns, nil
+}
diff --git a/vendor/github.com/prometheus/procfs/proc_psi.go b/vendor/github.com/prometheus/procfs/proc_psi.go
new file mode 100644
index 0000000..46fe266
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/proc_psi.go
@@ -0,0 +1,101 @@
+// Copyright 2019 The Prometheus 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 procfs
+
+// The PSI / pressure interface is described at
+//   https://git.kernel.org/pub/scm/linux/kernel/git/torvalds/linux.git/tree/Documentation/accounting/psi.txt
+// Each resource (cpu, io, memory, ...) is exposed as a single file.
+// Each file may contain up to two lines, one for "some" pressure and one for "full" pressure.
+// Each line contains several averages (over n seconds) and a total in µs.
+//
+// Example io pressure file:
+// > some avg10=0.06 avg60=0.21 avg300=0.99 total=8537362
+// > full avg10=0.00 avg60=0.13 avg300=0.96 total=8183134
+
+import (
+	"fmt"
+	"io"
+	"io/ioutil"
+	"os"
+	"strings"
+)
+
+const lineFormat = "avg10=%f avg60=%f avg300=%f total=%d"
+
+// PSILine is a single line of values as returned by /proc/pressure/*
+// The Avg entries are averages over n seconds, as a percentage
+// The Total line is in microseconds
+type PSILine struct {
+	Avg10  float64
+	Avg60  float64
+	Avg300 float64
+	Total  uint64
+}
+
+// PSIStats represent pressure stall information from /proc/pressure/*
+// Some indicates the share of time in which at least some tasks are stalled
+// Full indicates the share of time in which all non-idle tasks are stalled simultaneously
+type PSIStats struct {
+	Some *PSILine
+	Full *PSILine
+}
+
+// PSIStatsForResource reads pressure stall information for the specified
+// resource from /proc/pressure/<resource>. At time of writing this can be
+// either "cpu", "memory" or "io".
+func (fs FS) PSIStatsForResource(resource string) (PSIStats, error) {
+	file, err := os.Open(fs.proc.Path(fmt.Sprintf("%s/%s", "pressure", resource)))
+	if err != nil {
+		return PSIStats{}, fmt.Errorf("psi_stats: unavailable for %s", resource)
+	}
+
+	defer file.Close()
+	return parsePSIStats(resource, file)
+}
+
+// parsePSIStats parses the specified file for pressure stall information
+func parsePSIStats(resource string, file io.Reader) (PSIStats, error) {
+	psiStats := PSIStats{}
+	stats, err := ioutil.ReadAll(file)
+	if err != nil {
+		return psiStats, fmt.Errorf("psi_stats: unable to read data for %s", resource)
+	}
+
+	for _, l := range strings.Split(string(stats), "\n") {
+		prefix := strings.Split(l, " ")[0]
+		switch prefix {
+		case "some":
+			psi := PSILine{}
+			_, err := fmt.Sscanf(l, fmt.Sprintf("some %s", lineFormat), &psi.Avg10, &psi.Avg60, &psi.Avg300, &psi.Total)
+			if err != nil {
+				return PSIStats{}, err
+			}
+			psiStats.Some = &psi
+		case "full":
+			psi := PSILine{}
+			_, err := fmt.Sscanf(l, fmt.Sprintf("full %s", lineFormat), &psi.Avg10, &psi.Avg60, &psi.Avg300, &psi.Total)
+			if err != nil {
+				return PSIStats{}, err
+			}
+			psiStats.Full = &psi
+		default:
+			// If we encounter a line with an unknown prefix, ignore it and move on
+			// Should new measurement types be added in the future we'll simply ignore them instead
+			// of erroring on retrieval
+			continue
+		}
+	}
+
+	return psiStats, nil
+}
diff --git a/vendor/github.com/prometheus/procfs/proc_stat.go b/vendor/github.com/prometheus/procfs/proc_stat.go
new file mode 100644
index 0000000..6ed98a8
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/proc_stat.go
@@ -0,0 +1,198 @@
+// Copyright 2018 The Prometheus 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 procfs
+
+import (
+	"bytes"
+	"fmt"
+	"io/ioutil"
+	"os"
+
+	"github.com/prometheus/procfs/internal/fs"
+)
+
+// Originally, this USER_HZ value was dynamically retrieved via a sysconf call
+// which required cgo. However, that caused a lot of problems regarding
+// cross-compilation. Alternatives such as running a binary to determine the
+// value, or trying to derive it in some other way were all problematic.  After
+// much research it was determined that USER_HZ is actually hardcoded to 100 on
+// all Go-supported platforms as of the time of this writing. This is why we
+// decided to hardcode it here as well. It is not impossible that there could
+// be systems with exceptions, but they should be very exotic edge cases, and
+// in that case, the worst outcome will be two misreported metrics.
+//
+// See also the following discussions:
+//
+// - https://github.com/prometheus/node_exporter/issues/52
+// - https://github.com/prometheus/procfs/pull/2
+// - http://stackoverflow.com/questions/17410841/how-does-user-hz-solve-the-jiffy-scaling-issue
+const userHZ = 100
+
+// ProcStat provides status information about the process,
+// read from /proc/[pid]/stat.
+type ProcStat struct {
+	// The process ID.
+	PID int
+	// The filename of the executable.
+	Comm string
+	// The process state.
+	State string
+	// The PID of the parent of this process.
+	PPID int
+	// The process group ID of the process.
+	PGRP int
+	// The session ID of the process.
+	Session int
+	// The controlling terminal of the process.
+	TTY int
+	// The ID of the foreground process group of the controlling terminal of
+	// the process.
+	TPGID int
+	// The kernel flags word of the process.
+	Flags uint
+	// The number of minor faults the process has made which have not required
+	// loading a memory page from disk.
+	MinFlt uint
+	// The number of minor faults that the process's waited-for children have
+	// made.
+	CMinFlt uint
+	// The number of major faults the process has made which have required
+	// loading a memory page from disk.
+	MajFlt uint
+	// The number of major faults that the process's waited-for children have
+	// made.
+	CMajFlt uint
+	// Amount of time that this process has been scheduled in user mode,
+	// measured in clock ticks.
+	UTime uint
+	// Amount of time that this process has been scheduled in kernel mode,
+	// measured in clock ticks.
+	STime uint
+	// Amount of time that this process's waited-for children have been
+	// scheduled in user mode, measured in clock ticks.
+	CUTime uint
+	// Amount of time that this process's waited-for children have been
+	// scheduled in kernel mode, measured in clock ticks.
+	CSTime uint
+	// For processes running a real-time scheduling policy, this is the negated
+	// scheduling priority, minus one.
+	Priority int
+	// The nice value, a value in the range 19 (low priority) to -20 (high
+	// priority).
+	Nice int
+	// Number of threads in this process.
+	NumThreads int
+	// The time the process started after system boot, the value is expressed
+	// in clock ticks.
+	Starttime uint64
+	// Virtual memory size in bytes.
+	VSize uint
+	// Resident set size in pages.
+	RSS int
+
+	proc fs.FS
+}
+
+// NewStat returns the current status information of the process.
+//
+// Deprecated: use NewStat() instead
+func (p Proc) NewStat() (ProcStat, error) {
+	return p.Stat()
+}
+
+// Stat returns the current status information of the process.
+func (p Proc) Stat() (ProcStat, error) {
+	f, err := os.Open(p.path("stat"))
+	if err != nil {
+		return ProcStat{}, err
+	}
+	defer f.Close()
+
+	data, err := ioutil.ReadAll(f)
+	if err != nil {
+		return ProcStat{}, err
+	}
+
+	var (
+		ignore int
+
+		s = ProcStat{PID: p.PID, proc: p.fs}
+		l = bytes.Index(data, []byte("("))
+		r = bytes.LastIndex(data, []byte(")"))
+	)
+
+	if l < 0 || r < 0 {
+		return ProcStat{}, fmt.Errorf(
+			"unexpected format, couldn't extract comm: %s",
+			data,
+		)
+	}
+
+	s.Comm = string(data[l+1 : r])
+	_, err = fmt.Fscan(
+		bytes.NewBuffer(data[r+2:]),
+		&s.State,
+		&s.PPID,
+		&s.PGRP,
+		&s.Session,
+		&s.TTY,
+		&s.TPGID,
+		&s.Flags,
+		&s.MinFlt,
+		&s.CMinFlt,
+		&s.MajFlt,
+		&s.CMajFlt,
+		&s.UTime,
+		&s.STime,
+		&s.CUTime,
+		&s.CSTime,
+		&s.Priority,
+		&s.Nice,
+		&s.NumThreads,
+		&ignore,
+		&s.Starttime,
+		&s.VSize,
+		&s.RSS,
+	)
+	if err != nil {
+		return ProcStat{}, err
+	}
+
+	return s, nil
+}
+
+// VirtualMemory returns the virtual memory size in bytes.
+func (s ProcStat) VirtualMemory() uint {
+	return s.VSize
+}
+
+// ResidentMemory returns the resident memory size in bytes.
+func (s ProcStat) ResidentMemory() int {
+	return s.RSS * os.Getpagesize()
+}
+
+// StartTime returns the unix timestamp of the process in seconds.
+func (s ProcStat) StartTime() (float64, error) {
+	fs := FS{proc: s.proc}
+	stat, err := fs.Stat()
+	if err != nil {
+		return 0, err
+	}
+	return float64(stat.BootTime) + (float64(s.Starttime) / userHZ), nil
+}
+
+// CPUTime returns the total CPU user and system time in seconds.
+func (s ProcStat) CPUTime() float64 {
+	return float64(s.UTime+s.STime) / userHZ
+}
diff --git a/vendor/github.com/prometheus/procfs/proc_status.go b/vendor/github.com/prometheus/procfs/proc_status.go
new file mode 100644
index 0000000..6b4b61f
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/proc_status.go
@@ -0,0 +1,162 @@
+// Copyright 2018 The Prometheus 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 procfs
+
+import (
+	"bytes"
+	"io/ioutil"
+	"os"
+	"strconv"
+	"strings"
+)
+
+// ProcStat provides status information about the process,
+// read from /proc/[pid]/stat.
+type ProcStatus struct {
+	// The process ID.
+	PID int
+	// The process name.
+	Name string
+
+	// Peak virtual memory size.
+	VmPeak uint64
+	// Virtual memory size.
+	VmSize uint64
+	// Locked memory size.
+	VmLck uint64
+	// Pinned memory size.
+	VmPin uint64
+	// Peak resident set size.
+	VmHWM uint64
+	// Resident set size (sum of RssAnnon RssFile and RssShmem).
+	VmRSS uint64
+	// Size of resident anonymous memory.
+	RssAnon uint64
+	// Size of resident file mappings.
+	RssFile uint64
+	// Size of resident shared memory.
+	RssShmem uint64
+	// Size of data segments.
+	VmData uint64
+	// Size of stack segments.
+	VmStk uint64
+	// Size of text segments.
+	VmExe uint64
+	// Shared library code size.
+	VmLib uint64
+	// Page table entries size.
+	VmPTE uint64
+	// Size of second-level page tables.
+	VmPMD uint64
+	// Swapped-out virtual memory size by anonymous private.
+	VmSwap uint64
+	// Size of hugetlb memory portions
+	HugetlbPages uint64
+
+	// Number of voluntary context switches.
+	VoluntaryCtxtSwitches uint64
+	// Number of involuntary context switches.
+	NonVoluntaryCtxtSwitches uint64
+}
+
+// NewStatus returns the current status information of the process.
+func (p Proc) NewStatus() (ProcStatus, error) {
+	f, err := os.Open(p.path("status"))
+	if err != nil {
+		return ProcStatus{}, err
+	}
+	defer f.Close()
+
+	data, err := ioutil.ReadAll(f)
+	if err != nil {
+		return ProcStatus{}, err
+	}
+
+	s := ProcStatus{PID: p.PID}
+
+	lines := strings.Split(string(data), "\n")
+	for _, line := range lines {
+		if !bytes.Contains([]byte(line), []byte(":")) {
+			continue
+		}
+
+		kv := strings.SplitN(line, ":", 2)
+
+		// removes spaces
+		k := string(strings.TrimSpace(kv[0]))
+		v := string(strings.TrimSpace(kv[1]))
+		// removes "kB"
+		v = string(bytes.Trim([]byte(v), " kB"))
+
+		// value to int when possible
+		// we can skip error check here, 'cause vKBytes is not used when value is a string
+		vKBytes, _ := strconv.ParseUint(v, 10, 64)
+		// convert kB to B
+		vBytes := vKBytes * 1024
+
+		s.fillStatus(k, v, vKBytes, vBytes)
+	}
+
+	return s, nil
+}
+
+func (s *ProcStatus) fillStatus(k string, vString string, vUint uint64, vUintBytes uint64) {
+	switch k {
+	case "Name":
+		s.Name = vString
+	case "VmPeak":
+		s.VmPeak = vUintBytes
+	case "VmSize":
+		s.VmSize = vUintBytes
+	case "VmLck":
+		s.VmLck = vUintBytes
+	case "VmPin":
+		s.VmPin = vUintBytes
+	case "VmHWM":
+		s.VmHWM = vUintBytes
+	case "VmRSS":
+		s.VmRSS = vUintBytes
+	case "RssAnon":
+		s.RssAnon = vUintBytes
+	case "RssFile":
+		s.RssFile = vUintBytes
+	case "RssShmem":
+		s.RssShmem = vUintBytes
+	case "VmData":
+		s.VmData = vUintBytes
+	case "VmStk":
+		s.VmStk = vUintBytes
+	case "VmExe":
+		s.VmExe = vUintBytes
+	case "VmLib":
+		s.VmLib = vUintBytes
+	case "VmPTE":
+		s.VmPTE = vUintBytes
+	case "VmPMD":
+		s.VmPMD = vUintBytes
+	case "VmSwap":
+		s.VmSwap = vUintBytes
+	case "HugetlbPages":
+		s.HugetlbPages = vUintBytes
+	case "voluntary_ctxt_switches":
+		s.VoluntaryCtxtSwitches = vUint
+	case "nonvoluntary_ctxt_switches":
+		s.NonVoluntaryCtxtSwitches = vUint
+	}
+}
+
+// TotalCtxtSwitches returns the total context switch.
+func (s ProcStatus) TotalCtxtSwitches() uint64 {
+	return s.VoluntaryCtxtSwitches + s.NonVoluntaryCtxtSwitches
+}
diff --git a/vendor/github.com/prometheus/procfs/stat.go b/vendor/github.com/prometheus/procfs/stat.go
new file mode 100644
index 0000000..6661ee0
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/stat.go
@@ -0,0 +1,244 @@
+// Copyright 2018 The Prometheus 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 procfs
+
+import (
+	"bufio"
+	"fmt"
+	"io"
+	"os"
+	"strconv"
+	"strings"
+
+	"github.com/prometheus/procfs/internal/fs"
+)
+
+// CPUStat shows how much time the cpu spend in various stages.
+type CPUStat struct {
+	User      float64
+	Nice      float64
+	System    float64
+	Idle      float64
+	Iowait    float64
+	IRQ       float64
+	SoftIRQ   float64
+	Steal     float64
+	Guest     float64
+	GuestNice float64
+}
+
+// SoftIRQStat represent the softirq statistics as exported in the procfs stat file.
+// A nice introduction can be found at https://0xax.gitbooks.io/linux-insides/content/interrupts/interrupts-9.html
+// It is possible to get per-cpu stats by reading /proc/softirqs
+type SoftIRQStat struct {
+	Hi          uint64
+	Timer       uint64
+	NetTx       uint64
+	NetRx       uint64
+	Block       uint64
+	BlockIoPoll uint64
+	Tasklet     uint64
+	Sched       uint64
+	Hrtimer     uint64
+	Rcu         uint64
+}
+
+// Stat represents kernel/system statistics.
+type Stat struct {
+	// Boot time in seconds since the Epoch.
+	BootTime uint64
+	// Summed up cpu statistics.
+	CPUTotal CPUStat
+	// Per-CPU statistics.
+	CPU []CPUStat
+	// Number of times interrupts were handled, which contains numbered and unnumbered IRQs.
+	IRQTotal uint64
+	// Number of times a numbered IRQ was triggered.
+	IRQ []uint64
+	// Number of times a context switch happened.
+	ContextSwitches uint64
+	// Number of times a process was created.
+	ProcessCreated uint64
+	// Number of processes currently running.
+	ProcessesRunning uint64
+	// Number of processes currently blocked (waiting for IO).
+	ProcessesBlocked uint64
+	// Number of times a softirq was scheduled.
+	SoftIRQTotal uint64
+	// Detailed softirq statistics.
+	SoftIRQ SoftIRQStat
+}
+
+// Parse a cpu statistics line and returns the CPUStat struct plus the cpu id (or -1 for the overall sum).
+func parseCPUStat(line string) (CPUStat, int64, error) {
+	cpuStat := CPUStat{}
+	var cpu string
+
+	count, err := fmt.Sscanf(line, "%s %f %f %f %f %f %f %f %f %f %f",
+		&cpu,
+		&cpuStat.User, &cpuStat.Nice, &cpuStat.System, &cpuStat.Idle,
+		&cpuStat.Iowait, &cpuStat.IRQ, &cpuStat.SoftIRQ, &cpuStat.Steal,
+		&cpuStat.Guest, &cpuStat.GuestNice)
+
+	if err != nil && err != io.EOF {
+		return CPUStat{}, -1, fmt.Errorf("couldn't parse %s (cpu): %s", line, err)
+	}
+	if count == 0 {
+		return CPUStat{}, -1, fmt.Errorf("couldn't parse %s (cpu): 0 elements parsed", line)
+	}
+
+	cpuStat.User /= userHZ
+	cpuStat.Nice /= userHZ
+	cpuStat.System /= userHZ
+	cpuStat.Idle /= userHZ
+	cpuStat.Iowait /= userHZ
+	cpuStat.IRQ /= userHZ
+	cpuStat.SoftIRQ /= userHZ
+	cpuStat.Steal /= userHZ
+	cpuStat.Guest /= userHZ
+	cpuStat.GuestNice /= userHZ
+
+	if cpu == "cpu" {
+		return cpuStat, -1, nil
+	}
+
+	cpuID, err := strconv.ParseInt(cpu[3:], 10, 64)
+	if err != nil {
+		return CPUStat{}, -1, fmt.Errorf("couldn't parse %s (cpu/cpuid): %s", line, err)
+	}
+
+	return cpuStat, cpuID, nil
+}
+
+// Parse a softirq line.
+func parseSoftIRQStat(line string) (SoftIRQStat, uint64, error) {
+	softIRQStat := SoftIRQStat{}
+	var total uint64
+	var prefix string
+
+	_, err := fmt.Sscanf(line, "%s %d %d %d %d %d %d %d %d %d %d %d",
+		&prefix, &total,
+		&softIRQStat.Hi, &softIRQStat.Timer, &softIRQStat.NetTx, &softIRQStat.NetRx,
+		&softIRQStat.Block, &softIRQStat.BlockIoPoll,
+		&softIRQStat.Tasklet, &softIRQStat.Sched,
+		&softIRQStat.Hrtimer, &softIRQStat.Rcu)
+
+	if err != nil {
+		return SoftIRQStat{}, 0, fmt.Errorf("couldn't parse %s (softirq): %s", line, err)
+	}
+
+	return softIRQStat, total, nil
+}
+
+// NewStat returns information about current cpu/process statistics.
+// See https://www.kernel.org/doc/Documentation/filesystems/proc.txt
+//
+// Deprecated: use fs.Stat() instead
+func NewStat() (Stat, error) {
+	fs, err := NewFS(fs.DefaultProcMountPoint)
+	if err != nil {
+		return Stat{}, err
+	}
+	return fs.Stat()
+}
+
+// NewStat returns information about current cpu/process statistics.
+// See https://www.kernel.org/doc/Documentation/filesystems/proc.txt
+//
+// Deprecated: use fs.Stat() instead
+func (fs FS) NewStat() (Stat, error) {
+	return fs.Stat()
+}
+
+// Stat returns information about current cpu/process statistics.
+// See https://www.kernel.org/doc/Documentation/filesystems/proc.txt
+func (fs FS) Stat() (Stat, error) {
+
+	f, err := os.Open(fs.proc.Path("stat"))
+	if err != nil {
+		return Stat{}, err
+	}
+	defer f.Close()
+
+	stat := Stat{}
+
+	scanner := bufio.NewScanner(f)
+	for scanner.Scan() {
+		line := scanner.Text()
+		parts := strings.Fields(scanner.Text())
+		// require at least <key> <value>
+		if len(parts) < 2 {
+			continue
+		}
+		switch {
+		case parts[0] == "btime":
+			if stat.BootTime, err = strconv.ParseUint(parts[1], 10, 64); err != nil {
+				return Stat{}, fmt.Errorf("couldn't parse %s (btime): %s", parts[1], err)
+			}
+		case parts[0] == "intr":
+			if stat.IRQTotal, err = strconv.ParseUint(parts[1], 10, 64); err != nil {
+				return Stat{}, fmt.Errorf("couldn't parse %s (intr): %s", parts[1], err)
+			}
+			numberedIRQs := parts[2:]
+			stat.IRQ = make([]uint64, len(numberedIRQs))
+			for i, count := range numberedIRQs {
+				if stat.IRQ[i], err = strconv.ParseUint(count, 10, 64); err != nil {
+					return Stat{}, fmt.Errorf("couldn't parse %s (intr%d): %s", count, i, err)
+				}
+			}
+		case parts[0] == "ctxt":
+			if stat.ContextSwitches, err = strconv.ParseUint(parts[1], 10, 64); err != nil {
+				return Stat{}, fmt.Errorf("couldn't parse %s (ctxt): %s", parts[1], err)
+			}
+		case parts[0] == "processes":
+			if stat.ProcessCreated, err = strconv.ParseUint(parts[1], 10, 64); err != nil {
+				return Stat{}, fmt.Errorf("couldn't parse %s (processes): %s", parts[1], err)
+			}
+		case parts[0] == "procs_running":
+			if stat.ProcessesRunning, err = strconv.ParseUint(parts[1], 10, 64); err != nil {
+				return Stat{}, fmt.Errorf("couldn't parse %s (procs_running): %s", parts[1], err)
+			}
+		case parts[0] == "procs_blocked":
+			if stat.ProcessesBlocked, err = strconv.ParseUint(parts[1], 10, 64); err != nil {
+				return Stat{}, fmt.Errorf("couldn't parse %s (procs_blocked): %s", parts[1], err)
+			}
+		case parts[0] == "softirq":
+			softIRQStats, total, err := parseSoftIRQStat(line)
+			if err != nil {
+				return Stat{}, err
+			}
+			stat.SoftIRQTotal = total
+			stat.SoftIRQ = softIRQStats
+		case strings.HasPrefix(parts[0], "cpu"):
+			cpuStat, cpuID, err := parseCPUStat(line)
+			if err != nil {
+				return Stat{}, err
+			}
+			if cpuID == -1 {
+				stat.CPUTotal = cpuStat
+			} else {
+				for int64(len(stat.CPU)) <= cpuID {
+					stat.CPU = append(stat.CPU, CPUStat{})
+				}
+				stat.CPU[cpuID] = cpuStat
+			}
+		}
+	}
+
+	if err := scanner.Err(); err != nil {
+		return Stat{}, fmt.Errorf("couldn't parse %s: %s", f.Name(), err)
+	}
+
+	return stat, nil
+}
diff --git a/vendor/github.com/prometheus/procfs/ttar b/vendor/github.com/prometheus/procfs/ttar
new file mode 100644
index 0000000..19ef02b
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/ttar
@@ -0,0 +1,413 @@
+#!/usr/bin/env bash
+
+# Purpose: plain text tar format
+# Limitations: - only suitable for text files, directories, and symlinks
+#              - stores only filename, content, and mode
+#              - not designed for untrusted input
+#
+# Note: must work with bash version 3.2 (macOS)
+
+# Copyright 2017 Roger Luethi
+#
+# 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.
+
+set -o errexit -o nounset
+
+# Sanitize environment (for instance, standard sorting of glob matches)
+export LC_ALL=C
+
+path=""
+CMD=""
+ARG_STRING="$*"
+
+#------------------------------------------------------------------------------
+# Not all sed implementations can work on null bytes. In order to make ttar
+# work out of the box on macOS, use Python as a stream editor.
+
+USE_PYTHON=0
+
+PYTHON_CREATE_FILTER=$(cat << 'PCF'
+#!/usr/bin/env python
+
+import re
+import sys
+
+for line in sys.stdin:
+    line = re.sub(r'EOF', r'\EOF', line)
+    line = re.sub(r'NULLBYTE', r'\NULLBYTE', line)
+    line = re.sub('\x00', r'NULLBYTE', line)
+    sys.stdout.write(line)
+PCF
+)
+
+PYTHON_EXTRACT_FILTER=$(cat << 'PEF'
+#!/usr/bin/env python
+
+import re
+import sys
+
+for line in sys.stdin:
+    line = re.sub(r'(?<!\\)NULLBYTE', '\x00', line)
+    line = re.sub(r'\\NULLBYTE', 'NULLBYTE', line)
+    line = re.sub(r'([^\\])EOF', r'\1', line)
+    line = re.sub(r'\\EOF', 'EOF', line)
+    sys.stdout.write(line)
+PEF
+)
+
+function test_environment {
+    if [[ "$(echo "a" | sed 's/a/\x0/' | wc -c)" -ne 2 ]]; then
+        echo "WARNING sed unable to handle null bytes, using Python (slow)."
+        if ! which python >/dev/null; then
+            echo "ERROR Python not found. Aborting."
+            exit 2
+        fi
+        USE_PYTHON=1
+    fi
+}
+
+#------------------------------------------------------------------------------
+
+function usage {
+    bname=$(basename "$0")
+    cat << USAGE
+Usage:   $bname [-C <DIR>] -c -f <ARCHIVE> <FILE...> (create archive)
+         $bname            -t -f <ARCHIVE>           (list archive contents)
+         $bname [-C <DIR>] -x -f <ARCHIVE>           (extract archive)
+
+Options:
+         -C <DIR>           (change directory)
+         -v                 (verbose)
+         --recursive-unlink (recursively delete existing directory if path
+                             collides with file or directory to extract)
+
+Example: Change to sysfs directory, create ttar file from fixtures directory
+         $bname -C sysfs -c -f sysfs/fixtures.ttar fixtures/
+USAGE
+exit "$1"
+}
+
+function vecho {
+    if [ "${VERBOSE:-}" == "yes" ]; then
+        echo >&7 "$@"
+    fi
+}
+
+function set_cmd {
+    if [ -n "$CMD" ]; then
+        echo "ERROR: more than one command given"
+        echo
+        usage 2
+    fi
+    CMD=$1
+}
+
+unset VERBOSE
+unset RECURSIVE_UNLINK
+
+while getopts :cf:-:htxvC: opt; do
+    case $opt in
+        c)
+            set_cmd "create"
+            ;;
+        f)
+            ARCHIVE=$OPTARG
+            ;;
+        h)
+            usage 0
+            ;;
+        t)
+            set_cmd "list"
+            ;;
+        x)
+            set_cmd "extract"
+            ;;
+        v)
+            VERBOSE=yes
+            exec 7>&1
+            ;;
+        C)
+            CDIR=$OPTARG
+            ;;
+        -)
+            case $OPTARG in
+                recursive-unlink)
+                    RECURSIVE_UNLINK="yes"
+                    ;;
+                *)
+                    echo -e "Error: invalid option -$OPTARG"
+                    echo
+                    usage 1
+                    ;;
+            esac
+            ;;
+        *)
+            echo >&2 "ERROR: invalid option -$OPTARG"
+            echo
+            usage 1
+            ;;
+    esac
+done
+
+# Remove processed options from arguments
+shift $(( OPTIND - 1 ));
+
+if [ "${CMD:-}" == "" ]; then
+    echo >&2 "ERROR: no command given"
+    echo
+    usage 1
+elif [ "${ARCHIVE:-}" == "" ]; then
+    echo >&2 "ERROR: no archive name given"
+    echo
+    usage 1
+fi
+
+function list {
+    local path=""
+    local size=0
+    local line_no=0
+    local ttar_file=$1
+    if [ -n "${2:-}" ]; then
+        echo >&2 "ERROR: too many arguments."
+        echo
+        usage 1
+    fi
+    if [ ! -e "$ttar_file" ]; then
+        echo >&2 "ERROR: file not found ($ttar_file)"
+        echo
+        usage 1
+    fi
+    while read -r line; do
+        line_no=$(( line_no + 1 ))
+        if [ $size -gt 0 ]; then
+            size=$(( size - 1 ))
+            continue
+        fi
+        if [[ $line =~ ^Path:\ (.*)$ ]]; then
+            path=${BASH_REMATCH[1]}
+        elif [[ $line =~ ^Lines:\ (.*)$ ]]; then
+            size=${BASH_REMATCH[1]}
+            echo "$path"
+        elif [[ $line =~ ^Directory:\ (.*)$ ]]; then
+            path=${BASH_REMATCH[1]}
+            echo "$path/"
+        elif [[ $line =~ ^SymlinkTo:\ (.*)$ ]]; then
+            echo  "$path -> ${BASH_REMATCH[1]}"
+        fi
+    done < "$ttar_file"
+}
+
+function extract {
+    local path=""
+    local size=0
+    local line_no=0
+    local ttar_file=$1
+    if [ -n "${2:-}" ]; then
+        echo >&2 "ERROR: too many arguments."
+        echo
+        usage 1
+    fi
+    if [ ! -e "$ttar_file" ]; then
+        echo >&2 "ERROR: file not found ($ttar_file)"
+        echo
+        usage 1
+    fi
+    while IFS= read -r line; do
+        line_no=$(( line_no + 1 ))
+        local eof_without_newline
+        if [ "$size" -gt 0 ]; then
+            if [[ "$line" =~ [^\\]EOF ]]; then
+                # An EOF not preceded by a backslash indicates that the line
+                # does not end with a newline
+                eof_without_newline=1
+            else
+                eof_without_newline=0
+            fi
+            # Replace NULLBYTE with null byte if at beginning of line
+            # Replace NULLBYTE with null byte unless preceded by backslash
+            # Remove one backslash in front of NULLBYTE (if any)
+            # Remove EOF unless preceded by backslash
+            # Remove one backslash in front of EOF
+            if [ $USE_PYTHON -eq 1 ]; then
+                echo -n "$line" | python -c "$PYTHON_EXTRACT_FILTER" >> "$path"
+            else
+                # The repeated pattern makes up for sed's lack of negative
+                # lookbehind assertions (for consecutive null bytes).
+                echo -n "$line" | \
+                    sed -e 's/^NULLBYTE/\x0/g;
+                            s/\([^\\]\)NULLBYTE/\1\x0/g;
+                            s/\([^\\]\)NULLBYTE/\1\x0/g;
+                            s/\\NULLBYTE/NULLBYTE/g;
+                            s/\([^\\]\)EOF/\1/g;
+                            s/\\EOF/EOF/g;
+                    ' >> "$path"
+            fi
+            if [[ "$eof_without_newline" -eq 0 ]]; then
+                echo >> "$path"
+            fi
+            size=$(( size - 1 ))
+            continue
+        fi
+        if [[ $line =~ ^Path:\ (.*)$ ]]; then
+            path=${BASH_REMATCH[1]}
+            if [ -L "$path" ]; then
+                rm "$path"
+            elif [ -d "$path" ]; then
+                if [ "${RECURSIVE_UNLINK:-}" == "yes" ]; then
+                    rm -r "$path"
+                else
+                    # Safe because symlinks to directories are dealt with above
+                    rmdir "$path"
+                fi
+            elif [ -e "$path" ]; then
+                rm "$path"
+            fi
+        elif [[ $line =~ ^Lines:\ (.*)$ ]]; then
+            size=${BASH_REMATCH[1]}
+            # Create file even if it is zero-length.
+            touch "$path"
+            vecho "    $path"
+        elif [[ $line =~ ^Mode:\ (.*)$ ]]; then
+            mode=${BASH_REMATCH[1]}
+            chmod "$mode" "$path"
+            vecho "$mode"
+        elif [[ $line =~ ^Directory:\ (.*)$ ]]; then
+            path=${BASH_REMATCH[1]}
+            mkdir -p "$path"
+            vecho "    $path/"
+        elif [[ $line =~ ^SymlinkTo:\ (.*)$ ]]; then
+            ln -s "${BASH_REMATCH[1]}" "$path"
+            vecho "    $path -> ${BASH_REMATCH[1]}"
+        elif [[ $line =~ ^# ]]; then
+            # Ignore comments between files
+            continue
+        else
+            echo >&2 "ERROR: Unknown keyword on line $line_no: $line"
+            exit 1
+        fi
+    done < "$ttar_file"
+}
+
+function div {
+    echo "# ttar - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -" \
+         "- - - - - -"
+}
+
+function get_mode {
+    local mfile=$1
+    if [ -z "${STAT_OPTION:-}" ]; then
+        if stat -c '%a' "$mfile" >/dev/null 2>&1; then
+            # GNU stat
+            STAT_OPTION='-c'
+            STAT_FORMAT='%a'
+        else
+            # BSD stat
+            STAT_OPTION='-f'
+            # Octal output, user/group/other (omit file type, sticky bit)
+            STAT_FORMAT='%OLp'
+        fi
+    fi
+    stat "${STAT_OPTION}" "${STAT_FORMAT}" "$mfile"
+}
+
+function _create {
+    shopt -s nullglob
+    local mode
+    local eof_without_newline
+    while (( "$#" )); do
+        file=$1
+        if [ -L "$file" ]; then
+            echo "Path: $file"
+            symlinkTo=$(readlink "$file")
+            echo "SymlinkTo: $symlinkTo"
+            vecho "    $file -> $symlinkTo"
+            div
+        elif [ -d "$file" ]; then
+            # Strip trailing slash (if there is one)
+            file=${file%/}
+            echo "Directory: $file"
+            mode=$(get_mode "$file")
+            echo "Mode: $mode"
+            vecho "$mode $file/"
+            div
+            # Find all files and dirs, including hidden/dot files
+            for x in "$file/"{*,.[^.]*}; do
+                _create "$x"
+            done
+        elif [ -f "$file" ]; then
+            echo "Path: $file"
+            lines=$(wc -l "$file"|awk '{print $1}')
+            eof_without_newline=0
+            if [[ "$(wc -c "$file"|awk '{print $1}')" -gt 0 ]] && \
+                    [[ "$(tail -c 1 "$file" | wc -l)" -eq 0 ]]; then
+                eof_without_newline=1
+                lines=$((lines+1))
+            fi
+            echo "Lines: $lines"
+            # Add backslash in front of EOF
+            # Add backslash in front of NULLBYTE
+            # Replace null byte with NULLBYTE
+            if [ $USE_PYTHON -eq 1 ]; then
+                < "$file" python -c "$PYTHON_CREATE_FILTER"
+            else
+                < "$file" \
+                    sed 's/EOF/\\EOF/g;
+                            s/NULLBYTE/\\NULLBYTE/g;
+                            s/\x0/NULLBYTE/g;
+                    '
+            fi
+            if [[ "$eof_without_newline" -eq 1 ]]; then
+                # Finish line with EOF to indicate that the original line did
+                # not end with a linefeed
+                echo "EOF"
+            fi
+            mode=$(get_mode "$file")
+            echo "Mode: $mode"
+            vecho "$mode $file"
+            div
+        else
+            echo >&2 "ERROR: file not found ($file in $(pwd))"
+            exit 2
+        fi
+        shift
+    done
+}
+
+function create {
+    ttar_file=$1
+    shift
+    if [ -z "${1:-}" ]; then
+        echo >&2 "ERROR: missing arguments."
+        echo
+        usage 1
+    fi
+    if [ -e "$ttar_file" ]; then
+        rm "$ttar_file"
+    fi
+    exec > "$ttar_file"
+    echo "# Archive created by ttar $ARG_STRING"
+    _create "$@"
+}
+
+test_environment
+
+if [ -n "${CDIR:-}" ]; then
+    if [[ "$ARCHIVE" != /* ]]; then
+        # Relative path: preserve the archive's location before changing
+        # directory
+        ARCHIVE="$(pwd)/$ARCHIVE"
+    fi
+    cd "$CDIR"
+fi
+
+"$CMD" "$ARCHIVE" "$@"
diff --git a/vendor/github.com/prometheus/procfs/xfrm.go b/vendor/github.com/prometheus/procfs/xfrm.go
new file mode 100644
index 0000000..30aa417
--- /dev/null
+++ b/vendor/github.com/prometheus/procfs/xfrm.go
@@ -0,0 +1,187 @@
+// Copyright 2017 Prometheus Team
+// 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 procfs
+
+import (
+	"bufio"
+	"fmt"
+	"os"
+	"strconv"
+	"strings"
+)
+
+// XfrmStat models the contents of /proc/net/xfrm_stat.
+type XfrmStat struct {
+	// All errors which are not matched by other
+	XfrmInError int
+	// No buffer is left
+	XfrmInBufferError int
+	// Header Error
+	XfrmInHdrError int
+	// No state found
+	// i.e. either inbound SPI, address, or IPSEC protocol at SA is wrong
+	XfrmInNoStates int
+	// Transformation protocol specific error
+	// e.g. SA Key is wrong
+	XfrmInStateProtoError int
+	// Transformation mode specific error
+	XfrmInStateModeError int
+	// Sequence error
+	// e.g. sequence number is out of window
+	XfrmInStateSeqError int
+	// State is expired
+	XfrmInStateExpired int
+	// State has mismatch option
+	// e.g. UDP encapsulation type is mismatched
+	XfrmInStateMismatch int
+	// State is invalid
+	XfrmInStateInvalid int
+	// No matching template for states
+	// e.g. Inbound SAs are correct but SP rule is wrong
+	XfrmInTmplMismatch int
+	// No policy is found for states
+	// e.g. Inbound SAs are correct but no SP is found
+	XfrmInNoPols int
+	// Policy discards
+	XfrmInPolBlock int
+	// Policy error
+	XfrmInPolError int
+	// All errors which are not matched by others
+	XfrmOutError int
+	// Bundle generation error
+	XfrmOutBundleGenError int
+	// Bundle check error
+	XfrmOutBundleCheckError int
+	// No state was found
+	XfrmOutNoStates int
+	// Transformation protocol specific error
+	XfrmOutStateProtoError int
+	// Transportation mode specific error
+	XfrmOutStateModeError int
+	// Sequence error
+	// i.e sequence number overflow
+	XfrmOutStateSeqError int
+	// State is expired
+	XfrmOutStateExpired int
+	// Policy discads
+	XfrmOutPolBlock int
+	// Policy is dead
+	XfrmOutPolDead int
+	// Policy Error
+	XfrmOutPolError     int
+	XfrmFwdHdrError     int
+	XfrmOutStateInvalid int
+	XfrmAcquireError    int
+}
+
+// NewXfrmStat reads the xfrm_stat statistics.
+func NewXfrmStat() (XfrmStat, error) {
+	fs, err := NewFS(DefaultMountPoint)
+	if err != nil {
+		return XfrmStat{}, err
+	}
+
+	return fs.NewXfrmStat()
+}
+
+// NewXfrmStat reads the xfrm_stat statistics from the 'proc' filesystem.
+func (fs FS) NewXfrmStat() (XfrmStat, error) {
+	file, err := os.Open(fs.proc.Path("net/xfrm_stat"))
+	if err != nil {
+		return XfrmStat{}, err
+	}
+	defer file.Close()
+
+	var (
+		x = XfrmStat{}
+		s = bufio.NewScanner(file)
+	)
+
+	for s.Scan() {
+		fields := strings.Fields(s.Text())
+
+		if len(fields) != 2 {
+			return XfrmStat{}, fmt.Errorf(
+				"couldn't parse %s line %s", file.Name(), s.Text())
+		}
+
+		name := fields[0]
+		value, err := strconv.Atoi(fields[1])
+		if err != nil {
+			return XfrmStat{}, err
+		}
+
+		switch name {
+		case "XfrmInError":
+			x.XfrmInError = value
+		case "XfrmInBufferError":
+			x.XfrmInBufferError = value
+		case "XfrmInHdrError":
+			x.XfrmInHdrError = value
+		case "XfrmInNoStates":
+			x.XfrmInNoStates = value
+		case "XfrmInStateProtoError":
+			x.XfrmInStateProtoError = value
+		case "XfrmInStateModeError":
+			x.XfrmInStateModeError = value
+		case "XfrmInStateSeqError":
+			x.XfrmInStateSeqError = value
+		case "XfrmInStateExpired":
+			x.XfrmInStateExpired = value
+		case "XfrmInStateInvalid":
+			x.XfrmInStateInvalid = value
+		case "XfrmInTmplMismatch":
+			x.XfrmInTmplMismatch = value
+		case "XfrmInNoPols":
+			x.XfrmInNoPols = value
+		case "XfrmInPolBlock":
+			x.XfrmInPolBlock = value
+		case "XfrmInPolError":
+			x.XfrmInPolError = value
+		case "XfrmOutError":
+			x.XfrmOutError = value
+		case "XfrmInStateMismatch":
+			x.XfrmInStateMismatch = value
+		case "XfrmOutBundleGenError":
+			x.XfrmOutBundleGenError = value
+		case "XfrmOutBundleCheckError":
+			x.XfrmOutBundleCheckError = value
+		case "XfrmOutNoStates":
+			x.XfrmOutNoStates = value
+		case "XfrmOutStateProtoError":
+			x.XfrmOutStateProtoError = value
+		case "XfrmOutStateModeError":
+			x.XfrmOutStateModeError = value
+		case "XfrmOutStateSeqError":
+			x.XfrmOutStateSeqError = value
+		case "XfrmOutStateExpired":
+			x.XfrmOutStateExpired = value
+		case "XfrmOutPolBlock":
+			x.XfrmOutPolBlock = value
+		case "XfrmOutPolDead":
+			x.XfrmOutPolDead = value
+		case "XfrmOutPolError":
+			x.XfrmOutPolError = value
+		case "XfrmFwdHdrError":
+			x.XfrmFwdHdrError = value
+		case "XfrmOutStateInvalid":
+			x.XfrmOutStateInvalid = value
+		case "XfrmAcquireError":
+			x.XfrmAcquireError = value
+		}
+
+	}
+
+	return x, s.Err()
+}
diff --git a/vendor/github.com/sirupsen/logrus/.gitignore b/vendor/github.com/sirupsen/logrus/.gitignore
new file mode 100644
index 0000000..6b7d7d1
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/.gitignore
@@ -0,0 +1,2 @@
+logrus
+vendor
diff --git a/vendor/github.com/sirupsen/logrus/.travis.yml b/vendor/github.com/sirupsen/logrus/.travis.yml
new file mode 100644
index 0000000..1f953be
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/.travis.yml
@@ -0,0 +1,51 @@
+language: go
+env:
+  - GOMAXPROCS=4 GORACE=halt_on_error=1
+matrix:
+  include:
+    - go: 1.10.x
+      install:
+        - go get github.com/stretchr/testify/assert
+        - go get golang.org/x/crypto/ssh/terminal
+        - go get golang.org/x/sys/unix
+        - go get golang.org/x/sys/windows
+      script:
+        - go test -race -v ./...
+    - go: 1.11.x
+      env: GO111MODULE=on
+      install:
+        - go mod download
+      script:
+        - go test -race -v ./...
+    - go: 1.11.x
+      env: GO111MODULE=off
+      install:
+        - go get github.com/stretchr/testify/assert
+        - go get golang.org/x/crypto/ssh/terminal
+        - go get golang.org/x/sys/unix
+        - go get golang.org/x/sys/windows
+      script:
+        - go test -race -v ./...
+    - go: 1.10.x
+      install:
+        - go get github.com/stretchr/testify/assert
+        - go get golang.org/x/crypto/ssh/terminal
+        - go get golang.org/x/sys/unix
+        - go get golang.org/x/sys/windows
+      script:
+        - go test -race -v -tags appengine ./...
+    - go: 1.11.x
+      env: GO111MODULE=on
+      install:
+        - go mod download
+      script:
+        - go test -race -v -tags appengine ./...
+    - go: 1.11.x
+      env: GO111MODULE=off
+      install:
+        - go get github.com/stretchr/testify/assert
+        - go get golang.org/x/crypto/ssh/terminal
+        - go get golang.org/x/sys/unix
+        - go get golang.org/x/sys/windows
+      script:
+        - go test -race -v -tags appengine ./...
diff --git a/vendor/github.com/sirupsen/logrus/CHANGELOG.md b/vendor/github.com/sirupsen/logrus/CHANGELOG.md
new file mode 100644
index 0000000..cb85d9f
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/CHANGELOG.md
@@ -0,0 +1,165 @@
+# 1.2.0
+This new release introduces:
+  * A new method `SetReportCaller` in the `Logger` to enable the file, line and calling function from which the trace has been issued
+  * A new trace level named `Trace` whose level is below `Debug`
+  * A configurable exit function to be called upon a Fatal trace
+  * The `Level` object now implements `encoding.TextUnmarshaler` interface
+
+# 1.1.1
+This is a bug fix release.
+  * fix the build break on Solaris
+  * don't drop a whole trace in JSONFormatter when a field param is a function pointer which can not be serialized
+
+# 1.1.0
+This new release introduces:
+  * several fixes:
+    * a fix for a race condition on entry formatting
+    * proper cleanup of previously used entries before putting them back in the pool
+    * the extra new line at the end of message in text formatter has been removed
+  * a new global public API to check if a level is activated: IsLevelEnabled
+  * the following methods have been added to the Logger object
+    * IsLevelEnabled
+    * SetFormatter
+    * SetOutput
+    * ReplaceHooks
+  * introduction of go module
+  * an indent configuration for the json formatter
+  * output colour support for windows
+  * the field sort function is now configurable for text formatter
+  * the CLICOLOR and CLICOLOR\_FORCE environment variable support in text formater
+
+# 1.0.6
+
+This new release introduces:
+  * a new api WithTime which allows to easily force the time of the log entry
+    which is mostly useful for logger wrapper
+  * a fix reverting the immutability of the entry given as parameter to the hooks
+    a new configuration field of the json formatter in order to put all the fields
+    in a nested dictionnary
+  * a new SetOutput method in the Logger
+  * a new configuration of the textformatter to configure the name of the default keys
+  * a new configuration of the text formatter to disable the level truncation
+
+# 1.0.5
+
+* Fix hooks race (#707)
+* Fix panic deadlock (#695)
+
+# 1.0.4
+
+* Fix race when adding hooks (#612)
+* Fix terminal check in AppEngine (#635)
+
+# 1.0.3
+
+* Replace example files with testable examples
+
+# 1.0.2
+
+* bug: quote non-string values in text formatter (#583)
+* Make (*Logger) SetLevel a public method
+
+# 1.0.1
+
+* bug: fix escaping in text formatter (#575)
+
+# 1.0.0
+
+* Officially changed name to lower-case
+* bug: colors on Windows 10 (#541)
+* bug: fix race in accessing level (#512)
+
+# 0.11.5
+
+* feature: add writer and writerlevel to entry (#372)
+
+# 0.11.4
+
+* bug: fix undefined variable on solaris (#493)
+
+# 0.11.3
+
+* formatter: configure quoting of empty values (#484)
+* formatter: configure quoting character (default is `"`) (#484)
+* bug: fix not importing io correctly in non-linux environments (#481)
+
+# 0.11.2
+
+* bug: fix windows terminal detection (#476)
+
+# 0.11.1
+
+* bug: fix tty detection with custom out (#471)
+
+# 0.11.0
+
+* performance: Use bufferpool to allocate (#370)
+* terminal: terminal detection for app-engine (#343)
+* feature: exit handler (#375)
+
+# 0.10.0
+
+* feature: Add a test hook (#180)
+* feature: `ParseLevel` is now case-insensitive (#326)
+* feature: `FieldLogger` interface that generalizes `Logger` and `Entry` (#308)
+* performance: avoid re-allocations on `WithFields` (#335)
+
+# 0.9.0
+
+* logrus/text_formatter: don't emit empty msg
+* logrus/hooks/airbrake: move out of main repository
+* logrus/hooks/sentry: move out of main repository
+* logrus/hooks/papertrail: move out of main repository
+* logrus/hooks/bugsnag: move out of main repository
+* logrus/core: run tests with `-race`
+* logrus/core: detect TTY based on `stderr`
+* logrus/core: support `WithError` on logger
+* logrus/core: Solaris support
+
+# 0.8.7
+
+* logrus/core: fix possible race (#216)
+* logrus/doc: small typo fixes and doc improvements
+
+
+# 0.8.6
+
+* hooks/raven: allow passing an initialized client
+
+# 0.8.5
+
+* logrus/core: revert #208
+
+# 0.8.4
+
+* formatter/text: fix data race (#218)
+
+# 0.8.3
+
+* logrus/core: fix entry log level (#208)
+* logrus/core: improve performance of text formatter by 40%
+* logrus/core: expose `LevelHooks` type
+* logrus/core: add support for DragonflyBSD and NetBSD
+* formatter/text: print structs more verbosely
+
+# 0.8.2
+
+* logrus: fix more Fatal family functions
+
+# 0.8.1
+
+* logrus: fix not exiting on `Fatalf` and `Fatalln`
+
+# 0.8.0
+
+* logrus: defaults to stderr instead of stdout
+* hooks/sentry: add special field for `*http.Request`
+* formatter/text: ignore Windows for colors
+
+# 0.7.3
+
+* formatter/\*: allow configuration of timestamp layout
+
+# 0.7.2
+
+* formatter/text: Add configuration option for time format (#158)
diff --git a/vendor/github.com/sirupsen/logrus/LICENSE b/vendor/github.com/sirupsen/logrus/LICENSE
new file mode 100644
index 0000000..f090cb4
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/LICENSE
@@ -0,0 +1,21 @@
+The MIT License (MIT)
+
+Copyright (c) 2014 Simon Eskildsen
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
diff --git a/vendor/github.com/sirupsen/logrus/README.md b/vendor/github.com/sirupsen/logrus/README.md
new file mode 100644
index 0000000..093bb13
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/README.md
@@ -0,0 +1,493 @@
+# Logrus <img src="http://i.imgur.com/hTeVwmJ.png" width="40" height="40" alt=":walrus:" class="emoji" title=":walrus:"/>&nbsp;[![Build Status](https://travis-ci.org/sirupsen/logrus.svg?branch=master)](https://travis-ci.org/sirupsen/logrus)&nbsp;[![GoDoc](https://godoc.org/github.com/sirupsen/logrus?status.svg)](https://godoc.org/github.com/sirupsen/logrus)
+
+Logrus is a structured logger for Go (golang), completely API compatible with
+the standard library logger.
+
+**Seeing weird case-sensitive problems?** It's in the past been possible to
+import Logrus as both upper- and lower-case. Due to the Go package environment,
+this caused issues in the community and we needed a standard. Some environments
+experienced problems with the upper-case variant, so the lower-case was decided.
+Everything using `logrus` will need to use the lower-case:
+`github.com/sirupsen/logrus`. Any package that isn't, should be changed.
+
+To fix Glide, see [these
+comments](https://github.com/sirupsen/logrus/issues/553#issuecomment-306591437).
+For an in-depth explanation of the casing issue, see [this
+comment](https://github.com/sirupsen/logrus/issues/570#issuecomment-313933276).
+
+**Are you interested in assisting in maintaining Logrus?** Currently I have a
+lot of obligations, and I am unable to provide Logrus with the maintainership it
+needs. If you'd like to help, please reach out to me at `simon at author's
+username dot com`.
+
+Nicely color-coded in development (when a TTY is attached, otherwise just
+plain text):
+
+![Colored](http://i.imgur.com/PY7qMwd.png)
+
+With `log.SetFormatter(&log.JSONFormatter{})`, for easy parsing by logstash
+or Splunk:
+
+```json
+{"animal":"walrus","level":"info","msg":"A group of walrus emerges from the
+ocean","size":10,"time":"2014-03-10 19:57:38.562264131 -0400 EDT"}
+
+{"level":"warning","msg":"The group's number increased tremendously!",
+"number":122,"omg":true,"time":"2014-03-10 19:57:38.562471297 -0400 EDT"}
+
+{"animal":"walrus","level":"info","msg":"A giant walrus appears!",
+"size":10,"time":"2014-03-10 19:57:38.562500591 -0400 EDT"}
+
+{"animal":"walrus","level":"info","msg":"Tremendously sized cow enters the ocean.",
+"size":9,"time":"2014-03-10 19:57:38.562527896 -0400 EDT"}
+
+{"level":"fatal","msg":"The ice breaks!","number":100,"omg":true,
+"time":"2014-03-10 19:57:38.562543128 -0400 EDT"}
+```
+
+With the default `log.SetFormatter(&log.TextFormatter{})` when a TTY is not
+attached, the output is compatible with the
+[logfmt](http://godoc.org/github.com/kr/logfmt) format:
+
+```text
+time="2015-03-26T01:27:38-04:00" level=debug msg="Started observing beach" animal=walrus number=8
+time="2015-03-26T01:27:38-04:00" level=info msg="A group of walrus emerges from the ocean" animal=walrus size=10
+time="2015-03-26T01:27:38-04:00" level=warning msg="The group's number increased tremendously!" number=122 omg=true
+time="2015-03-26T01:27:38-04:00" level=debug msg="Temperature changes" temperature=-4
+time="2015-03-26T01:27:38-04:00" level=panic msg="It's over 9000!" animal=orca size=9009
+time="2015-03-26T01:27:38-04:00" level=fatal msg="The ice breaks!" err=&{0x2082280c0 map[animal:orca size:9009] 2015-03-26 01:27:38.441574009 -0400 EDT panic It's over 9000!} number=100 omg=true
+```
+To ensure this behaviour even if a TTY is attached, set your formatter as follows:
+
+```go
+	log.SetFormatter(&log.TextFormatter{
+		DisableColors: true,
+		FullTimestamp: true,
+	})
+```
+
+#### Logging Method Name
+
+If you wish to add the calling method as a field, instruct the logger via:
+```go
+log.SetReportCaller(true)
+```
+This adds the caller as 'method' like so:
+
+```json
+{"animal":"penguin","level":"fatal","method":"github.com/sirupsen/arcticcreatures.migrate","msg":"a penguin swims by",
+"time":"2014-03-10 19:57:38.562543129 -0400 EDT"}
+```
+
+```text
+time="2015-03-26T01:27:38-04:00" level=fatal method=github.com/sirupsen/arcticcreatures.migrate msg="a penguin swims by" animal=penguin
+```
+Note that this does add measurable overhead - the cost will depend on the version of Go, but is
+between 20 and 40% in recent tests with 1.6 and 1.7.  You can validate this in your
+environment via benchmarks: 
+```
+go test -bench=.*CallerTracing
+```
+
+
+#### Case-sensitivity
+
+The organization's name was changed to lower-case--and this will not be changed
+back. If you are getting import conflicts due to case sensitivity, please use
+the lower-case import: `github.com/sirupsen/logrus`.
+
+#### Example
+
+The simplest way to use Logrus is simply the package-level exported logger:
+
+```go
+package main
+
+import (
+  log "github.com/sirupsen/logrus"
+)
+
+func main() {
+  log.WithFields(log.Fields{
+    "animal": "walrus",
+  }).Info("A walrus appears")
+}
+```
+
+Note that it's completely api-compatible with the stdlib logger, so you can
+replace your `log` imports everywhere with `log "github.com/sirupsen/logrus"`
+and you'll now have the flexibility of Logrus. You can customize it all you
+want:
+
+```go
+package main
+
+import (
+  "os"
+  log "github.com/sirupsen/logrus"
+)
+
+func init() {
+  // Log as JSON instead of the default ASCII formatter.
+  log.SetFormatter(&log.JSONFormatter{})
+
+  // Output to stdout instead of the default stderr
+  // Can be any io.Writer, see below for File example
+  log.SetOutput(os.Stdout)
+
+  // Only log the warning severity or above.
+  log.SetLevel(log.WarnLevel)
+}
+
+func main() {
+  log.WithFields(log.Fields{
+    "animal": "walrus",
+    "size":   10,
+  }).Info("A group of walrus emerges from the ocean")
+
+  log.WithFields(log.Fields{
+    "omg":    true,
+    "number": 122,
+  }).Warn("The group's number increased tremendously!")
+
+  log.WithFields(log.Fields{
+    "omg":    true,
+    "number": 100,
+  }).Fatal("The ice breaks!")
+
+  // A common pattern is to re-use fields between logging statements by re-using
+  // the logrus.Entry returned from WithFields()
+  contextLogger := log.WithFields(log.Fields{
+    "common": "this is a common field",
+    "other": "I also should be logged always",
+  })
+
+  contextLogger.Info("I'll be logged with common and other field")
+  contextLogger.Info("Me too")
+}
+```
+
+For more advanced usage such as logging to multiple locations from the same
+application, you can also create an instance of the `logrus` Logger:
+
+```go
+package main
+
+import (
+  "os"
+  "github.com/sirupsen/logrus"
+)
+
+// Create a new instance of the logger. You can have any number of instances.
+var log = logrus.New()
+
+func main() {
+  // The API for setting attributes is a little different than the package level
+  // exported logger. See Godoc.
+  log.Out = os.Stdout
+
+  // You could set this to any `io.Writer` such as a file
+  // file, err := os.OpenFile("logrus.log", os.O_CREATE|os.O_WRONLY, 0666)
+  // if err == nil {
+  //  log.Out = file
+  // } else {
+  //  log.Info("Failed to log to file, using default stderr")
+  // }
+
+  log.WithFields(logrus.Fields{
+    "animal": "walrus",
+    "size":   10,
+  }).Info("A group of walrus emerges from the ocean")
+}
+```
+
+#### Fields
+
+Logrus encourages careful, structured logging through logging fields instead of
+long, unparseable error messages. For example, instead of: `log.Fatalf("Failed
+to send event %s to topic %s with key %d")`, you should log the much more
+discoverable:
+
+```go
+log.WithFields(log.Fields{
+  "event": event,
+  "topic": topic,
+  "key": key,
+}).Fatal("Failed to send event")
+```
+
+We've found this API forces you to think about logging in a way that produces
+much more useful logging messages. We've been in countless situations where just
+a single added field to a log statement that was already there would've saved us
+hours. The `WithFields` call is optional.
+
+In general, with Logrus using any of the `printf`-family functions should be
+seen as a hint you should add a field, however, you can still use the
+`printf`-family functions with Logrus.
+
+#### Default Fields
+
+Often it's helpful to have fields _always_ attached to log statements in an
+application or parts of one. For example, you may want to always log the
+`request_id` and `user_ip` in the context of a request. Instead of writing
+`log.WithFields(log.Fields{"request_id": request_id, "user_ip": user_ip})` on
+every line, you can create a `logrus.Entry` to pass around instead:
+
+```go
+requestLogger := log.WithFields(log.Fields{"request_id": request_id, "user_ip": user_ip})
+requestLogger.Info("something happened on that request") # will log request_id and user_ip
+requestLogger.Warn("something not great happened")
+```
+
+#### Hooks
+
+You can add hooks for logging levels. For example to send errors to an exception
+tracking service on `Error`, `Fatal` and `Panic`, info to StatsD or log to
+multiple places simultaneously, e.g. syslog.
+
+Logrus comes with [built-in hooks](hooks/). Add those, or your custom hook, in
+`init`:
+
+```go
+import (
+  log "github.com/sirupsen/logrus"
+  "gopkg.in/gemnasium/logrus-airbrake-hook.v2" // the package is named "airbrake"
+  logrus_syslog "github.com/sirupsen/logrus/hooks/syslog"
+  "log/syslog"
+)
+
+func init() {
+
+  // Use the Airbrake hook to report errors that have Error severity or above to
+  // an exception tracker. You can create custom hooks, see the Hooks section.
+  log.AddHook(airbrake.NewHook(123, "xyz", "production"))
+
+  hook, err := logrus_syslog.NewSyslogHook("udp", "localhost:514", syslog.LOG_INFO, "")
+  if err != nil {
+    log.Error("Unable to connect to local syslog daemon")
+  } else {
+    log.AddHook(hook)
+  }
+}
+```
+Note: Syslog hook also support connecting to local syslog (Ex. "/dev/log" or "/var/run/syslog" or "/var/run/log"). For the detail, please check the [syslog hook README](hooks/syslog/README.md).
+
+A list of currently known of service hook can be found in this wiki [page](https://github.com/sirupsen/logrus/wiki/Hooks)
+
+
+#### Level logging
+
+Logrus has seven logging levels: Trace, Debug, Info, Warning, Error, Fatal and Panic.
+
+```go
+log.Trace("Something very low level.")
+log.Debug("Useful debugging information.")
+log.Info("Something noteworthy happened!")
+log.Warn("You should probably take a look at this.")
+log.Error("Something failed but I'm not quitting.")
+// Calls os.Exit(1) after logging
+log.Fatal("Bye.")
+// Calls panic() after logging
+log.Panic("I'm bailing.")
+```
+
+You can set the logging level on a `Logger`, then it will only log entries with
+that severity or anything above it:
+
+```go
+// Will log anything that is info or above (warn, error, fatal, panic). Default.
+log.SetLevel(log.InfoLevel)
+```
+
+It may be useful to set `log.Level = logrus.DebugLevel` in a debug or verbose
+environment if your application has that.
+
+#### Entries
+
+Besides the fields added with `WithField` or `WithFields` some fields are
+automatically added to all logging events:
+
+1. `time`. The timestamp when the entry was created.
+2. `msg`. The logging message passed to `{Info,Warn,Error,Fatal,Panic}` after
+   the `AddFields` call. E.g. `Failed to send event.`
+3. `level`. The logging level. E.g. `info`.
+
+#### Environments
+
+Logrus has no notion of environment.
+
+If you wish for hooks and formatters to only be used in specific environments,
+you should handle that yourself. For example, if your application has a global
+variable `Environment`, which is a string representation of the environment you
+could do:
+
+```go
+import (
+  log "github.com/sirupsen/logrus"
+)
+
+init() {
+  // do something here to set environment depending on an environment variable
+  // or command-line flag
+  if Environment == "production" {
+    log.SetFormatter(&log.JSONFormatter{})
+  } else {
+    // The TextFormatter is default, you don't actually have to do this.
+    log.SetFormatter(&log.TextFormatter{})
+  }
+}
+```
+
+This configuration is how `logrus` was intended to be used, but JSON in
+production is mostly only useful if you do log aggregation with tools like
+Splunk or Logstash.
+
+#### Formatters
+
+The built-in logging formatters are:
+
+* `logrus.TextFormatter`. Logs the event in colors if stdout is a tty, otherwise
+  without colors.
+  * *Note:* to force colored output when there is no TTY, set the `ForceColors`
+    field to `true`.  To force no colored output even if there is a TTY  set the
+    `DisableColors` field to `true`. For Windows, see
+    [github.com/mattn/go-colorable](https://github.com/mattn/go-colorable).
+  * When colors are enabled, levels are truncated to 4 characters by default. To disable
+    truncation set the `DisableLevelTruncation` field to `true`.
+  * All options are listed in the [generated docs](https://godoc.org/github.com/sirupsen/logrus#TextFormatter).
+* `logrus.JSONFormatter`. Logs fields as JSON.
+  * All options are listed in the [generated docs](https://godoc.org/github.com/sirupsen/logrus#JSONFormatter).
+
+Third party logging formatters:
+
+* [`FluentdFormatter`](https://github.com/joonix/log). Formats entries that can be parsed by Kubernetes and Google Container Engine.
+* [`logstash`](https://github.com/bshuster-repo/logrus-logstash-hook). Logs fields as [Logstash](http://logstash.net) Events.
+* [`prefixed`](https://github.com/x-cray/logrus-prefixed-formatter). Displays log entry source along with alternative layout.
+* [`zalgo`](https://github.com/aybabtme/logzalgo). Invoking the P͉̫o̳̼̊w̖͈̰͎e̬͔̭͂r͚̼̹̲ ̫͓͉̳͈ō̠͕͖̚f̝͍̠ ͕̲̞͖͑Z̖̫̤̫ͪa͉̬͈̗l͖͎g̳̥o̰̥̅!̣͔̲̻͊̄ ̙̘̦̹̦.
+
+You can define your formatter by implementing the `Formatter` interface,
+requiring a `Format` method. `Format` takes an `*Entry`. `entry.Data` is a
+`Fields` type (`map[string]interface{}`) with all your fields as well as the
+default ones (see Entries section above):
+
+```go
+type MyJSONFormatter struct {
+}
+
+log.SetFormatter(new(MyJSONFormatter))
+
+func (f *MyJSONFormatter) Format(entry *Entry) ([]byte, error) {
+  // Note this doesn't include Time, Level and Message which are available on
+  // the Entry. Consult `godoc` on information about those fields or read the
+  // source of the official loggers.
+  serialized, err := json.Marshal(entry.Data)
+    if err != nil {
+      return nil, fmt.Errorf("Failed to marshal fields to JSON, %v", err)
+    }
+  return append(serialized, '\n'), nil
+}
+```
+
+#### Logger as an `io.Writer`
+
+Logrus can be transformed into an `io.Writer`. That writer is the end of an `io.Pipe` and it is your responsibility to close it.
+
+```go
+w := logger.Writer()
+defer w.Close()
+
+srv := http.Server{
+    // create a stdlib log.Logger that writes to
+    // logrus.Logger.
+    ErrorLog: log.New(w, "", 0),
+}
+```
+
+Each line written to that writer will be printed the usual way, using formatters
+and hooks. The level for those entries is `info`.
+
+This means that we can override the standard library logger easily:
+
+```go
+logger := logrus.New()
+logger.Formatter = &logrus.JSONFormatter{}
+
+// Use logrus for standard log output
+// Note that `log` here references stdlib's log
+// Not logrus imported under the name `log`.
+log.SetOutput(logger.Writer())
+```
+
+#### Rotation
+
+Log rotation is not provided with Logrus. Log rotation should be done by an
+external program (like `logrotate(8)`) that can compress and delete old log
+entries. It should not be a feature of the application-level logger.
+
+#### Tools
+
+| Tool | Description |
+| ---- | ----------- |
+|[Logrus Mate](https://github.com/gogap/logrus_mate)|Logrus mate is a tool for Logrus to manage loggers, you can initial logger's level, hook and formatter by config file, the logger will generated with different config at different environment.|
+|[Logrus Viper Helper](https://github.com/heirko/go-contrib/tree/master/logrusHelper)|An Helper around Logrus to wrap with spf13/Viper to load configuration with fangs! And to simplify Logrus configuration use some behavior of [Logrus Mate](https://github.com/gogap/logrus_mate). [sample](https://github.com/heirko/iris-contrib/blob/master/middleware/logrus-logger/example) |
+
+#### Testing
+
+Logrus has a built in facility for asserting the presence of log messages. This is implemented through the `test` hook and provides:
+
+* decorators for existing logger (`test.NewLocal` and `test.NewGlobal`) which basically just add the `test` hook
+* a test logger (`test.NewNullLogger`) that just records log messages (and does not output any):
+
+```go
+import(
+  "github.com/sirupsen/logrus"
+  "github.com/sirupsen/logrus/hooks/test"
+  "github.com/stretchr/testify/assert"
+  "testing"
+)
+
+func TestSomething(t*testing.T){
+  logger, hook := test.NewNullLogger()
+  logger.Error("Helloerror")
+
+  assert.Equal(t, 1, len(hook.Entries))
+  assert.Equal(t, logrus.ErrorLevel, hook.LastEntry().Level)
+  assert.Equal(t, "Helloerror", hook.LastEntry().Message)
+
+  hook.Reset()
+  assert.Nil(t, hook.LastEntry())
+}
+```
+
+#### Fatal handlers
+
+Logrus can register one or more functions that will be called when any `fatal`
+level message is logged. The registered handlers will be executed before
+logrus performs a `os.Exit(1)`. This behavior may be helpful if callers need
+to gracefully shutdown. Unlike a `panic("Something went wrong...")` call which can be intercepted with a deferred `recover` a call to `os.Exit(1)` can not be intercepted.
+
+```
+...
+handler := func() {
+  // gracefully shutdown something...
+}
+logrus.RegisterExitHandler(handler)
+...
+```
+
+#### Thread safety
+
+By default, Logger is protected by a mutex for concurrent writes. The mutex is held when calling hooks and writing logs.
+If you are sure such locking is not needed, you can call logger.SetNoLock() to disable the locking.
+
+Situation when locking is not needed includes:
+
+* You have no hooks registered, or hooks calling is already thread-safe.
+
+* Writing to logger.Out is already thread-safe, for example:
+
+  1) logger.Out is protected by locks.
+
+  2) logger.Out is a os.File handler opened with `O_APPEND` flag, and every write is smaller than 4k. (This allow multi-thread/multi-process writing)
+
+     (Refer to http://www.notthewizard.com/2014/06/17/are-files-appends-really-atomic/)
diff --git a/vendor/github.com/sirupsen/logrus/alt_exit.go b/vendor/github.com/sirupsen/logrus/alt_exit.go
new file mode 100644
index 0000000..8af9063
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/alt_exit.go
@@ -0,0 +1,64 @@
+package logrus
+
+// The following code was sourced and modified from the
+// https://github.com/tebeka/atexit package governed by the following license:
+//
+// Copyright (c) 2012 Miki Tebeka <miki.tebeka@gmail.com>.
+//
+// Permission is hereby granted, free of charge, to any person obtaining a copy of
+// this software and associated documentation files (the "Software"), to deal in
+// the Software without restriction, including without limitation the rights to
+// use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
+// the Software, and to permit persons to whom the Software is furnished to do so,
+// subject to the following conditions:
+//
+// The above copyright notice and this permission notice shall be included in all
+// copies or substantial portions of the Software.
+//
+// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
+// FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
+// COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
+// IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
+// CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+import (
+	"fmt"
+	"os"
+)
+
+var handlers = []func(){}
+
+func runHandler(handler func()) {
+	defer func() {
+		if err := recover(); err != nil {
+			fmt.Fprintln(os.Stderr, "Error: Logrus exit handler error:", err)
+		}
+	}()
+
+	handler()
+}
+
+func runHandlers() {
+	for _, handler := range handlers {
+		runHandler(handler)
+	}
+}
+
+// Exit runs all the Logrus atexit handlers and then terminates the program using os.Exit(code)
+func Exit(code int) {
+	runHandlers()
+	os.Exit(code)
+}
+
+// RegisterExitHandler adds a Logrus Exit handler, call logrus.Exit to invoke
+// all handlers. The handlers will also be invoked when any Fatal log entry is
+// made.
+//
+// This method is useful when a caller wishes to use logrus to log a fatal
+// message but also needs to gracefully shutdown. An example usecase could be
+// closing database connections, or sending a alert that the application is
+// closing.
+func RegisterExitHandler(handler func()) {
+	handlers = append(handlers, handler)
+}
diff --git a/vendor/github.com/sirupsen/logrus/appveyor.yml b/vendor/github.com/sirupsen/logrus/appveyor.yml
new file mode 100644
index 0000000..96c2ce1
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/appveyor.yml
@@ -0,0 +1,14 @@
+version: "{build}"

+platform: x64

+clone_folder: c:\gopath\src\github.com\sirupsen\logrus

+environment:  

+  GOPATH: c:\gopath

+branches:  

+  only:

+    - master

+install:  

+  - set PATH=%GOPATH%\bin;c:\go\bin;%PATH%

+  - go version

+build_script:  

+  - go get -t

+  - go test

diff --git a/vendor/github.com/sirupsen/logrus/doc.go b/vendor/github.com/sirupsen/logrus/doc.go
new file mode 100644
index 0000000..da67aba
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/doc.go
@@ -0,0 +1,26 @@
+/*
+Package logrus is a structured logger for Go, completely API compatible with the standard library logger.
+
+
+The simplest way to use Logrus is simply the package-level exported logger:
+
+  package main
+
+  import (
+    log "github.com/sirupsen/logrus"
+  )
+
+  func main() {
+    log.WithFields(log.Fields{
+      "animal": "walrus",
+      "number": 1,
+      "size":   10,
+    }).Info("A walrus appears")
+  }
+
+Output:
+  time="2015-09-07T08:48:33Z" level=info msg="A walrus appears" animal=walrus number=1 size=10
+
+For a full guide visit https://github.com/sirupsen/logrus
+*/
+package logrus
diff --git a/vendor/github.com/sirupsen/logrus/entry.go b/vendor/github.com/sirupsen/logrus/entry.go
new file mode 100644
index 0000000..cc85d3a
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/entry.go
@@ -0,0 +1,408 @@
+package logrus
+
+import (
+	"bytes"
+	"fmt"
+	"os"
+	"reflect"
+	"runtime"
+	"strings"
+	"sync"
+	"time"
+)
+
+var (
+	bufferPool *sync.Pool
+
+	// qualified package name, cached at first use
+	logrusPackage string
+
+	// Positions in the call stack when tracing to report the calling method
+	minimumCallerDepth int
+
+	// Used for caller information initialisation
+	callerInitOnce sync.Once
+)
+
+const (
+	maximumCallerDepth int = 25
+	knownLogrusFrames  int = 4
+)
+
+func init() {
+	bufferPool = &sync.Pool{
+		New: func() interface{} {
+			return new(bytes.Buffer)
+		},
+	}
+
+	// start at the bottom of the stack before the package-name cache is primed
+	minimumCallerDepth = 1
+}
+
+// Defines the key when adding errors using WithError.
+var ErrorKey = "error"
+
+// An entry is the final or intermediate Logrus logging entry. It contains all
+// the fields passed with WithField{,s}. It's finally logged when Trace, Debug,
+// Info, Warn, Error, Fatal or Panic is called on it. These objects can be
+// reused and passed around as much as you wish to avoid field duplication.
+type Entry struct {
+	Logger *Logger
+
+	// Contains all the fields set by the user.
+	Data Fields
+
+	// Time at which the log entry was created
+	Time time.Time
+
+	// Level the log entry was logged at: Trace, Debug, Info, Warn, Error, Fatal or Panic
+	// This field will be set on entry firing and the value will be equal to the one in Logger struct field.
+	Level Level
+
+	// Calling method, with package name
+	Caller *runtime.Frame
+
+	// Message passed to Trace, Debug, Info, Warn, Error, Fatal or Panic
+	Message string
+
+	// When formatter is called in entry.log(), a Buffer may be set to entry
+	Buffer *bytes.Buffer
+
+	// err may contain a field formatting error
+	err string
+}
+
+func NewEntry(logger *Logger) *Entry {
+	return &Entry{
+		Logger: logger,
+		// Default is three fields, plus one optional.  Give a little extra room.
+		Data: make(Fields, 6),
+	}
+}
+
+// Returns the string representation from the reader and ultimately the
+// formatter.
+func (entry *Entry) String() (string, error) {
+	serialized, err := entry.Logger.Formatter.Format(entry)
+	if err != nil {
+		return "", err
+	}
+	str := string(serialized)
+	return str, nil
+}
+
+// Add an error as single field (using the key defined in ErrorKey) to the Entry.
+func (entry *Entry) WithError(err error) *Entry {
+	return entry.WithField(ErrorKey, err)
+}
+
+// Add a single field to the Entry.
+func (entry *Entry) WithField(key string, value interface{}) *Entry {
+	return entry.WithFields(Fields{key: value})
+}
+
+// Add a map of fields to the Entry.
+func (entry *Entry) WithFields(fields Fields) *Entry {
+	data := make(Fields, len(entry.Data)+len(fields))
+	for k, v := range entry.Data {
+		data[k] = v
+	}
+	var field_err string
+	for k, v := range fields {
+		if t := reflect.TypeOf(v); t != nil && t.Kind() == reflect.Func {
+			field_err = fmt.Sprintf("can not add field %q", k)
+			if entry.err != "" {
+				field_err = entry.err + ", " + field_err
+			}
+		} else {
+			data[k] = v
+		}
+	}
+	return &Entry{Logger: entry.Logger, Data: data, Time: entry.Time, err: field_err}
+}
+
+// Overrides the time of the Entry.
+func (entry *Entry) WithTime(t time.Time) *Entry {
+	return &Entry{Logger: entry.Logger, Data: entry.Data, Time: t}
+}
+
+// getPackageName reduces a fully qualified function name to the package name
+// There really ought to be to be a better way...
+func getPackageName(f string) string {
+	for {
+		lastPeriod := strings.LastIndex(f, ".")
+		lastSlash := strings.LastIndex(f, "/")
+		if lastPeriod > lastSlash {
+			f = f[:lastPeriod]
+		} else {
+			break
+		}
+	}
+
+	return f
+}
+
+// getCaller retrieves the name of the first non-logrus calling function
+func getCaller() *runtime.Frame {
+	// Restrict the lookback frames to avoid runaway lookups
+	pcs := make([]uintptr, maximumCallerDepth)
+	depth := runtime.Callers(minimumCallerDepth, pcs)
+	frames := runtime.CallersFrames(pcs[:depth])
+
+	// cache this package's fully-qualified name
+	callerInitOnce.Do(func() {
+		logrusPackage = getPackageName(runtime.FuncForPC(pcs[0]).Name())
+
+		// now that we have the cache, we can skip a minimum count of known-logrus functions
+		// XXX this is dubious, the number of frames may vary store an entry in a logger interface
+		minimumCallerDepth = knownLogrusFrames
+	})
+
+	for f, again := frames.Next(); again; f, again = frames.Next() {
+		pkg := getPackageName(f.Function)
+
+		// If the caller isn't part of this package, we're done
+		if pkg != logrusPackage {
+			return &f
+		}
+	}
+
+	// if we got here, we failed to find the caller's context
+	return nil
+}
+
+func (entry Entry) HasCaller() (has bool) {
+	return entry.Logger != nil &&
+		entry.Logger.ReportCaller &&
+		entry.Caller != nil
+}
+
+// This function is not declared with a pointer value because otherwise
+// race conditions will occur when using multiple goroutines
+func (entry Entry) log(level Level, msg string) {
+	var buffer *bytes.Buffer
+
+	// Default to now, but allow users to override if they want.
+	//
+	// We don't have to worry about polluting future calls to Entry#log()
+	// with this assignment because this function is declared with a
+	// non-pointer receiver.
+	if entry.Time.IsZero() {
+		entry.Time = time.Now()
+	}
+
+	entry.Level = level
+	entry.Message = msg
+	if entry.Logger.ReportCaller {
+		entry.Caller = getCaller()
+	}
+
+	entry.fireHooks()
+
+	buffer = bufferPool.Get().(*bytes.Buffer)
+	buffer.Reset()
+	defer bufferPool.Put(buffer)
+	entry.Buffer = buffer
+
+	entry.write()
+
+	entry.Buffer = nil
+
+	// To avoid Entry#log() returning a value that only would make sense for
+	// panic() to use in Entry#Panic(), we avoid the allocation by checking
+	// directly here.
+	if level <= PanicLevel {
+		panic(&entry)
+	}
+}
+
+func (entry *Entry) fireHooks() {
+	entry.Logger.mu.Lock()
+	defer entry.Logger.mu.Unlock()
+	err := entry.Logger.Hooks.Fire(entry.Level, entry)
+	if err != nil {
+		fmt.Fprintf(os.Stderr, "Failed to fire hook: %v\n", err)
+	}
+}
+
+func (entry *Entry) write() {
+	entry.Logger.mu.Lock()
+	defer entry.Logger.mu.Unlock()
+	serialized, err := entry.Logger.Formatter.Format(entry)
+	if err != nil {
+		fmt.Fprintf(os.Stderr, "Failed to obtain reader, %v\n", err)
+	} else {
+		_, err = entry.Logger.Out.Write(serialized)
+		if err != nil {
+			fmt.Fprintf(os.Stderr, "Failed to write to log, %v\n", err)
+		}
+	}
+}
+
+func (entry *Entry) Trace(args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(TraceLevel) {
+		entry.log(TraceLevel, fmt.Sprint(args...))
+	}
+}
+
+func (entry *Entry) Debug(args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(DebugLevel) {
+		entry.log(DebugLevel, fmt.Sprint(args...))
+	}
+}
+
+func (entry *Entry) Print(args ...interface{}) {
+	entry.Info(args...)
+}
+
+func (entry *Entry) Info(args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(InfoLevel) {
+		entry.log(InfoLevel, fmt.Sprint(args...))
+	}
+}
+
+func (entry *Entry) Warn(args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(WarnLevel) {
+		entry.log(WarnLevel, fmt.Sprint(args...))
+	}
+}
+
+func (entry *Entry) Warning(args ...interface{}) {
+	entry.Warn(args...)
+}
+
+func (entry *Entry) Error(args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(ErrorLevel) {
+		entry.log(ErrorLevel, fmt.Sprint(args...))
+	}
+}
+
+func (entry *Entry) Fatal(args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(FatalLevel) {
+		entry.log(FatalLevel, fmt.Sprint(args...))
+	}
+	entry.Logger.Exit(1)
+}
+
+func (entry *Entry) Panic(args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(PanicLevel) {
+		entry.log(PanicLevel, fmt.Sprint(args...))
+	}
+	panic(fmt.Sprint(args...))
+}
+
+// Entry Printf family functions
+
+func (entry *Entry) Tracef(format string, args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(TraceLevel) {
+		entry.Trace(fmt.Sprintf(format, args...))
+	}
+}
+
+func (entry *Entry) Debugf(format string, args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(DebugLevel) {
+		entry.Debug(fmt.Sprintf(format, args...))
+	}
+}
+
+func (entry *Entry) Infof(format string, args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(InfoLevel) {
+		entry.Info(fmt.Sprintf(format, args...))
+	}
+}
+
+func (entry *Entry) Printf(format string, args ...interface{}) {
+	entry.Infof(format, args...)
+}
+
+func (entry *Entry) Warnf(format string, args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(WarnLevel) {
+		entry.Warn(fmt.Sprintf(format, args...))
+	}
+}
+
+func (entry *Entry) Warningf(format string, args ...interface{}) {
+	entry.Warnf(format, args...)
+}
+
+func (entry *Entry) Errorf(format string, args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(ErrorLevel) {
+		entry.Error(fmt.Sprintf(format, args...))
+	}
+}
+
+func (entry *Entry) Fatalf(format string, args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(FatalLevel) {
+		entry.Fatal(fmt.Sprintf(format, args...))
+	}
+	entry.Logger.Exit(1)
+}
+
+func (entry *Entry) Panicf(format string, args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(PanicLevel) {
+		entry.Panic(fmt.Sprintf(format, args...))
+	}
+}
+
+// Entry Println family functions
+
+func (entry *Entry) Traceln(args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(TraceLevel) {
+		entry.Trace(entry.sprintlnn(args...))
+	}
+}
+
+func (entry *Entry) Debugln(args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(DebugLevel) {
+		entry.Debug(entry.sprintlnn(args...))
+	}
+}
+
+func (entry *Entry) Infoln(args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(InfoLevel) {
+		entry.Info(entry.sprintlnn(args...))
+	}
+}
+
+func (entry *Entry) Println(args ...interface{}) {
+	entry.Infoln(args...)
+}
+
+func (entry *Entry) Warnln(args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(WarnLevel) {
+		entry.Warn(entry.sprintlnn(args...))
+	}
+}
+
+func (entry *Entry) Warningln(args ...interface{}) {
+	entry.Warnln(args...)
+}
+
+func (entry *Entry) Errorln(args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(ErrorLevel) {
+		entry.Error(entry.sprintlnn(args...))
+	}
+}
+
+func (entry *Entry) Fatalln(args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(FatalLevel) {
+		entry.Fatal(entry.sprintlnn(args...))
+	}
+	entry.Logger.Exit(1)
+}
+
+func (entry *Entry) Panicln(args ...interface{}) {
+	if entry.Logger.IsLevelEnabled(PanicLevel) {
+		entry.Panic(entry.sprintlnn(args...))
+	}
+}
+
+// Sprintlnn => Sprint no newline. This is to get the behavior of how
+// fmt.Sprintln where spaces are always added between operands, regardless of
+// their type. Instead of vendoring the Sprintln implementation to spare a
+// string allocation, we do the simplest thing.
+func (entry *Entry) sprintlnn(args ...interface{}) string {
+	msg := fmt.Sprintln(args...)
+	return msg[:len(msg)-1]
+}
diff --git a/vendor/github.com/sirupsen/logrus/exported.go b/vendor/github.com/sirupsen/logrus/exported.go
new file mode 100644
index 0000000..7342613
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/exported.go
@@ -0,0 +1,219 @@
+package logrus
+
+import (
+	"io"
+	"time"
+)
+
+var (
+	// std is the name of the standard logger in stdlib `log`
+	std = New()
+)
+
+func StandardLogger() *Logger {
+	return std
+}
+
+// SetOutput sets the standard logger output.
+func SetOutput(out io.Writer) {
+	std.SetOutput(out)
+}
+
+// SetFormatter sets the standard logger formatter.
+func SetFormatter(formatter Formatter) {
+	std.SetFormatter(formatter)
+}
+
+// SetReportCaller sets whether the standard logger will include the calling
+// method as a field.
+func SetReportCaller(include bool) {
+	std.SetReportCaller(include)
+}
+
+// SetLevel sets the standard logger level.
+func SetLevel(level Level) {
+	std.SetLevel(level)
+}
+
+// GetLevel returns the standard logger level.
+func GetLevel() Level {
+	return std.GetLevel()
+}
+
+// IsLevelEnabled checks if the log level of the standard logger is greater than the level param
+func IsLevelEnabled(level Level) bool {
+	return std.IsLevelEnabled(level)
+}
+
+// AddHook adds a hook to the standard logger hooks.
+func AddHook(hook Hook) {
+	std.AddHook(hook)
+}
+
+// WithError creates an entry from the standard logger and adds an error to it, using the value defined in ErrorKey as key.
+func WithError(err error) *Entry {
+	return std.WithField(ErrorKey, err)
+}
+
+// WithField creates an entry from the standard logger and adds a field to
+// it. If you want multiple fields, use `WithFields`.
+//
+// Note that it doesn't log until you call Debug, Print, Info, Warn, Fatal
+// or Panic on the Entry it returns.
+func WithField(key string, value interface{}) *Entry {
+	return std.WithField(key, value)
+}
+
+// WithFields creates an entry from the standard logger and adds multiple
+// fields to it. This is simply a helper for `WithField`, invoking it
+// once for each field.
+//
+// Note that it doesn't log until you call Debug, Print, Info, Warn, Fatal
+// or Panic on the Entry it returns.
+func WithFields(fields Fields) *Entry {
+	return std.WithFields(fields)
+}
+
+// WithTime creats an entry from the standard logger and overrides the time of
+// logs generated with it.
+//
+// Note that it doesn't log until you call Debug, Print, Info, Warn, Fatal
+// or Panic on the Entry it returns.
+func WithTime(t time.Time) *Entry {
+	return std.WithTime(t)
+}
+
+// Trace logs a message at level Trace on the standard logger.
+func Trace(args ...interface{}) {
+	std.Trace(args...)
+}
+
+// Debug logs a message at level Debug on the standard logger.
+func Debug(args ...interface{}) {
+	std.Debug(args...)
+}
+
+// Print logs a message at level Info on the standard logger.
+func Print(args ...interface{}) {
+	std.Print(args...)
+}
+
+// Info logs a message at level Info on the standard logger.
+func Info(args ...interface{}) {
+	std.Info(args...)
+}
+
+// Warn logs a message at level Warn on the standard logger.
+func Warn(args ...interface{}) {
+	std.Warn(args...)
+}
+
+// Warning logs a message at level Warn on the standard logger.
+func Warning(args ...interface{}) {
+	std.Warning(args...)
+}
+
+// Error logs a message at level Error on the standard logger.
+func Error(args ...interface{}) {
+	std.Error(args...)
+}
+
+// Panic logs a message at level Panic on the standard logger.
+func Panic(args ...interface{}) {
+	std.Panic(args...)
+}
+
+// Fatal logs a message at level Fatal on the standard logger then the process will exit with status set to 1.
+func Fatal(args ...interface{}) {
+	std.Fatal(args...)
+}
+
+// Tracef logs a message at level Trace on the standard logger.
+func Tracef(format string, args ...interface{}) {
+	std.Tracef(format, args...)
+}
+
+// Debugf logs a message at level Debug on the standard logger.
+func Debugf(format string, args ...interface{}) {
+	std.Debugf(format, args...)
+}
+
+// Printf logs a message at level Info on the standard logger.
+func Printf(format string, args ...interface{}) {
+	std.Printf(format, args...)
+}
+
+// Infof logs a message at level Info on the standard logger.
+func Infof(format string, args ...interface{}) {
+	std.Infof(format, args...)
+}
+
+// Warnf logs a message at level Warn on the standard logger.
+func Warnf(format string, args ...interface{}) {
+	std.Warnf(format, args...)
+}
+
+// Warningf logs a message at level Warn on the standard logger.
+func Warningf(format string, args ...interface{}) {
+	std.Warningf(format, args...)
+}
+
+// Errorf logs a message at level Error on the standard logger.
+func Errorf(format string, args ...interface{}) {
+	std.Errorf(format, args...)
+}
+
+// Panicf logs a message at level Panic on the standard logger.
+func Panicf(format string, args ...interface{}) {
+	std.Panicf(format, args...)
+}
+
+// Fatalf logs a message at level Fatal on the standard logger then the process will exit with status set to 1.
+func Fatalf(format string, args ...interface{}) {
+	std.Fatalf(format, args...)
+}
+
+// Traceln logs a message at level Trace on the standard logger.
+func Traceln(args ...interface{}) {
+	std.Traceln(args...)
+}
+
+// Debugln logs a message at level Debug on the standard logger.
+func Debugln(args ...interface{}) {
+	std.Debugln(args...)
+}
+
+// Println logs a message at level Info on the standard logger.
+func Println(args ...interface{}) {
+	std.Println(args...)
+}
+
+// Infoln logs a message at level Info on the standard logger.
+func Infoln(args ...interface{}) {
+	std.Infoln(args...)
+}
+
+// Warnln logs a message at level Warn on the standard logger.
+func Warnln(args ...interface{}) {
+	std.Warnln(args...)
+}
+
+// Warningln logs a message at level Warn on the standard logger.
+func Warningln(args ...interface{}) {
+	std.Warningln(args...)
+}
+
+// Errorln logs a message at level Error on the standard logger.
+func Errorln(args ...interface{}) {
+	std.Errorln(args...)
+}
+
+// Panicln logs a message at level Panic on the standard logger.
+func Panicln(args ...interface{}) {
+	std.Panicln(args...)
+}
+
+// Fatalln logs a message at level Fatal on the standard logger then the process will exit with status set to 1.
+func Fatalln(args ...interface{}) {
+	std.Fatalln(args...)
+}
diff --git a/vendor/github.com/sirupsen/logrus/formatter.go b/vendor/github.com/sirupsen/logrus/formatter.go
new file mode 100644
index 0000000..4088837
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/formatter.go
@@ -0,0 +1,78 @@
+package logrus
+
+import "time"
+
+// Default key names for the default fields
+const (
+	defaultTimestampFormat = time.RFC3339
+	FieldKeyMsg            = "msg"
+	FieldKeyLevel          = "level"
+	FieldKeyTime           = "time"
+	FieldKeyLogrusError    = "logrus_error"
+	FieldKeyFunc           = "func"
+	FieldKeyFile           = "file"
+)
+
+// The Formatter interface is used to implement a custom Formatter. It takes an
+// `Entry`. It exposes all the fields, including the default ones:
+//
+// * `entry.Data["msg"]`. The message passed from Info, Warn, Error ..
+// * `entry.Data["time"]`. The timestamp.
+// * `entry.Data["level"]. The level the entry was logged at.
+//
+// Any additional fields added with `WithField` or `WithFields` are also in
+// `entry.Data`. Format is expected to return an array of bytes which are then
+// logged to `logger.Out`.
+type Formatter interface {
+	Format(*Entry) ([]byte, error)
+}
+
+// This is to not silently overwrite `time`, `msg`, `func` and `level` fields when
+// dumping it. If this code wasn't there doing:
+//
+//  logrus.WithField("level", 1).Info("hello")
+//
+// Would just silently drop the user provided level. Instead with this code
+// it'll logged as:
+//
+//  {"level": "info", "fields.level": 1, "msg": "hello", "time": "..."}
+//
+// It's not exported because it's still using Data in an opinionated way. It's to
+// avoid code duplication between the two default formatters.
+func prefixFieldClashes(data Fields, fieldMap FieldMap, reportCaller bool) {
+	timeKey := fieldMap.resolve(FieldKeyTime)
+	if t, ok := data[timeKey]; ok {
+		data["fields."+timeKey] = t
+		delete(data, timeKey)
+	}
+
+	msgKey := fieldMap.resolve(FieldKeyMsg)
+	if m, ok := data[msgKey]; ok {
+		data["fields."+msgKey] = m
+		delete(data, msgKey)
+	}
+
+	levelKey := fieldMap.resolve(FieldKeyLevel)
+	if l, ok := data[levelKey]; ok {
+		data["fields."+levelKey] = l
+		delete(data, levelKey)
+	}
+
+	logrusErrKey := fieldMap.resolve(FieldKeyLogrusError)
+	if l, ok := data[logrusErrKey]; ok {
+		data["fields."+logrusErrKey] = l
+		delete(data, logrusErrKey)
+	}
+
+	// If reportCaller is not set, 'func' will not conflict.
+	if reportCaller {
+		funcKey := fieldMap.resolve(FieldKeyFunc)
+		if l, ok := data[funcKey]; ok {
+			data["fields."+funcKey] = l
+		}
+		fileKey := fieldMap.resolve(FieldKeyFile)
+		if l, ok := data[fileKey]; ok {
+			data["fields."+fileKey] = l
+		}
+	}
+}
diff --git a/vendor/github.com/sirupsen/logrus/go.mod b/vendor/github.com/sirupsen/logrus/go.mod
new file mode 100644
index 0000000..94574cc
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/go.mod
@@ -0,0 +1,11 @@
+module github.com/sirupsen/logrus
+
+require (
+	github.com/davecgh/go-spew v1.1.1 // indirect
+	github.com/konsorten/go-windows-terminal-sequences v1.0.1
+	github.com/pmezard/go-difflib v1.0.0 // indirect
+	github.com/stretchr/objx v0.1.1 // indirect
+	github.com/stretchr/testify v1.2.2
+	golang.org/x/crypto v0.0.0-20180904163835-0709b304e793
+	golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33
+)
diff --git a/vendor/github.com/sirupsen/logrus/go.sum b/vendor/github.com/sirupsen/logrus/go.sum
new file mode 100644
index 0000000..133d34a
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/go.sum
@@ -0,0 +1,15 @@
+github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
+github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
+github.com/konsorten/go-windows-terminal-sequences v0.0.0-20180402223658-b729f2633dfe h1:CHRGQ8V7OlCYtwaKPJi3iA7J+YdNKdo8j7nG5IgDhjs=
+github.com/konsorten/go-windows-terminal-sequences v0.0.0-20180402223658-b729f2633dfe/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
+github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
+github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM=
+github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
+github.com/stretchr/objx v0.1.1 h1:2vfRuCMp5sSVIDSqO8oNnWJq7mPa6KVP3iPIwFBuy8A=
+github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
+github.com/stretchr/testify v1.2.2 h1:bSDNvY7ZPG5RlJ8otE/7V6gMiyenm9RtJ7IUVIAoJ1w=
+github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs=
+golang.org/x/crypto v0.0.0-20180904163835-0709b304e793 h1:u+LnwYTOOW7Ukr/fppxEb1Nwz0AtPflrblfvUudpo+I=
+golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
+golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33 h1:I6FyU15t786LL7oL/hn43zqTuEGr4PN7F4XJ1p4E3Y8=
+golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
diff --git a/vendor/github.com/sirupsen/logrus/hooks.go b/vendor/github.com/sirupsen/logrus/hooks.go
new file mode 100644
index 0000000..3f151cd
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/hooks.go
@@ -0,0 +1,34 @@
+package logrus
+
+// A hook to be fired when logging on the logging levels returned from
+// `Levels()` on your implementation of the interface. Note that this is not
+// fired in a goroutine or a channel with workers, you should handle such
+// functionality yourself if your call is non-blocking and you don't wish for
+// the logging calls for levels returned from `Levels()` to block.
+type Hook interface {
+	Levels() []Level
+	Fire(*Entry) error
+}
+
+// Internal type for storing the hooks on a logger instance.
+type LevelHooks map[Level][]Hook
+
+// Add a hook to an instance of logger. This is called with
+// `log.Hooks.Add(new(MyHook))` where `MyHook` implements the `Hook` interface.
+func (hooks LevelHooks) Add(hook Hook) {
+	for _, level := range hook.Levels() {
+		hooks[level] = append(hooks[level], hook)
+	}
+}
+
+// Fire all the hooks for the passed level. Used by `entry.log` to fire
+// appropriate hooks for a log entry.
+func (hooks LevelHooks) Fire(level Level, entry *Entry) error {
+	for _, hook := range hooks[level] {
+		if err := hook.Fire(entry); err != nil {
+			return err
+		}
+	}
+
+	return nil
+}
diff --git a/vendor/github.com/sirupsen/logrus/json_formatter.go b/vendor/github.com/sirupsen/logrus/json_formatter.go
new file mode 100644
index 0000000..2605753
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/json_formatter.go
@@ -0,0 +1,105 @@
+package logrus
+
+import (
+	"bytes"
+	"encoding/json"
+	"fmt"
+)
+
+type fieldKey string
+
+// FieldMap allows customization of the key names for default fields.
+type FieldMap map[fieldKey]string
+
+func (f FieldMap) resolve(key fieldKey) string {
+	if k, ok := f[key]; ok {
+		return k
+	}
+
+	return string(key)
+}
+
+// JSONFormatter formats logs into parsable json
+type JSONFormatter struct {
+	// TimestampFormat sets the format used for marshaling timestamps.
+	TimestampFormat string
+
+	// DisableTimestamp allows disabling automatic timestamps in output
+	DisableTimestamp bool
+
+	// DataKey allows users to put all the log entry parameters into a nested dictionary at a given key.
+	DataKey string
+
+	// FieldMap allows users to customize the names of keys for default fields.
+	// As an example:
+	// formatter := &JSONFormatter{
+	//   	FieldMap: FieldMap{
+	// 		 FieldKeyTime:  "@timestamp",
+	// 		 FieldKeyLevel: "@level",
+	// 		 FieldKeyMsg:   "@message",
+	// 		 FieldKeyFunc:  "@caller",
+	//    },
+	// }
+	FieldMap FieldMap
+
+	// PrettyPrint will indent all json logs
+	PrettyPrint bool
+}
+
+// Format renders a single log entry
+func (f *JSONFormatter) Format(entry *Entry) ([]byte, error) {
+	data := make(Fields, len(entry.Data)+4)
+	for k, v := range entry.Data {
+		switch v := v.(type) {
+		case error:
+			// Otherwise errors are ignored by `encoding/json`
+			// https://github.com/sirupsen/logrus/issues/137
+			data[k] = v.Error()
+		default:
+			data[k] = v
+		}
+	}
+
+	if f.DataKey != "" {
+		newData := make(Fields, 4)
+		newData[f.DataKey] = data
+		data = newData
+	}
+
+	prefixFieldClashes(data, f.FieldMap, entry.HasCaller())
+
+	timestampFormat := f.TimestampFormat
+	if timestampFormat == "" {
+		timestampFormat = defaultTimestampFormat
+	}
+
+	if entry.err != "" {
+		data[f.FieldMap.resolve(FieldKeyLogrusError)] = entry.err
+	}
+	if !f.DisableTimestamp {
+		data[f.FieldMap.resolve(FieldKeyTime)] = entry.Time.Format(timestampFormat)
+	}
+	data[f.FieldMap.resolve(FieldKeyMsg)] = entry.Message
+	data[f.FieldMap.resolve(FieldKeyLevel)] = entry.Level.String()
+	if entry.HasCaller() {
+		data[f.FieldMap.resolve(FieldKeyFunc)] = entry.Caller.Function
+		data[f.FieldMap.resolve(FieldKeyFile)] = fmt.Sprintf("%s:%d", entry.Caller.File, entry.Caller.Line)
+	}
+
+	var b *bytes.Buffer
+	if entry.Buffer != nil {
+		b = entry.Buffer
+	} else {
+		b = &bytes.Buffer{}
+	}
+
+	encoder := json.NewEncoder(b)
+	if f.PrettyPrint {
+		encoder.SetIndent("", "  ")
+	}
+	if err := encoder.Encode(data); err != nil {
+		return nil, fmt.Errorf("Failed to marshal fields to JSON, %v", err)
+	}
+
+	return b.Bytes(), nil
+}
diff --git a/vendor/github.com/sirupsen/logrus/logger.go b/vendor/github.com/sirupsen/logrus/logger.go
new file mode 100644
index 0000000..5ceca0e
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/logger.go
@@ -0,0 +1,415 @@
+package logrus
+
+import (
+	"io"
+	"os"
+	"sync"
+	"sync/atomic"
+	"time"
+)
+
+type Logger struct {
+	// The logs are `io.Copy`'d to this in a mutex. It's common to set this to a
+	// file, or leave it default which is `os.Stderr`. You can also set this to
+	// something more adventurous, such as logging to Kafka.
+	Out io.Writer
+	// Hooks for the logger instance. These allow firing events based on logging
+	// levels and log entries. For example, to send errors to an error tracking
+	// service, log to StatsD or dump the core on fatal errors.
+	Hooks LevelHooks
+	// All log entries pass through the formatter before logged to Out. The
+	// included formatters are `TextFormatter` and `JSONFormatter` for which
+	// TextFormatter is the default. In development (when a TTY is attached) it
+	// logs with colors, but to a file it wouldn't. You can easily implement your
+	// own that implements the `Formatter` interface, see the `README` or included
+	// formatters for examples.
+	Formatter Formatter
+
+	// Flag for whether to log caller info (off by default)
+	ReportCaller bool
+
+	// The logging level the logger should log at. This is typically (and defaults
+	// to) `logrus.Info`, which allows Info(), Warn(), Error() and Fatal() to be
+	// logged.
+	Level Level
+	// Used to sync writing to the log. Locking is enabled by Default
+	mu MutexWrap
+	// Reusable empty entry
+	entryPool sync.Pool
+	// Function to exit the application, defaults to `os.Exit()`
+	ExitFunc exitFunc
+}
+
+type exitFunc func(int)
+
+type MutexWrap struct {
+	lock     sync.Mutex
+	disabled bool
+}
+
+func (mw *MutexWrap) Lock() {
+	if !mw.disabled {
+		mw.lock.Lock()
+	}
+}
+
+func (mw *MutexWrap) Unlock() {
+	if !mw.disabled {
+		mw.lock.Unlock()
+	}
+}
+
+func (mw *MutexWrap) Disable() {
+	mw.disabled = true
+}
+
+// Creates a new logger. Configuration should be set by changing `Formatter`,
+// `Out` and `Hooks` directly on the default logger instance. You can also just
+// instantiate your own:
+//
+//    var log = &Logger{
+//      Out: os.Stderr,
+//      Formatter: new(JSONFormatter),
+//      Hooks: make(LevelHooks),
+//      Level: logrus.DebugLevel,
+//    }
+//
+// It's recommended to make this a global instance called `log`.
+func New() *Logger {
+	return &Logger{
+		Out:          os.Stderr,
+		Formatter:    new(TextFormatter),
+		Hooks:        make(LevelHooks),
+		Level:        InfoLevel,
+		ExitFunc:     os.Exit,
+		ReportCaller: false,
+	}
+}
+
+func (logger *Logger) newEntry() *Entry {
+	entry, ok := logger.entryPool.Get().(*Entry)
+	if ok {
+		return entry
+	}
+	return NewEntry(logger)
+}
+
+func (logger *Logger) releaseEntry(entry *Entry) {
+	entry.Data = map[string]interface{}{}
+	logger.entryPool.Put(entry)
+}
+
+// Adds a field to the log entry, note that it doesn't log until you call
+// Debug, Print, Info, Warn, Error, Fatal or Panic. It only creates a log entry.
+// If you want multiple fields, use `WithFields`.
+func (logger *Logger) WithField(key string, value interface{}) *Entry {
+	entry := logger.newEntry()
+	defer logger.releaseEntry(entry)
+	return entry.WithField(key, value)
+}
+
+// Adds a struct of fields to the log entry. All it does is call `WithField` for
+// each `Field`.
+func (logger *Logger) WithFields(fields Fields) *Entry {
+	entry := logger.newEntry()
+	defer logger.releaseEntry(entry)
+	return entry.WithFields(fields)
+}
+
+// Add an error as single field to the log entry.  All it does is call
+// `WithError` for the given `error`.
+func (logger *Logger) WithError(err error) *Entry {
+	entry := logger.newEntry()
+	defer logger.releaseEntry(entry)
+	return entry.WithError(err)
+}
+
+// Overrides the time of the log entry.
+func (logger *Logger) WithTime(t time.Time) *Entry {
+	entry := logger.newEntry()
+	defer logger.releaseEntry(entry)
+	return entry.WithTime(t)
+}
+
+func (logger *Logger) Tracef(format string, args ...interface{}) {
+	if logger.IsLevelEnabled(TraceLevel) {
+		entry := logger.newEntry()
+		entry.Tracef(format, args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Debugf(format string, args ...interface{}) {
+	if logger.IsLevelEnabled(DebugLevel) {
+		entry := logger.newEntry()
+		entry.Debugf(format, args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Infof(format string, args ...interface{}) {
+	if logger.IsLevelEnabled(InfoLevel) {
+		entry := logger.newEntry()
+		entry.Infof(format, args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Printf(format string, args ...interface{}) {
+	entry := logger.newEntry()
+	entry.Printf(format, args...)
+	logger.releaseEntry(entry)
+}
+
+func (logger *Logger) Warnf(format string, args ...interface{}) {
+	if logger.IsLevelEnabled(WarnLevel) {
+		entry := logger.newEntry()
+		entry.Warnf(format, args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Warningf(format string, args ...interface{}) {
+	if logger.IsLevelEnabled(WarnLevel) {
+		entry := logger.newEntry()
+		entry.Warnf(format, args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Errorf(format string, args ...interface{}) {
+	if logger.IsLevelEnabled(ErrorLevel) {
+		entry := logger.newEntry()
+		entry.Errorf(format, args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Fatalf(format string, args ...interface{}) {
+	if logger.IsLevelEnabled(FatalLevel) {
+		entry := logger.newEntry()
+		entry.Fatalf(format, args...)
+		logger.releaseEntry(entry)
+	}
+	logger.Exit(1)
+}
+
+func (logger *Logger) Panicf(format string, args ...interface{}) {
+	if logger.IsLevelEnabled(PanicLevel) {
+		entry := logger.newEntry()
+		entry.Panicf(format, args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Trace(args ...interface{}) {
+	if logger.IsLevelEnabled(TraceLevel) {
+		entry := logger.newEntry()
+		entry.Trace(args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Debug(args ...interface{}) {
+	if logger.IsLevelEnabled(DebugLevel) {
+		entry := logger.newEntry()
+		entry.Debug(args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Info(args ...interface{}) {
+	if logger.IsLevelEnabled(InfoLevel) {
+		entry := logger.newEntry()
+		entry.Info(args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Print(args ...interface{}) {
+	entry := logger.newEntry()
+	entry.Info(args...)
+	logger.releaseEntry(entry)
+}
+
+func (logger *Logger) Warn(args ...interface{}) {
+	if logger.IsLevelEnabled(WarnLevel) {
+		entry := logger.newEntry()
+		entry.Warn(args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Warning(args ...interface{}) {
+	if logger.IsLevelEnabled(WarnLevel) {
+		entry := logger.newEntry()
+		entry.Warn(args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Error(args ...interface{}) {
+	if logger.IsLevelEnabled(ErrorLevel) {
+		entry := logger.newEntry()
+		entry.Error(args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Fatal(args ...interface{}) {
+	if logger.IsLevelEnabled(FatalLevel) {
+		entry := logger.newEntry()
+		entry.Fatal(args...)
+		logger.releaseEntry(entry)
+	}
+	logger.Exit(1)
+}
+
+func (logger *Logger) Panic(args ...interface{}) {
+	if logger.IsLevelEnabled(PanicLevel) {
+		entry := logger.newEntry()
+		entry.Panic(args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Traceln(args ...interface{}) {
+	if logger.IsLevelEnabled(TraceLevel) {
+		entry := logger.newEntry()
+		entry.Traceln(args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Debugln(args ...interface{}) {
+	if logger.IsLevelEnabled(DebugLevel) {
+		entry := logger.newEntry()
+		entry.Debugln(args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Infoln(args ...interface{}) {
+	if logger.IsLevelEnabled(InfoLevel) {
+		entry := logger.newEntry()
+		entry.Infoln(args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Println(args ...interface{}) {
+	entry := logger.newEntry()
+	entry.Println(args...)
+	logger.releaseEntry(entry)
+}
+
+func (logger *Logger) Warnln(args ...interface{}) {
+	if logger.IsLevelEnabled(WarnLevel) {
+		entry := logger.newEntry()
+		entry.Warnln(args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Warningln(args ...interface{}) {
+	if logger.IsLevelEnabled(WarnLevel) {
+		entry := logger.newEntry()
+		entry.Warnln(args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Errorln(args ...interface{}) {
+	if logger.IsLevelEnabled(ErrorLevel) {
+		entry := logger.newEntry()
+		entry.Errorln(args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Fatalln(args ...interface{}) {
+	if logger.IsLevelEnabled(FatalLevel) {
+		entry := logger.newEntry()
+		entry.Fatalln(args...)
+		logger.releaseEntry(entry)
+	}
+	logger.Exit(1)
+}
+
+func (logger *Logger) Panicln(args ...interface{}) {
+	if logger.IsLevelEnabled(PanicLevel) {
+		entry := logger.newEntry()
+		entry.Panicln(args...)
+		logger.releaseEntry(entry)
+	}
+}
+
+func (logger *Logger) Exit(code int) {
+	runHandlers()
+	if logger.ExitFunc == nil {
+		logger.ExitFunc = os.Exit
+	}
+	logger.ExitFunc(code)
+}
+
+//When file is opened with appending mode, it's safe to
+//write concurrently to a file (within 4k message on Linux).
+//In these cases user can choose to disable the lock.
+func (logger *Logger) SetNoLock() {
+	logger.mu.Disable()
+}
+
+func (logger *Logger) level() Level {
+	return Level(atomic.LoadUint32((*uint32)(&logger.Level)))
+}
+
+// SetLevel sets the logger level.
+func (logger *Logger) SetLevel(level Level) {
+	atomic.StoreUint32((*uint32)(&logger.Level), uint32(level))
+}
+
+// GetLevel returns the logger level.
+func (logger *Logger) GetLevel() Level {
+	return logger.level()
+}
+
+// AddHook adds a hook to the logger hooks.
+func (logger *Logger) AddHook(hook Hook) {
+	logger.mu.Lock()
+	defer logger.mu.Unlock()
+	logger.Hooks.Add(hook)
+}
+
+// IsLevelEnabled checks if the log level of the logger is greater than the level param
+func (logger *Logger) IsLevelEnabled(level Level) bool {
+	return logger.level() >= level
+}
+
+// SetFormatter sets the logger formatter.
+func (logger *Logger) SetFormatter(formatter Formatter) {
+	logger.mu.Lock()
+	defer logger.mu.Unlock()
+	logger.Formatter = formatter
+}
+
+// SetOutput sets the logger output.
+func (logger *Logger) SetOutput(output io.Writer) {
+	logger.mu.Lock()
+	defer logger.mu.Unlock()
+	logger.Out = output
+}
+
+func (logger *Logger) SetReportCaller(reportCaller bool) {
+	logger.mu.Lock()
+	defer logger.mu.Unlock()
+	logger.ReportCaller = reportCaller
+}
+
+// ReplaceHooks replaces the logger hooks and returns the old ones
+func (logger *Logger) ReplaceHooks(hooks LevelHooks) LevelHooks {
+	logger.mu.Lock()
+	oldHooks := logger.Hooks
+	logger.Hooks = hooks
+	logger.mu.Unlock()
+	return oldHooks
+}
diff --git a/vendor/github.com/sirupsen/logrus/logrus.go b/vendor/github.com/sirupsen/logrus/logrus.go
new file mode 100644
index 0000000..4ef4518
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/logrus.go
@@ -0,0 +1,178 @@
+package logrus
+
+import (
+	"fmt"
+	"log"
+	"strings"
+)
+
+// Fields type, used to pass to `WithFields`.
+type Fields map[string]interface{}
+
+// Level type
+type Level uint32
+
+// Convert the Level to a string. E.g. PanicLevel becomes "panic".
+func (level Level) String() string {
+	switch level {
+	case TraceLevel:
+		return "trace"
+	case DebugLevel:
+		return "debug"
+	case InfoLevel:
+		return "info"
+	case WarnLevel:
+		return "warning"
+	case ErrorLevel:
+		return "error"
+	case FatalLevel:
+		return "fatal"
+	case PanicLevel:
+		return "panic"
+	}
+
+	return "unknown"
+}
+
+// ParseLevel takes a string level and returns the Logrus log level constant.
+func ParseLevel(lvl string) (Level, error) {
+	switch strings.ToLower(lvl) {
+	case "panic":
+		return PanicLevel, nil
+	case "fatal":
+		return FatalLevel, nil
+	case "error":
+		return ErrorLevel, nil
+	case "warn", "warning":
+		return WarnLevel, nil
+	case "info":
+		return InfoLevel, nil
+	case "debug":
+		return DebugLevel, nil
+	case "trace":
+		return TraceLevel, nil
+	}
+
+	var l Level
+	return l, fmt.Errorf("not a valid logrus Level: %q", lvl)
+}
+
+// UnmarshalText implements encoding.TextUnmarshaler.
+func (level *Level) UnmarshalText(text []byte) error {
+	l, err := ParseLevel(string(text))
+	if err != nil {
+		return err
+	}
+
+	*level = Level(l)
+
+	return nil
+}
+
+// A constant exposing all logging levels
+var AllLevels = []Level{
+	PanicLevel,
+	FatalLevel,
+	ErrorLevel,
+	WarnLevel,
+	InfoLevel,
+	DebugLevel,
+	TraceLevel,
+}
+
+// These are the different logging levels. You can set the logging level to log
+// on your instance of logger, obtained with `logrus.New()`.
+const (
+	// PanicLevel level, highest level of severity. Logs and then calls panic with the
+	// message passed to Debug, Info, ...
+	PanicLevel Level = iota
+	// FatalLevel level. Logs and then calls `logger.Exit(1)`. It will exit even if the
+	// logging level is set to Panic.
+	FatalLevel
+	// ErrorLevel level. Logs. Used for errors that should definitely be noted.
+	// Commonly used for hooks to send errors to an error tracking service.
+	ErrorLevel
+	// WarnLevel level. Non-critical entries that deserve eyes.
+	WarnLevel
+	// InfoLevel level. General operational entries about what's going on inside the
+	// application.
+	InfoLevel
+	// DebugLevel level. Usually only enabled when debugging. Very verbose logging.
+	DebugLevel
+	// TraceLevel level. Designates finer-grained informational events than the Debug.
+	TraceLevel
+)
+
+// Won't compile if StdLogger can't be realized by a log.Logger
+var (
+	_ StdLogger = &log.Logger{}
+	_ StdLogger = &Entry{}
+	_ StdLogger = &Logger{}
+)
+
+// StdLogger is what your logrus-enabled library should take, that way
+// it'll accept a stdlib logger and a logrus logger. There's no standard
+// interface, this is the closest we get, unfortunately.
+type StdLogger interface {
+	Print(...interface{})
+	Printf(string, ...interface{})
+	Println(...interface{})
+
+	Fatal(...interface{})
+	Fatalf(string, ...interface{})
+	Fatalln(...interface{})
+
+	Panic(...interface{})
+	Panicf(string, ...interface{})
+	Panicln(...interface{})
+}
+
+// The FieldLogger interface generalizes the Entry and Logger types
+type FieldLogger interface {
+	WithField(key string, value interface{}) *Entry
+	WithFields(fields Fields) *Entry
+	WithError(err error) *Entry
+
+	Debugf(format string, args ...interface{})
+	Infof(format string, args ...interface{})
+	Printf(format string, args ...interface{})
+	Warnf(format string, args ...interface{})
+	Warningf(format string, args ...interface{})
+	Errorf(format string, args ...interface{})
+	Fatalf(format string, args ...interface{})
+	Panicf(format string, args ...interface{})
+
+	Debug(args ...interface{})
+	Info(args ...interface{})
+	Print(args ...interface{})
+	Warn(args ...interface{})
+	Warning(args ...interface{})
+	Error(args ...interface{})
+	Fatal(args ...interface{})
+	Panic(args ...interface{})
+
+	Debugln(args ...interface{})
+	Infoln(args ...interface{})
+	Println(args ...interface{})
+	Warnln(args ...interface{})
+	Warningln(args ...interface{})
+	Errorln(args ...interface{})
+	Fatalln(args ...interface{})
+	Panicln(args ...interface{})
+
+	// IsDebugEnabled() bool
+	// IsInfoEnabled() bool
+	// IsWarnEnabled() bool
+	// IsErrorEnabled() bool
+	// IsFatalEnabled() bool
+	// IsPanicEnabled() bool
+}
+
+// Ext1FieldLogger (the first extension to FieldLogger) is superfluous, it is
+// here for consistancy. Do not use. Use Logger or Entry instead.
+type Ext1FieldLogger interface {
+	FieldLogger
+	Tracef(format string, args ...interface{})
+	Trace(args ...interface{})
+	Traceln(args ...interface{})
+}
diff --git a/vendor/github.com/sirupsen/logrus/terminal_check_appengine.go b/vendor/github.com/sirupsen/logrus/terminal_check_appengine.go
new file mode 100644
index 0000000..2403de9
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/terminal_check_appengine.go
@@ -0,0 +1,11 @@
+// +build appengine
+
+package logrus
+
+import (
+	"io"
+)
+
+func checkIfTerminal(w io.Writer) bool {
+	return true
+}
diff --git a/vendor/github.com/sirupsen/logrus/terminal_check_js.go b/vendor/github.com/sirupsen/logrus/terminal_check_js.go
new file mode 100644
index 0000000..0c20975
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/terminal_check_js.go
@@ -0,0 +1,11 @@
+// +build js
+
+package logrus
+
+import (
+	"io"
+)
+
+func checkIfTerminal(w io.Writer) bool {
+	return false
+}
diff --git a/vendor/github.com/sirupsen/logrus/terminal_check_notappengine.go b/vendor/github.com/sirupsen/logrus/terminal_check_notappengine.go
new file mode 100644
index 0000000..cf309d6
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/terminal_check_notappengine.go
@@ -0,0 +1,19 @@
+// +build !appengine,!js,!windows
+
+package logrus
+
+import (
+	"io"
+	"os"
+
+	"golang.org/x/crypto/ssh/terminal"
+)
+
+func checkIfTerminal(w io.Writer) bool {
+	switch v := w.(type) {
+	case *os.File:
+		return terminal.IsTerminal(int(v.Fd()))
+	default:
+		return false
+	}
+}
diff --git a/vendor/github.com/sirupsen/logrus/terminal_check_windows.go b/vendor/github.com/sirupsen/logrus/terminal_check_windows.go
new file mode 100644
index 0000000..3b9d286
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/terminal_check_windows.go
@@ -0,0 +1,20 @@
+// +build !appengine,!js,windows
+
+package logrus
+
+import (
+	"io"
+	"os"
+	"syscall"
+)
+
+func checkIfTerminal(w io.Writer) bool {
+	switch v := w.(type) {
+	case *os.File:
+		var mode uint32
+		err := syscall.GetConsoleMode(syscall.Handle(v.Fd()), &mode)
+		return err == nil
+	default:
+		return false
+	}
+}
diff --git a/vendor/github.com/sirupsen/logrus/terminal_notwindows.go b/vendor/github.com/sirupsen/logrus/terminal_notwindows.go
new file mode 100644
index 0000000..3dbd237
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/terminal_notwindows.go
@@ -0,0 +1,8 @@
+// +build !windows
+
+package logrus
+
+import "io"
+
+func initTerminal(w io.Writer) {
+}
diff --git a/vendor/github.com/sirupsen/logrus/terminal_windows.go b/vendor/github.com/sirupsen/logrus/terminal_windows.go
new file mode 100644
index 0000000..b4ef528
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/terminal_windows.go
@@ -0,0 +1,18 @@
+// +build !appengine,!js,windows
+
+package logrus
+
+import (
+	"io"
+	"os"
+	"syscall"
+
+	sequences "github.com/konsorten/go-windows-terminal-sequences"
+)
+
+func initTerminal(w io.Writer) {
+	switch v := w.(type) {
+	case *os.File:
+		sequences.EnableVirtualTerminalProcessing(syscall.Handle(v.Fd()), true)
+	}
+}
diff --git a/vendor/github.com/sirupsen/logrus/text_formatter.go b/vendor/github.com/sirupsen/logrus/text_formatter.go
new file mode 100644
index 0000000..49ec92f
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/text_formatter.go
@@ -0,0 +1,269 @@
+package logrus
+
+import (
+	"bytes"
+	"fmt"
+	"os"
+	"sort"
+	"strings"
+	"sync"
+	"time"
+)
+
+const (
+	nocolor = 0
+	red     = 31
+	green   = 32
+	yellow  = 33
+	blue    = 36
+	gray    = 37
+)
+
+var (
+	baseTimestamp time.Time
+	emptyFieldMap FieldMap
+)
+
+func init() {
+	baseTimestamp = time.Now()
+}
+
+// TextFormatter formats logs into text
+type TextFormatter struct {
+	// Set to true to bypass checking for a TTY before outputting colors.
+	ForceColors bool
+
+	// Force disabling colors.
+	DisableColors bool
+
+	// Override coloring based on CLICOLOR and CLICOLOR_FORCE. - https://bixense.com/clicolors/
+	EnvironmentOverrideColors bool
+
+	// Disable timestamp logging. useful when output is redirected to logging
+	// system that already adds timestamps.
+	DisableTimestamp bool
+
+	// Enable logging the full timestamp when a TTY is attached instead of just
+	// the time passed since beginning of execution.
+	FullTimestamp bool
+
+	// TimestampFormat to use for display when a full timestamp is printed
+	TimestampFormat string
+
+	// The fields are sorted by default for a consistent output. For applications
+	// that log extremely frequently and don't use the JSON formatter this may not
+	// be desired.
+	DisableSorting bool
+
+	// The keys sorting function, when uninitialized it uses sort.Strings.
+	SortingFunc func([]string)
+
+	// Disables the truncation of the level text to 4 characters.
+	DisableLevelTruncation bool
+
+	// QuoteEmptyFields will wrap empty fields in quotes if true
+	QuoteEmptyFields bool
+
+	// Whether the logger's out is to a terminal
+	isTerminal bool
+
+	// FieldMap allows users to customize the names of keys for default fields.
+	// As an example:
+	// formatter := &TextFormatter{
+	//     FieldMap: FieldMap{
+	//         FieldKeyTime:  "@timestamp",
+	//         FieldKeyLevel: "@level",
+	//         FieldKeyMsg:   "@message"}}
+	FieldMap FieldMap
+
+	terminalInitOnce sync.Once
+}
+
+func (f *TextFormatter) init(entry *Entry) {
+	if entry.Logger != nil {
+		f.isTerminal = checkIfTerminal(entry.Logger.Out)
+
+		if f.isTerminal {
+			initTerminal(entry.Logger.Out)
+		}
+	}
+}
+
+func (f *TextFormatter) isColored() bool {
+	isColored := f.ForceColors || f.isTerminal
+
+	if f.EnvironmentOverrideColors {
+		if force, ok := os.LookupEnv("CLICOLOR_FORCE"); ok && force != "0" {
+			isColored = true
+		} else if ok && force == "0" {
+			isColored = false
+		} else if os.Getenv("CLICOLOR") == "0" {
+			isColored = false
+		}
+	}
+
+	return isColored && !f.DisableColors
+}
+
+// Format renders a single log entry
+func (f *TextFormatter) Format(entry *Entry) ([]byte, error) {
+	prefixFieldClashes(entry.Data, f.FieldMap, entry.HasCaller())
+
+	keys := make([]string, 0, len(entry.Data))
+	for k := range entry.Data {
+		keys = append(keys, k)
+	}
+
+	fixedKeys := make([]string, 0, 4+len(entry.Data))
+	if !f.DisableTimestamp {
+		fixedKeys = append(fixedKeys, f.FieldMap.resolve(FieldKeyTime))
+	}
+	fixedKeys = append(fixedKeys, f.FieldMap.resolve(FieldKeyLevel))
+	if entry.Message != "" {
+		fixedKeys = append(fixedKeys, f.FieldMap.resolve(FieldKeyMsg))
+	}
+	if entry.err != "" {
+		fixedKeys = append(fixedKeys, f.FieldMap.resolve(FieldKeyLogrusError))
+	}
+	if entry.HasCaller() {
+		fixedKeys = append(fixedKeys,
+			f.FieldMap.resolve(FieldKeyFunc), f.FieldMap.resolve(FieldKeyFile))
+	}
+
+	if !f.DisableSorting {
+		if f.SortingFunc == nil {
+			sort.Strings(keys)
+			fixedKeys = append(fixedKeys, keys...)
+		} else {
+			if !f.isColored() {
+				fixedKeys = append(fixedKeys, keys...)
+				f.SortingFunc(fixedKeys)
+			} else {
+				f.SortingFunc(keys)
+			}
+		}
+	} else {
+		fixedKeys = append(fixedKeys, keys...)
+	}
+
+	var b *bytes.Buffer
+	if entry.Buffer != nil {
+		b = entry.Buffer
+	} else {
+		b = &bytes.Buffer{}
+	}
+
+	f.terminalInitOnce.Do(func() { f.init(entry) })
+
+	timestampFormat := f.TimestampFormat
+	if timestampFormat == "" {
+		timestampFormat = defaultTimestampFormat
+	}
+	if f.isColored() {
+		f.printColored(b, entry, keys, timestampFormat)
+	} else {
+		for _, key := range fixedKeys {
+			var value interface{}
+			switch {
+			case key == f.FieldMap.resolve(FieldKeyTime):
+				value = entry.Time.Format(timestampFormat)
+			case key == f.FieldMap.resolve(FieldKeyLevel):
+				value = entry.Level.String()
+			case key == f.FieldMap.resolve(FieldKeyMsg):
+				value = entry.Message
+			case key == f.FieldMap.resolve(FieldKeyLogrusError):
+				value = entry.err
+			case key == f.FieldMap.resolve(FieldKeyFunc) && entry.HasCaller():
+				value = entry.Caller.Function
+			case key == f.FieldMap.resolve(FieldKeyFile) && entry.HasCaller():
+				value = fmt.Sprintf("%s:%d", entry.Caller.File, entry.Caller.Line)
+			default:
+				value = entry.Data[key]
+			}
+			f.appendKeyValue(b, key, value)
+		}
+	}
+
+	b.WriteByte('\n')
+	return b.Bytes(), nil
+}
+
+func (f *TextFormatter) printColored(b *bytes.Buffer, entry *Entry, keys []string, timestampFormat string) {
+	var levelColor int
+	switch entry.Level {
+	case DebugLevel, TraceLevel:
+		levelColor = gray
+	case WarnLevel:
+		levelColor = yellow
+	case ErrorLevel, FatalLevel, PanicLevel:
+		levelColor = red
+	default:
+		levelColor = blue
+	}
+
+	levelText := strings.ToUpper(entry.Level.String())
+	if !f.DisableLevelTruncation {
+		levelText = levelText[0:4]
+	}
+
+	// Remove a single newline if it already exists in the message to keep
+	// the behavior of logrus text_formatter the same as the stdlib log package
+	entry.Message = strings.TrimSuffix(entry.Message, "\n")
+
+	caller := ""
+
+	if entry.HasCaller() {
+		caller = fmt.Sprintf("%s:%d %s()",
+			entry.Caller.File, entry.Caller.Line, entry.Caller.Function)
+	}
+
+	if f.DisableTimestamp {
+		fmt.Fprintf(b, "\x1b[%dm%s\x1b[0m%s %-44s ", levelColor, levelText, caller, entry.Message)
+	} else if !f.FullTimestamp {
+		fmt.Fprintf(b, "\x1b[%dm%s\x1b[0m[%04d]%s %-44s ", levelColor, levelText, int(entry.Time.Sub(baseTimestamp)/time.Second), caller, entry.Message)
+	} else {
+		fmt.Fprintf(b, "\x1b[%dm%s\x1b[0m[%s]%s %-44s ", levelColor, levelText, entry.Time.Format(timestampFormat), caller, entry.Message)
+	}
+	for _, k := range keys {
+		v := entry.Data[k]
+		fmt.Fprintf(b, " \x1b[%dm%s\x1b[0m=", levelColor, k)
+		f.appendValue(b, v)
+	}
+}
+
+func (f *TextFormatter) needsQuoting(text string) bool {
+	if f.QuoteEmptyFields && len(text) == 0 {
+		return true
+	}
+	for _, ch := range text {
+		if !((ch >= 'a' && ch <= 'z') ||
+			(ch >= 'A' && ch <= 'Z') ||
+			(ch >= '0' && ch <= '9') ||
+			ch == '-' || ch == '.' || ch == '_' || ch == '/' || ch == '@' || ch == '^' || ch == '+') {
+			return true
+		}
+	}
+	return false
+}
+
+func (f *TextFormatter) appendKeyValue(b *bytes.Buffer, key string, value interface{}) {
+	if b.Len() > 0 {
+		b.WriteByte(' ')
+	}
+	b.WriteString(key)
+	b.WriteByte('=')
+	f.appendValue(b, value)
+}
+
+func (f *TextFormatter) appendValue(b *bytes.Buffer, value interface{}) {
+	stringVal, ok := value.(string)
+	if !ok {
+		stringVal = fmt.Sprint(value)
+	}
+
+	if !f.needsQuoting(stringVal) {
+		b.WriteString(stringVal)
+	} else {
+		b.WriteString(fmt.Sprintf("%q", stringVal))
+	}
+}
diff --git a/vendor/github.com/sirupsen/logrus/writer.go b/vendor/github.com/sirupsen/logrus/writer.go
new file mode 100644
index 0000000..9e1f751
--- /dev/null
+++ b/vendor/github.com/sirupsen/logrus/writer.go
@@ -0,0 +1,64 @@
+package logrus
+
+import (
+	"bufio"
+	"io"
+	"runtime"
+)
+
+func (logger *Logger) Writer() *io.PipeWriter {
+	return logger.WriterLevel(InfoLevel)
+}
+
+func (logger *Logger) WriterLevel(level Level) *io.PipeWriter {
+	return NewEntry(logger).WriterLevel(level)
+}
+
+func (entry *Entry) Writer() *io.PipeWriter {
+	return entry.WriterLevel(InfoLevel)
+}
+
+func (entry *Entry) WriterLevel(level Level) *io.PipeWriter {
+	reader, writer := io.Pipe()
+
+	var printFunc func(args ...interface{})
+
+	switch level {
+	case TraceLevel:
+		printFunc = entry.Trace
+	case DebugLevel:
+		printFunc = entry.Debug
+	case InfoLevel:
+		printFunc = entry.Info
+	case WarnLevel:
+		printFunc = entry.Warn
+	case ErrorLevel:
+		printFunc = entry.Error
+	case FatalLevel:
+		printFunc = entry.Fatal
+	case PanicLevel:
+		printFunc = entry.Panic
+	default:
+		printFunc = entry.Print
+	}
+
+	go entry.writerScanner(reader, printFunc)
+	runtime.SetFinalizer(writer, writerFinalizer)
+
+	return writer
+}
+
+func (entry *Entry) writerScanner(reader *io.PipeReader, printFunc func(args ...interface{})) {
+	scanner := bufio.NewScanner(reader)
+	for scanner.Scan() {
+		printFunc(scanner.Text())
+	}
+	if err := scanner.Err(); err != nil {
+		entry.Errorf("Error while reading from Writer: %s", err)
+	}
+	reader.Close()
+}
+
+func writerFinalizer(writer *io.PipeWriter) {
+	writer.Close()
+}
diff --git a/vendor/github.com/soheilhy/cmux/.gitignore b/vendor/github.com/soheilhy/cmux/.gitignore
new file mode 100644
index 0000000..daf913b
--- /dev/null
+++ b/vendor/github.com/soheilhy/cmux/.gitignore
@@ -0,0 +1,24 @@
+# Compiled Object files, Static and Dynamic libs (Shared Objects)
+*.o
+*.a
+*.so
+
+# Folders
+_obj
+_test
+
+# Architecture specific extensions/prefixes
+*.[568vq]
+[568vq].out
+
+*.cgo1.go
+*.cgo2.c
+_cgo_defun.c
+_cgo_gotypes.go
+_cgo_export.*
+
+_testmain.go
+
+*.exe
+*.test
+*.prof
diff --git a/vendor/github.com/soheilhy/cmux/.travis.yml b/vendor/github.com/soheilhy/cmux/.travis.yml
new file mode 100644
index 0000000..4bc48e0
--- /dev/null
+++ b/vendor/github.com/soheilhy/cmux/.travis.yml
@@ -0,0 +1,29 @@
+language: go
+
+go:
+  - 1.6
+  - 1.7
+  - 1.8
+  - tip
+
+matrix:
+  allow_failures:
+    - go: tip
+
+gobuild_args: -race
+
+before_install:
+  - if [[ $TRAVIS_GO_VERSION == 1.6* ]]; then go get -u github.com/kisielk/errcheck; fi
+  - if [[ $TRAVIS_GO_VERSION == 1.6* ]]; then go get -u github.com/golang/lint/golint; fi
+
+before_script:
+  - '! gofmt -s -l . | read'
+  - echo $TRAVIS_GO_VERSION
+  - if [[ $TRAVIS_GO_VERSION == 1.6* ]]; then golint ./...; fi
+  - if [[ $TRAVIS_GO_VERSION == 1.6* ]]; then errcheck ./...; fi
+  - if [[ $TRAVIS_GO_VERSION == 1.6* ]]; then go tool vet .; fi
+  - if [[ $TRAVIS_GO_VERSION == 1.6* ]]; then go tool vet --shadow .; fi
+
+script:
+  - go test -bench . -v ./...
+  - go test -race -bench . -v ./...
diff --git a/vendor/github.com/soheilhy/cmux/CONTRIBUTORS b/vendor/github.com/soheilhy/cmux/CONTRIBUTORS
new file mode 100644
index 0000000..49878f2
--- /dev/null
+++ b/vendor/github.com/soheilhy/cmux/CONTRIBUTORS
@@ -0,0 +1,12 @@
+# The list of people who have contributed code to the cmux repository.
+#
+# Auto-generated with:
+#		git log --oneline --pretty=format:'%an <%aE>' | sort -u
+#
+Andreas Jaekle <andreas@jaekle.net>
+Dmitri Shuralyov <shurcooL@gmail.com>
+Ethan Mosbaugh <emosbaugh@gmail.com>
+Soheil Hassas Yeganeh <soheil.h.y@gmail.com>
+Soheil Hassas Yeganeh <soheil@cs.toronto.edu>
+Tamir Duberstein <tamir@cockroachlabs.com>
+Tamir Duberstein <tamird@gmail.com>
diff --git a/vendor/github.com/soheilhy/cmux/LICENSE b/vendor/github.com/soheilhy/cmux/LICENSE
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/vendor/github.com/soheilhy/cmux/LICENSE
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
diff --git a/vendor/github.com/soheilhy/cmux/README.md b/vendor/github.com/soheilhy/cmux/README.md
new file mode 100644
index 0000000..70306e6
--- /dev/null
+++ b/vendor/github.com/soheilhy/cmux/README.md
@@ -0,0 +1,83 @@
+# cmux: Connection Mux ![Travis Build Status](https://api.travis-ci.org/soheilhy/args.svg?branch=master "Travis Build Status") [![GoDoc](https://godoc.org/github.com/soheilhy/cmux?status.svg)](http://godoc.org/github.com/soheilhy/cmux)
+
+cmux is a generic Go library to multiplex connections based on
+their payload. Using cmux, you can serve gRPC, SSH, HTTPS, HTTP,
+Go RPC, and pretty much any other protocol on the same TCP listener.
+
+## How-To
+Simply create your main listener, create a cmux for that listener,
+and then match connections:
+```go
+// Create the main listener.
+l, err := net.Listen("tcp", ":23456")
+if err != nil {
+	log.Fatal(err)
+}
+
+// Create a cmux.
+m := cmux.New(l)
+
+// Match connections in order:
+// First grpc, then HTTP, and otherwise Go RPC/TCP.
+grpcL := m.Match(cmux.HTTP2HeaderField("content-type", "application/grpc"))
+httpL := m.Match(cmux.HTTP1Fast())
+trpcL := m.Match(cmux.Any()) // Any means anything that is not yet matched.
+
+// Create your protocol servers.
+grpcS := grpc.NewServer()
+grpchello.RegisterGreeterServer(grpcs, &server{})
+
+httpS := &http.Server{
+	Handler: &helloHTTP1Handler{},
+}
+
+trpcS := rpc.NewServer()
+trpcS.Register(&ExampleRPCRcvr{})
+
+// Use the muxed listeners for your servers.
+go grpcS.Serve(grpcL)
+go httpS.Serve(httpL)
+go trpcS.Accept(trpcL)
+
+// Start serving!
+m.Serve()
+```
+
+Take a look at [other examples in the GoDoc](http://godoc.org/github.com/soheilhy/cmux/#pkg-examples).
+
+## Docs
+* [GoDocs](https://godoc.org/github.com/soheilhy/cmux)
+
+## Performance
+There is room for improvment but, since we are only matching
+the very first bytes of a connection, the performance overheads on
+long-lived connections (i.e., RPCs and pipelined HTTP streams)
+is negligible.
+
+*TODO(soheil)*: Add benchmarks.
+
+## Limitations
+* *TLS*: `net/http` uses a type assertion to identify TLS connections; since
+cmux's lookahead-implementing connection wraps the underlying TLS connection,
+this type assertion fails.
+Because of that, you can serve HTTPS using cmux but `http.Request.TLS`
+would not be set in your handlers.
+
+* *Different Protocols on The Same Connection*: `cmux` matches the connection
+when it's accepted. For example, one connection can be either gRPC or REST, but
+not both. That is, we assume that a client connection is either used for gRPC
+or REST.
+
+* *Java gRPC Clients*: Java gRPC client blocks until it receives a SETTINGS
+frame from the server. If you are using the Java client to connect to a cmux'ed
+gRPC server please match with writers:
+```go
+grpcl := m.MatchWithWriters(cmux.HTTP2MatchHeaderFieldSendSettings("content-type", "application/grpc"))
+```
+
+# Copyright and License
+Copyright 2016 The CMux Authors. All rights reserved.
+
+See [CONTRIBUTORS](https://github.com/soheilhy/cmux/blob/master/CONTRIBUTORS)
+for the CMux Authors. Code is released under
+[the Apache 2 license](https://github.com/soheilhy/cmux/blob/master/LICENSE).
diff --git a/vendor/github.com/soheilhy/cmux/buffer.go b/vendor/github.com/soheilhy/cmux/buffer.go
new file mode 100644
index 0000000..f8cf30a
--- /dev/null
+++ b/vendor/github.com/soheilhy/cmux/buffer.go
@@ -0,0 +1,67 @@
+// Copyright 2016 The CMux Authors. All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+// implied. See the License for the specific language governing
+// permissions and limitations under the License.
+
+package cmux
+
+import (
+	"bytes"
+	"io"
+)
+
+// bufferedReader is an optimized implementation of io.Reader that behaves like
+// ```
+// io.MultiReader(bytes.NewReader(buffer.Bytes()), io.TeeReader(source, buffer))
+// ```
+// without allocating.
+type bufferedReader struct {
+	source     io.Reader
+	buffer     bytes.Buffer
+	bufferRead int
+	bufferSize int
+	sniffing   bool
+	lastErr    error
+}
+
+func (s *bufferedReader) Read(p []byte) (int, error) {
+	if s.bufferSize > s.bufferRead {
+		// If we have already read something from the buffer before, we return the
+		// same data and the last error if any. We need to immediately return,
+		// otherwise we may block for ever, if we try to be smart and call
+		// source.Read() seeking a little bit of more data.
+		bn := copy(p, s.buffer.Bytes()[s.bufferRead:s.bufferSize])
+		s.bufferRead += bn
+		return bn, s.lastErr
+	} else if !s.sniffing && s.buffer.Cap() != 0 {
+		// We don't need the buffer anymore.
+		// Reset it to release the internal slice.
+		s.buffer = bytes.Buffer{}
+	}
+
+	// If there is nothing more to return in the sniffed buffer, read from the
+	// source.
+	sn, sErr := s.source.Read(p)
+	if sn > 0 && s.sniffing {
+		s.lastErr = sErr
+		if wn, wErr := s.buffer.Write(p[:sn]); wErr != nil {
+			return wn, wErr
+		}
+	}
+	return sn, sErr
+}
+
+func (s *bufferedReader) reset(snif bool) {
+	s.sniffing = snif
+	s.bufferRead = 0
+	s.bufferSize = s.buffer.Len()
+}
diff --git a/vendor/github.com/soheilhy/cmux/cmux.go b/vendor/github.com/soheilhy/cmux/cmux.go
new file mode 100644
index 0000000..8040342
--- /dev/null
+++ b/vendor/github.com/soheilhy/cmux/cmux.go
@@ -0,0 +1,270 @@
+// Copyright 2016 The CMux Authors. All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+// implied. See the License for the specific language governing
+// permissions and limitations under the License.
+
+package cmux
+
+import (
+	"fmt"
+	"io"
+	"net"
+	"sync"
+	"time"
+)
+
+// Matcher matches a connection based on its content.
+type Matcher func(io.Reader) bool
+
+// MatchWriter is a match that can also write response (say to do handshake).
+type MatchWriter func(io.Writer, io.Reader) bool
+
+// ErrorHandler handles an error and returns whether
+// the mux should continue serving the listener.
+type ErrorHandler func(error) bool
+
+var _ net.Error = ErrNotMatched{}
+
+// ErrNotMatched is returned whenever a connection is not matched by any of
+// the matchers registered in the multiplexer.
+type ErrNotMatched struct {
+	c net.Conn
+}
+
+func (e ErrNotMatched) Error() string {
+	return fmt.Sprintf("mux: connection %v not matched by an matcher",
+		e.c.RemoteAddr())
+}
+
+// Temporary implements the net.Error interface.
+func (e ErrNotMatched) Temporary() bool { return true }
+
+// Timeout implements the net.Error interface.
+func (e ErrNotMatched) Timeout() bool { return false }
+
+type errListenerClosed string
+
+func (e errListenerClosed) Error() string   { return string(e) }
+func (e errListenerClosed) Temporary() bool { return false }
+func (e errListenerClosed) Timeout() bool   { return false }
+
+// ErrListenerClosed is returned from muxListener.Accept when the underlying
+// listener is closed.
+var ErrListenerClosed = errListenerClosed("mux: listener closed")
+
+// for readability of readTimeout
+var noTimeout time.Duration
+
+// New instantiates a new connection multiplexer.
+func New(l net.Listener) CMux {
+	return &cMux{
+		root:        l,
+		bufLen:      1024,
+		errh:        func(_ error) bool { return true },
+		donec:       make(chan struct{}),
+		readTimeout: noTimeout,
+	}
+}
+
+// CMux is a multiplexer for network connections.
+type CMux interface {
+	// Match returns a net.Listener that sees (i.e., accepts) only
+	// the connections matched by at least one of the matcher.
+	//
+	// The order used to call Match determines the priority of matchers.
+	Match(...Matcher) net.Listener
+	// MatchWithWriters returns a net.Listener that accepts only the
+	// connections that matched by at least of the matcher writers.
+	//
+	// Prefer Matchers over MatchWriters, since the latter can write on the
+	// connection before the actual handler.
+	//
+	// The order used to call Match determines the priority of matchers.
+	MatchWithWriters(...MatchWriter) net.Listener
+	// Serve starts multiplexing the listener. Serve blocks and perhaps
+	// should be invoked concurrently within a go routine.
+	Serve() error
+	// HandleError registers an error handler that handles listener errors.
+	HandleError(ErrorHandler)
+	// sets a timeout for the read of matchers
+	SetReadTimeout(time.Duration)
+}
+
+type matchersListener struct {
+	ss []MatchWriter
+	l  muxListener
+}
+
+type cMux struct {
+	root        net.Listener
+	bufLen      int
+	errh        ErrorHandler
+	donec       chan struct{}
+	sls         []matchersListener
+	readTimeout time.Duration
+}
+
+func matchersToMatchWriters(matchers []Matcher) []MatchWriter {
+	mws := make([]MatchWriter, 0, len(matchers))
+	for _, m := range matchers {
+		cm := m
+		mws = append(mws, func(w io.Writer, r io.Reader) bool {
+			return cm(r)
+		})
+	}
+	return mws
+}
+
+func (m *cMux) Match(matchers ...Matcher) net.Listener {
+	mws := matchersToMatchWriters(matchers)
+	return m.MatchWithWriters(mws...)
+}
+
+func (m *cMux) MatchWithWriters(matchers ...MatchWriter) net.Listener {
+	ml := muxListener{
+		Listener: m.root,
+		connc:    make(chan net.Conn, m.bufLen),
+	}
+	m.sls = append(m.sls, matchersListener{ss: matchers, l: ml})
+	return ml
+}
+
+func (m *cMux) SetReadTimeout(t time.Duration) {
+	m.readTimeout = t
+}
+
+func (m *cMux) Serve() error {
+	var wg sync.WaitGroup
+
+	defer func() {
+		close(m.donec)
+		wg.Wait()
+
+		for _, sl := range m.sls {
+			close(sl.l.connc)
+			// Drain the connections enqueued for the listener.
+			for c := range sl.l.connc {
+				_ = c.Close()
+			}
+		}
+	}()
+
+	for {
+		c, err := m.root.Accept()
+		if err != nil {
+			if !m.handleErr(err) {
+				return err
+			}
+			continue
+		}
+
+		wg.Add(1)
+		go m.serve(c, m.donec, &wg)
+	}
+}
+
+func (m *cMux) serve(c net.Conn, donec <-chan struct{}, wg *sync.WaitGroup) {
+	defer wg.Done()
+
+	muc := newMuxConn(c)
+	if m.readTimeout > noTimeout {
+		_ = c.SetReadDeadline(time.Now().Add(m.readTimeout))
+	}
+	for _, sl := range m.sls {
+		for _, s := range sl.ss {
+			matched := s(muc.Conn, muc.startSniffing())
+			if matched {
+				muc.doneSniffing()
+				if m.readTimeout > noTimeout {
+					_ = c.SetReadDeadline(time.Time{})
+				}
+				select {
+				case sl.l.connc <- muc:
+				case <-donec:
+					_ = c.Close()
+				}
+				return
+			}
+		}
+	}
+
+	_ = c.Close()
+	err := ErrNotMatched{c: c}
+	if !m.handleErr(err) {
+		_ = m.root.Close()
+	}
+}
+
+func (m *cMux) HandleError(h ErrorHandler) {
+	m.errh = h
+}
+
+func (m *cMux) handleErr(err error) bool {
+	if !m.errh(err) {
+		return false
+	}
+
+	if ne, ok := err.(net.Error); ok {
+		return ne.Temporary()
+	}
+
+	return false
+}
+
+type muxListener struct {
+	net.Listener
+	connc chan net.Conn
+}
+
+func (l muxListener) Accept() (net.Conn, error) {
+	c, ok := <-l.connc
+	if !ok {
+		return nil, ErrListenerClosed
+	}
+	return c, nil
+}
+
+// MuxConn wraps a net.Conn and provides transparent sniffing of connection data.
+type MuxConn struct {
+	net.Conn
+	buf bufferedReader
+}
+
+func newMuxConn(c net.Conn) *MuxConn {
+	return &MuxConn{
+		Conn: c,
+		buf:  bufferedReader{source: c},
+	}
+}
+
+// From the io.Reader documentation:
+//
+// When Read encounters an error or end-of-file condition after
+// successfully reading n > 0 bytes, it returns the number of
+// bytes read.  It may return the (non-nil) error from the same call
+// or return the error (and n == 0) from a subsequent call.
+// An instance of this general case is that a Reader returning
+// a non-zero number of bytes at the end of the input stream may
+// return either err == EOF or err == nil.  The next Read should
+// return 0, EOF.
+func (m *MuxConn) Read(p []byte) (int, error) {
+	return m.buf.Read(p)
+}
+
+func (m *MuxConn) startSniffing() io.Reader {
+	m.buf.reset(true)
+	return &m.buf
+}
+
+func (m *MuxConn) doneSniffing() {
+	m.buf.reset(false)
+}
diff --git a/vendor/github.com/soheilhy/cmux/doc.go b/vendor/github.com/soheilhy/cmux/doc.go
new file mode 100644
index 0000000..aaa8f31
--- /dev/null
+++ b/vendor/github.com/soheilhy/cmux/doc.go
@@ -0,0 +1,18 @@
+// Copyright 2016 The CMux Authors. All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+// implied. See the License for the specific language governing
+// permissions and limitations under the License.
+
+// Package cmux is a library to multiplex network connections based on
+// their payload. Using cmux, you can serve different protocols from the
+// same listener.
+package cmux
diff --git a/vendor/github.com/soheilhy/cmux/matchers.go b/vendor/github.com/soheilhy/cmux/matchers.go
new file mode 100644
index 0000000..878ae98
--- /dev/null
+++ b/vendor/github.com/soheilhy/cmux/matchers.go
@@ -0,0 +1,267 @@
+// Copyright 2016 The CMux Authors. All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+// implied. See the License for the specific language governing
+// permissions and limitations under the License.
+
+package cmux
+
+import (
+	"bufio"
+	"crypto/tls"
+	"io"
+	"io/ioutil"
+	"net/http"
+	"strings"
+
+	"golang.org/x/net/http2"
+	"golang.org/x/net/http2/hpack"
+)
+
+// Any is a Matcher that matches any connection.
+func Any() Matcher {
+	return func(r io.Reader) bool { return true }
+}
+
+// PrefixMatcher returns a matcher that matches a connection if it
+// starts with any of the strings in strs.
+func PrefixMatcher(strs ...string) Matcher {
+	pt := newPatriciaTreeString(strs...)
+	return pt.matchPrefix
+}
+
+func prefixByteMatcher(list ...[]byte) Matcher {
+	pt := newPatriciaTree(list...)
+	return pt.matchPrefix
+}
+
+var defaultHTTPMethods = []string{
+	"OPTIONS",
+	"GET",
+	"HEAD",
+	"POST",
+	"PUT",
+	"DELETE",
+	"TRACE",
+	"CONNECT",
+}
+
+// HTTP1Fast only matches the methods in the HTTP request.
+//
+// This matcher is very optimistic: if it returns true, it does not mean that
+// the request is a valid HTTP response. If you want a correct but slower HTTP1
+// matcher, use HTTP1 instead.
+func HTTP1Fast(extMethods ...string) Matcher {
+	return PrefixMatcher(append(defaultHTTPMethods, extMethods...)...)
+}
+
+// TLS matches HTTPS requests.
+//
+// By default, any TLS handshake packet is matched. An optional whitelist
+// of versions can be passed in to restrict the matcher, for example:
+//  TLS(tls.VersionTLS11, tls.VersionTLS12)
+func TLS(versions ...int) Matcher {
+	if len(versions) == 0 {
+		versions = []int{
+			tls.VersionSSL30,
+			tls.VersionTLS10,
+			tls.VersionTLS11,
+			tls.VersionTLS12,
+		}
+	}
+	prefixes := [][]byte{}
+	for _, v := range versions {
+		prefixes = append(prefixes, []byte{22, byte(v >> 8 & 0xff), byte(v & 0xff)})
+	}
+	return prefixByteMatcher(prefixes...)
+}
+
+const maxHTTPRead = 4096
+
+// HTTP1 parses the first line or upto 4096 bytes of the request to see if
+// the conection contains an HTTP request.
+func HTTP1() Matcher {
+	return func(r io.Reader) bool {
+		br := bufio.NewReader(&io.LimitedReader{R: r, N: maxHTTPRead})
+		l, part, err := br.ReadLine()
+		if err != nil || part {
+			return false
+		}
+
+		_, _, proto, ok := parseRequestLine(string(l))
+		if !ok {
+			return false
+		}
+
+		v, _, ok := http.ParseHTTPVersion(proto)
+		return ok && v == 1
+	}
+}
+
+// grabbed from net/http.
+func parseRequestLine(line string) (method, uri, proto string, ok bool) {
+	s1 := strings.Index(line, " ")
+	s2 := strings.Index(line[s1+1:], " ")
+	if s1 < 0 || s2 < 0 {
+		return
+	}
+	s2 += s1 + 1
+	return line[:s1], line[s1+1 : s2], line[s2+1:], true
+}
+
+// HTTP2 parses the frame header of the first frame to detect whether the
+// connection is an HTTP2 connection.
+func HTTP2() Matcher {
+	return hasHTTP2Preface
+}
+
+// HTTP1HeaderField returns a matcher matching the header fields of the first
+// request of an HTTP 1 connection.
+func HTTP1HeaderField(name, value string) Matcher {
+	return func(r io.Reader) bool {
+		return matchHTTP1Field(r, name, func(gotValue string) bool {
+			return gotValue == value
+		})
+	}
+}
+
+// HTTP1HeaderFieldPrefix returns a matcher matching the header fields of the
+// first request of an HTTP 1 connection. If the header with key name has a
+// value prefixed with valuePrefix, this will match.
+func HTTP1HeaderFieldPrefix(name, valuePrefix string) Matcher {
+	return func(r io.Reader) bool {
+		return matchHTTP1Field(r, name, func(gotValue string) bool {
+			return strings.HasPrefix(gotValue, valuePrefix)
+		})
+	}
+}
+
+// HTTP2HeaderField returns a matcher matching the header fields of the first
+// headers frame.
+func HTTP2HeaderField(name, value string) Matcher {
+	return func(r io.Reader) bool {
+		return matchHTTP2Field(ioutil.Discard, r, name, func(gotValue string) bool {
+			return gotValue == value
+		})
+	}
+}
+
+// HTTP2HeaderFieldPrefix returns a matcher matching the header fields of the
+// first headers frame. If the header with key name has a value prefixed with
+// valuePrefix, this will match.
+func HTTP2HeaderFieldPrefix(name, valuePrefix string) Matcher {
+	return func(r io.Reader) bool {
+		return matchHTTP2Field(ioutil.Discard, r, name, func(gotValue string) bool {
+			return strings.HasPrefix(gotValue, valuePrefix)
+		})
+	}
+}
+
+// HTTP2MatchHeaderFieldSendSettings matches the header field and writes the
+// settings to the server. Prefer HTTP2HeaderField over this one, if the client
+// does not block on receiving a SETTING frame.
+func HTTP2MatchHeaderFieldSendSettings(name, value string) MatchWriter {
+	return func(w io.Writer, r io.Reader) bool {
+		return matchHTTP2Field(w, r, name, func(gotValue string) bool {
+			return gotValue == value
+		})
+	}
+}
+
+// HTTP2MatchHeaderFieldPrefixSendSettings matches the header field prefix
+// and writes the settings to the server. Prefer HTTP2HeaderFieldPrefix over
+// this one, if the client does not block on receiving a SETTING frame.
+func HTTP2MatchHeaderFieldPrefixSendSettings(name, valuePrefix string) MatchWriter {
+	return func(w io.Writer, r io.Reader) bool {
+		return matchHTTP2Field(w, r, name, func(gotValue string) bool {
+			return strings.HasPrefix(gotValue, valuePrefix)
+		})
+	}
+}
+
+func hasHTTP2Preface(r io.Reader) bool {
+	var b [len(http2.ClientPreface)]byte
+	last := 0
+
+	for {
+		n, err := r.Read(b[last:])
+		if err != nil {
+			return false
+		}
+
+		last += n
+		eq := string(b[:last]) == http2.ClientPreface[:last]
+		if last == len(http2.ClientPreface) {
+			return eq
+		}
+		if !eq {
+			return false
+		}
+	}
+}
+
+func matchHTTP1Field(r io.Reader, name string, matches func(string) bool) (matched bool) {
+	req, err := http.ReadRequest(bufio.NewReader(r))
+	if err != nil {
+		return false
+	}
+
+	return matches(req.Header.Get(name))
+}
+
+func matchHTTP2Field(w io.Writer, r io.Reader, name string, matches func(string) bool) (matched bool) {
+	if !hasHTTP2Preface(r) {
+		return false
+	}
+
+	done := false
+	framer := http2.NewFramer(w, r)
+	hdec := hpack.NewDecoder(uint32(4<<10), func(hf hpack.HeaderField) {
+		if hf.Name == name {
+			done = true
+			if matches(hf.Value) {
+				matched = true
+			}
+		}
+	})
+	for {
+		f, err := framer.ReadFrame()
+		if err != nil {
+			return false
+		}
+
+		switch f := f.(type) {
+		case *http2.SettingsFrame:
+			// Sender acknoweldged the SETTINGS frame. No need to write
+			// SETTINGS again.
+			if f.IsAck() {
+				break
+			}
+			if err := framer.WriteSettings(); err != nil {
+				return false
+			}
+		case *http2.ContinuationFrame:
+			if _, err := hdec.Write(f.HeaderBlockFragment()); err != nil {
+				return false
+			}
+			done = done || f.FrameHeader.Flags&http2.FlagHeadersEndHeaders != 0
+		case *http2.HeadersFrame:
+			if _, err := hdec.Write(f.HeaderBlockFragment()); err != nil {
+				return false
+			}
+			done = done || f.FrameHeader.Flags&http2.FlagHeadersEndHeaders != 0
+		}
+
+		if done {
+			return matched
+		}
+	}
+}
diff --git a/vendor/github.com/soheilhy/cmux/patricia.go b/vendor/github.com/soheilhy/cmux/patricia.go
new file mode 100644
index 0000000..c3e3d85
--- /dev/null
+++ b/vendor/github.com/soheilhy/cmux/patricia.go
@@ -0,0 +1,179 @@
+// Copyright 2016 The CMux Authors. All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+// implied. See the License for the specific language governing
+// permissions and limitations under the License.
+
+package cmux
+
+import (
+	"bytes"
+	"io"
+)
+
+// patriciaTree is a simple patricia tree that handles []byte instead of string
+// and cannot be changed after instantiation.
+type patriciaTree struct {
+	root     *ptNode
+	maxDepth int // max depth of the tree.
+}
+
+func newPatriciaTree(bs ...[]byte) *patriciaTree {
+	max := 0
+	for _, b := range bs {
+		if max < len(b) {
+			max = len(b)
+		}
+	}
+	return &patriciaTree{
+		root:     newNode(bs),
+		maxDepth: max + 1,
+	}
+}
+
+func newPatriciaTreeString(strs ...string) *patriciaTree {
+	b := make([][]byte, len(strs))
+	for i, s := range strs {
+		b[i] = []byte(s)
+	}
+	return newPatriciaTree(b...)
+}
+
+func (t *patriciaTree) matchPrefix(r io.Reader) bool {
+	buf := make([]byte, t.maxDepth)
+	n, _ := io.ReadFull(r, buf)
+	return t.root.match(buf[:n], true)
+}
+
+func (t *patriciaTree) match(r io.Reader) bool {
+	buf := make([]byte, t.maxDepth)
+	n, _ := io.ReadFull(r, buf)
+	return t.root.match(buf[:n], false)
+}
+
+type ptNode struct {
+	prefix   []byte
+	next     map[byte]*ptNode
+	terminal bool
+}
+
+func newNode(strs [][]byte) *ptNode {
+	if len(strs) == 0 {
+		return &ptNode{
+			prefix:   []byte{},
+			terminal: true,
+		}
+	}
+
+	if len(strs) == 1 {
+		return &ptNode{
+			prefix:   strs[0],
+			terminal: true,
+		}
+	}
+
+	p, strs := splitPrefix(strs)
+	n := &ptNode{
+		prefix: p,
+	}
+
+	nexts := make(map[byte][][]byte)
+	for _, s := range strs {
+		if len(s) == 0 {
+			n.terminal = true
+			continue
+		}
+		nexts[s[0]] = append(nexts[s[0]], s[1:])
+	}
+
+	n.next = make(map[byte]*ptNode)
+	for first, rests := range nexts {
+		n.next[first] = newNode(rests)
+	}
+
+	return n
+}
+
+func splitPrefix(bss [][]byte) (prefix []byte, rest [][]byte) {
+	if len(bss) == 0 || len(bss[0]) == 0 {
+		return prefix, bss
+	}
+
+	if len(bss) == 1 {
+		return bss[0], [][]byte{{}}
+	}
+
+	for i := 0; ; i++ {
+		var cur byte
+		eq := true
+		for j, b := range bss {
+			if len(b) <= i {
+				eq = false
+				break
+			}
+
+			if j == 0 {
+				cur = b[i]
+				continue
+			}
+
+			if cur != b[i] {
+				eq = false
+				break
+			}
+		}
+
+		if !eq {
+			break
+		}
+
+		prefix = append(prefix, cur)
+	}
+
+	rest = make([][]byte, 0, len(bss))
+	for _, b := range bss {
+		rest = append(rest, b[len(prefix):])
+	}
+
+	return prefix, rest
+}
+
+func (n *ptNode) match(b []byte, prefix bool) bool {
+	l := len(n.prefix)
+	if l > 0 {
+		if l > len(b) {
+			l = len(b)
+		}
+		if !bytes.Equal(b[:l], n.prefix) {
+			return false
+		}
+	}
+
+	if n.terminal && (prefix || len(n.prefix) == len(b)) {
+		return true
+	}
+
+	if l >= len(b) {
+		return false
+	}
+
+	nextN, ok := n.next[b[l]]
+	if !ok {
+		return false
+	}
+
+	if l == len(b) {
+		b = b[l:l]
+	} else {
+		b = b[l+1:]
+	}
+	return nextN.match(b, prefix)
+}
diff --git a/vendor/github.com/spf13/pflag/.gitignore b/vendor/github.com/spf13/pflag/.gitignore
new file mode 100644
index 0000000..c3da290
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/.gitignore
@@ -0,0 +1,2 @@
+.idea/*
+
diff --git a/vendor/github.com/spf13/pflag/.travis.yml b/vendor/github.com/spf13/pflag/.travis.yml
new file mode 100644
index 0000000..f8a63b3
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/.travis.yml
@@ -0,0 +1,21 @@
+sudo: false
+
+language: go
+
+go:
+  - 1.7.3
+  - 1.8.1
+  - tip
+
+matrix:
+  allow_failures:
+    - go: tip
+
+install:
+  - go get github.com/golang/lint/golint
+  - export PATH=$GOPATH/bin:$PATH
+  - go install ./...
+
+script:
+  - verify/all.sh -v
+  - go test ./...
diff --git a/vendor/github.com/spf13/pflag/LICENSE b/vendor/github.com/spf13/pflag/LICENSE
new file mode 100644
index 0000000..63ed1cf
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/LICENSE
@@ -0,0 +1,28 @@
+Copyright (c) 2012 Alex Ogier. All rights reserved.
+Copyright (c) 2012 The Go Authors. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+   * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+   * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+   * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/vendor/github.com/spf13/pflag/README.md b/vendor/github.com/spf13/pflag/README.md
new file mode 100644
index 0000000..b052414
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/README.md
@@ -0,0 +1,296 @@
+[![Build Status](https://travis-ci.org/spf13/pflag.svg?branch=master)](https://travis-ci.org/spf13/pflag)
+[![Go Report Card](https://goreportcard.com/badge/github.com/spf13/pflag)](https://goreportcard.com/report/github.com/spf13/pflag)
+[![GoDoc](https://godoc.org/github.com/spf13/pflag?status.svg)](https://godoc.org/github.com/spf13/pflag)
+
+## Description
+
+pflag is a drop-in replacement for Go's flag package, implementing
+POSIX/GNU-style --flags.
+
+pflag is compatible with the [GNU extensions to the POSIX recommendations
+for command-line options][1]. For a more precise description, see the
+"Command-line flag syntax" section below.
+
+[1]: http://www.gnu.org/software/libc/manual/html_node/Argument-Syntax.html
+
+pflag is available under the same style of BSD license as the Go language,
+which can be found in the LICENSE file.
+
+## Installation
+
+pflag is available using the standard `go get` command.
+
+Install by running:
+
+    go get github.com/spf13/pflag
+
+Run tests by running:
+
+    go test github.com/spf13/pflag
+
+## Usage
+
+pflag is a drop-in replacement of Go's native flag package. If you import
+pflag under the name "flag" then all code should continue to function
+with no changes.
+
+``` go
+import flag "github.com/spf13/pflag"
+```
+
+There is one exception to this: if you directly instantiate the Flag struct
+there is one more field "Shorthand" that you will need to set.
+Most code never instantiates this struct directly, and instead uses
+functions such as String(), BoolVar(), and Var(), and is therefore
+unaffected.
+
+Define flags using flag.String(), Bool(), Int(), etc.
+
+This declares an integer flag, -flagname, stored in the pointer ip, with type *int.
+
+``` go
+var ip *int = flag.Int("flagname", 1234, "help message for flagname")
+```
+
+If you like, you can bind the flag to a variable using the Var() functions.
+
+``` go
+var flagvar int
+func init() {
+    flag.IntVar(&flagvar, "flagname", 1234, "help message for flagname")
+}
+```
+
+Or you can create custom flags that satisfy the Value interface (with
+pointer receivers) and couple them to flag parsing by
+
+``` go
+flag.Var(&flagVal, "name", "help message for flagname")
+```
+
+For such flags, the default value is just the initial value of the variable.
+
+After all flags are defined, call
+
+``` go
+flag.Parse()
+```
+
+to parse the command line into the defined flags.
+
+Flags may then be used directly. If you're using the flags themselves,
+they are all pointers; if you bind to variables, they're values.
+
+``` go
+fmt.Println("ip has value ", *ip)
+fmt.Println("flagvar has value ", flagvar)
+```
+
+There are helpers function to get values later if you have the FlagSet but
+it was difficult to keep up with all of the flag pointers in your code.
+If you have a pflag.FlagSet with a flag called 'flagname' of type int you
+can use GetInt() to get the int value. But notice that 'flagname' must exist
+and it must be an int. GetString("flagname") will fail.
+
+``` go
+i, err := flagset.GetInt("flagname")
+```
+
+After parsing, the arguments after the flag are available as the
+slice flag.Args() or individually as flag.Arg(i).
+The arguments are indexed from 0 through flag.NArg()-1.
+
+The pflag package also defines some new functions that are not in flag,
+that give one-letter shorthands for flags. You can use these by appending
+'P' to the name of any function that defines a flag.
+
+``` go
+var ip = flag.IntP("flagname", "f", 1234, "help message")
+var flagvar bool
+func init() {
+	flag.BoolVarP(&flagvar, "boolname", "b", true, "help message")
+}
+flag.VarP(&flagVal, "varname", "v", "help message")
+```
+
+Shorthand letters can be used with single dashes on the command line.
+Boolean shorthand flags can be combined with other shorthand flags.
+
+The default set of command-line flags is controlled by
+top-level functions.  The FlagSet type allows one to define
+independent sets of flags, such as to implement subcommands
+in a command-line interface. The methods of FlagSet are
+analogous to the top-level functions for the command-line
+flag set.
+
+## Setting no option default values for flags
+
+After you create a flag it is possible to set the pflag.NoOptDefVal for
+the given flag. Doing this changes the meaning of the flag slightly. If
+a flag has a NoOptDefVal and the flag is set on the command line without
+an option the flag will be set to the NoOptDefVal. For example given:
+
+``` go
+var ip = flag.IntP("flagname", "f", 1234, "help message")
+flag.Lookup("flagname").NoOptDefVal = "4321"
+```
+
+Would result in something like
+
+| Parsed Arguments | Resulting Value |
+| -------------    | -------------   |
+| --flagname=1357  | ip=1357         |
+| --flagname       | ip=4321         |
+| [nothing]        | ip=1234         |
+
+## Command line flag syntax
+
+```
+--flag    // boolean flags, or flags with no option default values
+--flag x  // only on flags without a default value
+--flag=x
+```
+
+Unlike the flag package, a single dash before an option means something
+different than a double dash. Single dashes signify a series of shorthand
+letters for flags. All but the last shorthand letter must be boolean flags
+or a flag with a default value
+
+```
+// boolean or flags where the 'no option default value' is set
+-f
+-f=true
+-abc
+but
+-b true is INVALID
+
+// non-boolean and flags without a 'no option default value'
+-n 1234
+-n=1234
+-n1234
+
+// mixed
+-abcs "hello"
+-absd="hello"
+-abcs1234
+```
+
+Flag parsing stops after the terminator "--". Unlike the flag package,
+flags can be interspersed with arguments anywhere on the command line
+before this terminator.
+
+Integer flags accept 1234, 0664, 0x1234 and may be negative.
+Boolean flags (in their long form) accept 1, 0, t, f, true, false,
+TRUE, FALSE, True, False.
+Duration flags accept any input valid for time.ParseDuration.
+
+## Mutating or "Normalizing" Flag names
+
+It is possible to set a custom flag name 'normalization function.' It allows flag names to be mutated both when created in the code and when used on the command line to some 'normalized' form. The 'normalized' form is used for comparison. Two examples of using the custom normalization func follow.
+
+**Example #1**: You want -, _, and . in flags to compare the same. aka --my-flag == --my_flag == --my.flag
+
+``` go
+func wordSepNormalizeFunc(f *pflag.FlagSet, name string) pflag.NormalizedName {
+	from := []string{"-", "_"}
+	to := "."
+	for _, sep := range from {
+		name = strings.Replace(name, sep, to, -1)
+	}
+	return pflag.NormalizedName(name)
+}
+
+myFlagSet.SetNormalizeFunc(wordSepNormalizeFunc)
+```
+
+**Example #2**: You want to alias two flags. aka --old-flag-name == --new-flag-name
+
+``` go
+func aliasNormalizeFunc(f *pflag.FlagSet, name string) pflag.NormalizedName {
+	switch name {
+	case "old-flag-name":
+		name = "new-flag-name"
+		break
+	}
+	return pflag.NormalizedName(name)
+}
+
+myFlagSet.SetNormalizeFunc(aliasNormalizeFunc)
+```
+
+## Deprecating a flag or its shorthand
+It is possible to deprecate a flag, or just its shorthand. Deprecating a flag/shorthand hides it from help text and prints a usage message when the deprecated flag/shorthand is used.
+
+**Example #1**: You want to deprecate a flag named "badflag" as well as inform the users what flag they should use instead.
+```go
+// deprecate a flag by specifying its name and a usage message
+flags.MarkDeprecated("badflag", "please use --good-flag instead")
+```
+This hides "badflag" from help text, and prints `Flag --badflag has been deprecated, please use --good-flag instead` when "badflag" is used.
+
+**Example #2**: You want to keep a flag name "noshorthandflag" but deprecate its shortname "n".
+```go
+// deprecate a flag shorthand by specifying its flag name and a usage message
+flags.MarkShorthandDeprecated("noshorthandflag", "please use --noshorthandflag only")
+```
+This hides the shortname "n" from help text, and prints `Flag shorthand -n has been deprecated, please use --noshorthandflag only` when the shorthand "n" is used.
+
+Note that usage message is essential here, and it should not be empty.
+
+## Hidden flags
+It is possible to mark a flag as hidden, meaning it will still function as normal, however will not show up in usage/help text.
+
+**Example**: You have a flag named "secretFlag" that you need for internal use only and don't want it showing up in help text, or for its usage text to be available.
+```go
+// hide a flag by specifying its name
+flags.MarkHidden("secretFlag")
+```
+
+## Disable sorting of flags
+`pflag` allows you to disable sorting of flags for help and usage message.
+
+**Example**:
+```go
+flags.BoolP("verbose", "v", false, "verbose output")
+flags.String("coolflag", "yeaah", "it's really cool flag")
+flags.Int("usefulflag", 777, "sometimes it's very useful")
+flags.SortFlags = false
+flags.PrintDefaults()
+```
+**Output**:
+```
+  -v, --verbose           verbose output
+      --coolflag string   it's really cool flag (default "yeaah")
+      --usefulflag int    sometimes it's very useful (default 777)
+```
+
+
+## Supporting Go flags when using pflag
+In order to support flags defined using Go's `flag` package, they must be added to the `pflag` flagset. This is usually necessary
+to support flags defined by third-party dependencies (e.g. `golang/glog`).
+
+**Example**: You want to add the Go flags to the `CommandLine` flagset
+```go
+import (
+	goflag "flag"
+	flag "github.com/spf13/pflag"
+)
+
+var ip *int = flag.Int("flagname", 1234, "help message for flagname")
+
+func main() {
+	flag.CommandLine.AddGoFlagSet(goflag.CommandLine)
+	flag.Parse()
+}
+```
+
+## More info
+
+You can see the full reference documentation of the pflag package
+[at godoc.org][3], or through go's standard documentation system by
+running `godoc -http=:6060` and browsing to
+[http://localhost:6060/pkg/github.com/spf13/pflag][2] after
+installation.
+
+[2]: http://localhost:6060/pkg/github.com/spf13/pflag
+[3]: http://godoc.org/github.com/spf13/pflag
diff --git a/vendor/github.com/spf13/pflag/bool.go b/vendor/github.com/spf13/pflag/bool.go
new file mode 100644
index 0000000..c4c5c0b
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/bool.go
@@ -0,0 +1,94 @@
+package pflag
+
+import "strconv"
+
+// optional interface to indicate boolean flags that can be
+// supplied without "=value" text
+type boolFlag interface {
+	Value
+	IsBoolFlag() bool
+}
+
+// -- bool Value
+type boolValue bool
+
+func newBoolValue(val bool, p *bool) *boolValue {
+	*p = val
+	return (*boolValue)(p)
+}
+
+func (b *boolValue) Set(s string) error {
+	v, err := strconv.ParseBool(s)
+	*b = boolValue(v)
+	return err
+}
+
+func (b *boolValue) Type() string {
+	return "bool"
+}
+
+func (b *boolValue) String() string { return strconv.FormatBool(bool(*b)) }
+
+func (b *boolValue) IsBoolFlag() bool { return true }
+
+func boolConv(sval string) (interface{}, error) {
+	return strconv.ParseBool(sval)
+}
+
+// GetBool return the bool value of a flag with the given name
+func (f *FlagSet) GetBool(name string) (bool, error) {
+	val, err := f.getFlagType(name, "bool", boolConv)
+	if err != nil {
+		return false, err
+	}
+	return val.(bool), nil
+}
+
+// BoolVar defines a bool flag with specified name, default value, and usage string.
+// The argument p points to a bool variable in which to store the value of the flag.
+func (f *FlagSet) BoolVar(p *bool, name string, value bool, usage string) {
+	f.BoolVarP(p, name, "", value, usage)
+}
+
+// BoolVarP is like BoolVar, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) BoolVarP(p *bool, name, shorthand string, value bool, usage string) {
+	flag := f.VarPF(newBoolValue(value, p), name, shorthand, usage)
+	flag.NoOptDefVal = "true"
+}
+
+// BoolVar defines a bool flag with specified name, default value, and usage string.
+// The argument p points to a bool variable in which to store the value of the flag.
+func BoolVar(p *bool, name string, value bool, usage string) {
+	BoolVarP(p, name, "", value, usage)
+}
+
+// BoolVarP is like BoolVar, but accepts a shorthand letter that can be used after a single dash.
+func BoolVarP(p *bool, name, shorthand string, value bool, usage string) {
+	flag := CommandLine.VarPF(newBoolValue(value, p), name, shorthand, usage)
+	flag.NoOptDefVal = "true"
+}
+
+// Bool defines a bool flag with specified name, default value, and usage string.
+// The return value is the address of a bool variable that stores the value of the flag.
+func (f *FlagSet) Bool(name string, value bool, usage string) *bool {
+	return f.BoolP(name, "", value, usage)
+}
+
+// BoolP is like Bool, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) BoolP(name, shorthand string, value bool, usage string) *bool {
+	p := new(bool)
+	f.BoolVarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// Bool defines a bool flag with specified name, default value, and usage string.
+// The return value is the address of a bool variable that stores the value of the flag.
+func Bool(name string, value bool, usage string) *bool {
+	return BoolP(name, "", value, usage)
+}
+
+// BoolP is like Bool, but accepts a shorthand letter that can be used after a single dash.
+func BoolP(name, shorthand string, value bool, usage string) *bool {
+	b := CommandLine.BoolP(name, shorthand, value, usage)
+	return b
+}
diff --git a/vendor/github.com/spf13/pflag/bool_slice.go b/vendor/github.com/spf13/pflag/bool_slice.go
new file mode 100644
index 0000000..5af02f1
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/bool_slice.go
@@ -0,0 +1,147 @@
+package pflag
+
+import (
+	"io"
+	"strconv"
+	"strings"
+)
+
+// -- boolSlice Value
+type boolSliceValue struct {
+	value   *[]bool
+	changed bool
+}
+
+func newBoolSliceValue(val []bool, p *[]bool) *boolSliceValue {
+	bsv := new(boolSliceValue)
+	bsv.value = p
+	*bsv.value = val
+	return bsv
+}
+
+// Set converts, and assigns, the comma-separated boolean argument string representation as the []bool value of this flag.
+// If Set is called on a flag that already has a []bool assigned, the newly converted values will be appended.
+func (s *boolSliceValue) Set(val string) error {
+
+	// remove all quote characters
+	rmQuote := strings.NewReplacer(`"`, "", `'`, "", "`", "")
+
+	// read flag arguments with CSV parser
+	boolStrSlice, err := readAsCSV(rmQuote.Replace(val))
+	if err != nil && err != io.EOF {
+		return err
+	}
+
+	// parse boolean values into slice
+	out := make([]bool, 0, len(boolStrSlice))
+	for _, boolStr := range boolStrSlice {
+		b, err := strconv.ParseBool(strings.TrimSpace(boolStr))
+		if err != nil {
+			return err
+		}
+		out = append(out, b)
+	}
+
+	if !s.changed {
+		*s.value = out
+	} else {
+		*s.value = append(*s.value, out...)
+	}
+
+	s.changed = true
+
+	return nil
+}
+
+// Type returns a string that uniquely represents this flag's type.
+func (s *boolSliceValue) Type() string {
+	return "boolSlice"
+}
+
+// String defines a "native" format for this boolean slice flag value.
+func (s *boolSliceValue) String() string {
+
+	boolStrSlice := make([]string, len(*s.value))
+	for i, b := range *s.value {
+		boolStrSlice[i] = strconv.FormatBool(b)
+	}
+
+	out, _ := writeAsCSV(boolStrSlice)
+
+	return "[" + out + "]"
+}
+
+func boolSliceConv(val string) (interface{}, error) {
+	val = strings.Trim(val, "[]")
+	// Empty string would cause a slice with one (empty) entry
+	if len(val) == 0 {
+		return []bool{}, nil
+	}
+	ss := strings.Split(val, ",")
+	out := make([]bool, len(ss))
+	for i, t := range ss {
+		var err error
+		out[i], err = strconv.ParseBool(t)
+		if err != nil {
+			return nil, err
+		}
+	}
+	return out, nil
+}
+
+// GetBoolSlice returns the []bool value of a flag with the given name.
+func (f *FlagSet) GetBoolSlice(name string) ([]bool, error) {
+	val, err := f.getFlagType(name, "boolSlice", boolSliceConv)
+	if err != nil {
+		return []bool{}, err
+	}
+	return val.([]bool), nil
+}
+
+// BoolSliceVar defines a boolSlice flag with specified name, default value, and usage string.
+// The argument p points to a []bool variable in which to store the value of the flag.
+func (f *FlagSet) BoolSliceVar(p *[]bool, name string, value []bool, usage string) {
+	f.VarP(newBoolSliceValue(value, p), name, "", usage)
+}
+
+// BoolSliceVarP is like BoolSliceVar, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) BoolSliceVarP(p *[]bool, name, shorthand string, value []bool, usage string) {
+	f.VarP(newBoolSliceValue(value, p), name, shorthand, usage)
+}
+
+// BoolSliceVar defines a []bool flag with specified name, default value, and usage string.
+// The argument p points to a []bool variable in which to store the value of the flag.
+func BoolSliceVar(p *[]bool, name string, value []bool, usage string) {
+	CommandLine.VarP(newBoolSliceValue(value, p), name, "", usage)
+}
+
+// BoolSliceVarP is like BoolSliceVar, but accepts a shorthand letter that can be used after a single dash.
+func BoolSliceVarP(p *[]bool, name, shorthand string, value []bool, usage string) {
+	CommandLine.VarP(newBoolSliceValue(value, p), name, shorthand, usage)
+}
+
+// BoolSlice defines a []bool flag with specified name, default value, and usage string.
+// The return value is the address of a []bool variable that stores the value of the flag.
+func (f *FlagSet) BoolSlice(name string, value []bool, usage string) *[]bool {
+	p := []bool{}
+	f.BoolSliceVarP(&p, name, "", value, usage)
+	return &p
+}
+
+// BoolSliceP is like BoolSlice, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) BoolSliceP(name, shorthand string, value []bool, usage string) *[]bool {
+	p := []bool{}
+	f.BoolSliceVarP(&p, name, shorthand, value, usage)
+	return &p
+}
+
+// BoolSlice defines a []bool flag with specified name, default value, and usage string.
+// The return value is the address of a []bool variable that stores the value of the flag.
+func BoolSlice(name string, value []bool, usage string) *[]bool {
+	return CommandLine.BoolSliceP(name, "", value, usage)
+}
+
+// BoolSliceP is like BoolSlice, but accepts a shorthand letter that can be used after a single dash.
+func BoolSliceP(name, shorthand string, value []bool, usage string) *[]bool {
+	return CommandLine.BoolSliceP(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/bytes.go b/vendor/github.com/spf13/pflag/bytes.go
new file mode 100644
index 0000000..67d5304
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/bytes.go
@@ -0,0 +1,209 @@
+package pflag
+
+import (
+	"encoding/base64"
+	"encoding/hex"
+	"fmt"
+	"strings"
+)
+
+// BytesHex adapts []byte for use as a flag. Value of flag is HEX encoded
+type bytesHexValue []byte
+
+// String implements pflag.Value.String.
+func (bytesHex bytesHexValue) String() string {
+	return fmt.Sprintf("%X", []byte(bytesHex))
+}
+
+// Set implements pflag.Value.Set.
+func (bytesHex *bytesHexValue) Set(value string) error {
+	bin, err := hex.DecodeString(strings.TrimSpace(value))
+
+	if err != nil {
+		return err
+	}
+
+	*bytesHex = bin
+
+	return nil
+}
+
+// Type implements pflag.Value.Type.
+func (*bytesHexValue) Type() string {
+	return "bytesHex"
+}
+
+func newBytesHexValue(val []byte, p *[]byte) *bytesHexValue {
+	*p = val
+	return (*bytesHexValue)(p)
+}
+
+func bytesHexConv(sval string) (interface{}, error) {
+
+	bin, err := hex.DecodeString(sval)
+
+	if err == nil {
+		return bin, nil
+	}
+
+	return nil, fmt.Errorf("invalid string being converted to Bytes: %s %s", sval, err)
+}
+
+// GetBytesHex return the []byte value of a flag with the given name
+func (f *FlagSet) GetBytesHex(name string) ([]byte, error) {
+	val, err := f.getFlagType(name, "bytesHex", bytesHexConv)
+
+	if err != nil {
+		return []byte{}, err
+	}
+
+	return val.([]byte), nil
+}
+
+// BytesHexVar defines an []byte flag with specified name, default value, and usage string.
+// The argument p points to an []byte variable in which to store the value of the flag.
+func (f *FlagSet) BytesHexVar(p *[]byte, name string, value []byte, usage string) {
+	f.VarP(newBytesHexValue(value, p), name, "", usage)
+}
+
+// BytesHexVarP is like BytesHexVar, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) BytesHexVarP(p *[]byte, name, shorthand string, value []byte, usage string) {
+	f.VarP(newBytesHexValue(value, p), name, shorthand, usage)
+}
+
+// BytesHexVar defines an []byte flag with specified name, default value, and usage string.
+// The argument p points to an []byte variable in which to store the value of the flag.
+func BytesHexVar(p *[]byte, name string, value []byte, usage string) {
+	CommandLine.VarP(newBytesHexValue(value, p), name, "", usage)
+}
+
+// BytesHexVarP is like BytesHexVar, but accepts a shorthand letter that can be used after a single dash.
+func BytesHexVarP(p *[]byte, name, shorthand string, value []byte, usage string) {
+	CommandLine.VarP(newBytesHexValue(value, p), name, shorthand, usage)
+}
+
+// BytesHex defines an []byte flag with specified name, default value, and usage string.
+// The return value is the address of an []byte variable that stores the value of the flag.
+func (f *FlagSet) BytesHex(name string, value []byte, usage string) *[]byte {
+	p := new([]byte)
+	f.BytesHexVarP(p, name, "", value, usage)
+	return p
+}
+
+// BytesHexP is like BytesHex, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) BytesHexP(name, shorthand string, value []byte, usage string) *[]byte {
+	p := new([]byte)
+	f.BytesHexVarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// BytesHex defines an []byte flag with specified name, default value, and usage string.
+// The return value is the address of an []byte variable that stores the value of the flag.
+func BytesHex(name string, value []byte, usage string) *[]byte {
+	return CommandLine.BytesHexP(name, "", value, usage)
+}
+
+// BytesHexP is like BytesHex, but accepts a shorthand letter that can be used after a single dash.
+func BytesHexP(name, shorthand string, value []byte, usage string) *[]byte {
+	return CommandLine.BytesHexP(name, shorthand, value, usage)
+}
+
+// BytesBase64 adapts []byte for use as a flag. Value of flag is Base64 encoded
+type bytesBase64Value []byte
+
+// String implements pflag.Value.String.
+func (bytesBase64 bytesBase64Value) String() string {
+	return base64.StdEncoding.EncodeToString([]byte(bytesBase64))
+}
+
+// Set implements pflag.Value.Set.
+func (bytesBase64 *bytesBase64Value) Set(value string) error {
+	bin, err := base64.StdEncoding.DecodeString(strings.TrimSpace(value))
+
+	if err != nil {
+		return err
+	}
+
+	*bytesBase64 = bin
+
+	return nil
+}
+
+// Type implements pflag.Value.Type.
+func (*bytesBase64Value) Type() string {
+	return "bytesBase64"
+}
+
+func newBytesBase64Value(val []byte, p *[]byte) *bytesBase64Value {
+	*p = val
+	return (*bytesBase64Value)(p)
+}
+
+func bytesBase64ValueConv(sval string) (interface{}, error) {
+
+	bin, err := base64.StdEncoding.DecodeString(sval)
+	if err == nil {
+		return bin, nil
+	}
+
+	return nil, fmt.Errorf("invalid string being converted to Bytes: %s %s", sval, err)
+}
+
+// GetBytesBase64 return the []byte value of a flag with the given name
+func (f *FlagSet) GetBytesBase64(name string) ([]byte, error) {
+	val, err := f.getFlagType(name, "bytesBase64", bytesBase64ValueConv)
+
+	if err != nil {
+		return []byte{}, err
+	}
+
+	return val.([]byte), nil
+}
+
+// BytesBase64Var defines an []byte flag with specified name, default value, and usage string.
+// The argument p points to an []byte variable in which to store the value of the flag.
+func (f *FlagSet) BytesBase64Var(p *[]byte, name string, value []byte, usage string) {
+	f.VarP(newBytesBase64Value(value, p), name, "", usage)
+}
+
+// BytesBase64VarP is like BytesBase64Var, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) BytesBase64VarP(p *[]byte, name, shorthand string, value []byte, usage string) {
+	f.VarP(newBytesBase64Value(value, p), name, shorthand, usage)
+}
+
+// BytesBase64Var defines an []byte flag with specified name, default value, and usage string.
+// The argument p points to an []byte variable in which to store the value of the flag.
+func BytesBase64Var(p *[]byte, name string, value []byte, usage string) {
+	CommandLine.VarP(newBytesBase64Value(value, p), name, "", usage)
+}
+
+// BytesBase64VarP is like BytesBase64Var, but accepts a shorthand letter that can be used after a single dash.
+func BytesBase64VarP(p *[]byte, name, shorthand string, value []byte, usage string) {
+	CommandLine.VarP(newBytesBase64Value(value, p), name, shorthand, usage)
+}
+
+// BytesBase64 defines an []byte flag with specified name, default value, and usage string.
+// The return value is the address of an []byte variable that stores the value of the flag.
+func (f *FlagSet) BytesBase64(name string, value []byte, usage string) *[]byte {
+	p := new([]byte)
+	f.BytesBase64VarP(p, name, "", value, usage)
+	return p
+}
+
+// BytesBase64P is like BytesBase64, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) BytesBase64P(name, shorthand string, value []byte, usage string) *[]byte {
+	p := new([]byte)
+	f.BytesBase64VarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// BytesBase64 defines an []byte flag with specified name, default value, and usage string.
+// The return value is the address of an []byte variable that stores the value of the flag.
+func BytesBase64(name string, value []byte, usage string) *[]byte {
+	return CommandLine.BytesBase64P(name, "", value, usage)
+}
+
+// BytesBase64P is like BytesBase64, but accepts a shorthand letter that can be used after a single dash.
+func BytesBase64P(name, shorthand string, value []byte, usage string) *[]byte {
+	return CommandLine.BytesBase64P(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/count.go b/vendor/github.com/spf13/pflag/count.go
new file mode 100644
index 0000000..aa126e4
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/count.go
@@ -0,0 +1,96 @@
+package pflag
+
+import "strconv"
+
+// -- count Value
+type countValue int
+
+func newCountValue(val int, p *int) *countValue {
+	*p = val
+	return (*countValue)(p)
+}
+
+func (i *countValue) Set(s string) error {
+	// "+1" means that no specific value was passed, so increment
+	if s == "+1" {
+		*i = countValue(*i + 1)
+		return nil
+	}
+	v, err := strconv.ParseInt(s, 0, 0)
+	*i = countValue(v)
+	return err
+}
+
+func (i *countValue) Type() string {
+	return "count"
+}
+
+func (i *countValue) String() string { return strconv.Itoa(int(*i)) }
+
+func countConv(sval string) (interface{}, error) {
+	i, err := strconv.Atoi(sval)
+	if err != nil {
+		return nil, err
+	}
+	return i, nil
+}
+
+// GetCount return the int value of a flag with the given name
+func (f *FlagSet) GetCount(name string) (int, error) {
+	val, err := f.getFlagType(name, "count", countConv)
+	if err != nil {
+		return 0, err
+	}
+	return val.(int), nil
+}
+
+// CountVar defines a count flag with specified name, default value, and usage string.
+// The argument p points to an int variable in which to store the value of the flag.
+// A count flag will add 1 to its value evey time it is found on the command line
+func (f *FlagSet) CountVar(p *int, name string, usage string) {
+	f.CountVarP(p, name, "", usage)
+}
+
+// CountVarP is like CountVar only take a shorthand for the flag name.
+func (f *FlagSet) CountVarP(p *int, name, shorthand string, usage string) {
+	flag := f.VarPF(newCountValue(0, p), name, shorthand, usage)
+	flag.NoOptDefVal = "+1"
+}
+
+// CountVar like CountVar only the flag is placed on the CommandLine instead of a given flag set
+func CountVar(p *int, name string, usage string) {
+	CommandLine.CountVar(p, name, usage)
+}
+
+// CountVarP is like CountVar only take a shorthand for the flag name.
+func CountVarP(p *int, name, shorthand string, usage string) {
+	CommandLine.CountVarP(p, name, shorthand, usage)
+}
+
+// Count defines a count flag with specified name, default value, and usage string.
+// The return value is the address of an int variable that stores the value of the flag.
+// A count flag will add 1 to its value evey time it is found on the command line
+func (f *FlagSet) Count(name string, usage string) *int {
+	p := new(int)
+	f.CountVarP(p, name, "", usage)
+	return p
+}
+
+// CountP is like Count only takes a shorthand for the flag name.
+func (f *FlagSet) CountP(name, shorthand string, usage string) *int {
+	p := new(int)
+	f.CountVarP(p, name, shorthand, usage)
+	return p
+}
+
+// Count defines a count flag with specified name, default value, and usage string.
+// The return value is the address of an int variable that stores the value of the flag.
+// A count flag will add 1 to its value evey time it is found on the command line
+func Count(name string, usage string) *int {
+	return CommandLine.CountP(name, "", usage)
+}
+
+// CountP is like Count only takes a shorthand for the flag name.
+func CountP(name, shorthand string, usage string) *int {
+	return CommandLine.CountP(name, shorthand, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/duration.go b/vendor/github.com/spf13/pflag/duration.go
new file mode 100644
index 0000000..e9debef
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/duration.go
@@ -0,0 +1,86 @@
+package pflag
+
+import (
+	"time"
+)
+
+// -- time.Duration Value
+type durationValue time.Duration
+
+func newDurationValue(val time.Duration, p *time.Duration) *durationValue {
+	*p = val
+	return (*durationValue)(p)
+}
+
+func (d *durationValue) Set(s string) error {
+	v, err := time.ParseDuration(s)
+	*d = durationValue(v)
+	return err
+}
+
+func (d *durationValue) Type() string {
+	return "duration"
+}
+
+func (d *durationValue) String() string { return (*time.Duration)(d).String() }
+
+func durationConv(sval string) (interface{}, error) {
+	return time.ParseDuration(sval)
+}
+
+// GetDuration return the duration value of a flag with the given name
+func (f *FlagSet) GetDuration(name string) (time.Duration, error) {
+	val, err := f.getFlagType(name, "duration", durationConv)
+	if err != nil {
+		return 0, err
+	}
+	return val.(time.Duration), nil
+}
+
+// DurationVar defines a time.Duration flag with specified name, default value, and usage string.
+// The argument p points to a time.Duration variable in which to store the value of the flag.
+func (f *FlagSet) DurationVar(p *time.Duration, name string, value time.Duration, usage string) {
+	f.VarP(newDurationValue(value, p), name, "", usage)
+}
+
+// DurationVarP is like DurationVar, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) DurationVarP(p *time.Duration, name, shorthand string, value time.Duration, usage string) {
+	f.VarP(newDurationValue(value, p), name, shorthand, usage)
+}
+
+// DurationVar defines a time.Duration flag with specified name, default value, and usage string.
+// The argument p points to a time.Duration variable in which to store the value of the flag.
+func DurationVar(p *time.Duration, name string, value time.Duration, usage string) {
+	CommandLine.VarP(newDurationValue(value, p), name, "", usage)
+}
+
+// DurationVarP is like DurationVar, but accepts a shorthand letter that can be used after a single dash.
+func DurationVarP(p *time.Duration, name, shorthand string, value time.Duration, usage string) {
+	CommandLine.VarP(newDurationValue(value, p), name, shorthand, usage)
+}
+
+// Duration defines a time.Duration flag with specified name, default value, and usage string.
+// The return value is the address of a time.Duration variable that stores the value of the flag.
+func (f *FlagSet) Duration(name string, value time.Duration, usage string) *time.Duration {
+	p := new(time.Duration)
+	f.DurationVarP(p, name, "", value, usage)
+	return p
+}
+
+// DurationP is like Duration, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) DurationP(name, shorthand string, value time.Duration, usage string) *time.Duration {
+	p := new(time.Duration)
+	f.DurationVarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// Duration defines a time.Duration flag with specified name, default value, and usage string.
+// The return value is the address of a time.Duration variable that stores the value of the flag.
+func Duration(name string, value time.Duration, usage string) *time.Duration {
+	return CommandLine.DurationP(name, "", value, usage)
+}
+
+// DurationP is like Duration, but accepts a shorthand letter that can be used after a single dash.
+func DurationP(name, shorthand string, value time.Duration, usage string) *time.Duration {
+	return CommandLine.DurationP(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/duration_slice.go b/vendor/github.com/spf13/pflag/duration_slice.go
new file mode 100644
index 0000000..52c6b6d
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/duration_slice.go
@@ -0,0 +1,128 @@
+package pflag
+
+import (
+	"fmt"
+	"strings"
+	"time"
+)
+
+// -- durationSlice Value
+type durationSliceValue struct {
+	value   *[]time.Duration
+	changed bool
+}
+
+func newDurationSliceValue(val []time.Duration, p *[]time.Duration) *durationSliceValue {
+	dsv := new(durationSliceValue)
+	dsv.value = p
+	*dsv.value = val
+	return dsv
+}
+
+func (s *durationSliceValue) Set(val string) error {
+	ss := strings.Split(val, ",")
+	out := make([]time.Duration, len(ss))
+	for i, d := range ss {
+		var err error
+		out[i], err = time.ParseDuration(d)
+		if err != nil {
+			return err
+		}
+
+	}
+	if !s.changed {
+		*s.value = out
+	} else {
+		*s.value = append(*s.value, out...)
+	}
+	s.changed = true
+	return nil
+}
+
+func (s *durationSliceValue) Type() string {
+	return "durationSlice"
+}
+
+func (s *durationSliceValue) String() string {
+	out := make([]string, len(*s.value))
+	for i, d := range *s.value {
+		out[i] = fmt.Sprintf("%s", d)
+	}
+	return "[" + strings.Join(out, ",") + "]"
+}
+
+func durationSliceConv(val string) (interface{}, error) {
+	val = strings.Trim(val, "[]")
+	// Empty string would cause a slice with one (empty) entry
+	if len(val) == 0 {
+		return []time.Duration{}, nil
+	}
+	ss := strings.Split(val, ",")
+	out := make([]time.Duration, len(ss))
+	for i, d := range ss {
+		var err error
+		out[i], err = time.ParseDuration(d)
+		if err != nil {
+			return nil, err
+		}
+
+	}
+	return out, nil
+}
+
+// GetDurationSlice returns the []time.Duration value of a flag with the given name
+func (f *FlagSet) GetDurationSlice(name string) ([]time.Duration, error) {
+	val, err := f.getFlagType(name, "durationSlice", durationSliceConv)
+	if err != nil {
+		return []time.Duration{}, err
+	}
+	return val.([]time.Duration), nil
+}
+
+// DurationSliceVar defines a durationSlice flag with specified name, default value, and usage string.
+// The argument p points to a []time.Duration variable in which to store the value of the flag.
+func (f *FlagSet) DurationSliceVar(p *[]time.Duration, name string, value []time.Duration, usage string) {
+	f.VarP(newDurationSliceValue(value, p), name, "", usage)
+}
+
+// DurationSliceVarP is like DurationSliceVar, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) DurationSliceVarP(p *[]time.Duration, name, shorthand string, value []time.Duration, usage string) {
+	f.VarP(newDurationSliceValue(value, p), name, shorthand, usage)
+}
+
+// DurationSliceVar defines a duration[] flag with specified name, default value, and usage string.
+// The argument p points to a duration[] variable in which to store the value of the flag.
+func DurationSliceVar(p *[]time.Duration, name string, value []time.Duration, usage string) {
+	CommandLine.VarP(newDurationSliceValue(value, p), name, "", usage)
+}
+
+// DurationSliceVarP is like DurationSliceVar, but accepts a shorthand letter that can be used after a single dash.
+func DurationSliceVarP(p *[]time.Duration, name, shorthand string, value []time.Duration, usage string) {
+	CommandLine.VarP(newDurationSliceValue(value, p), name, shorthand, usage)
+}
+
+// DurationSlice defines a []time.Duration flag with specified name, default value, and usage string.
+// The return value is the address of a []time.Duration variable that stores the value of the flag.
+func (f *FlagSet) DurationSlice(name string, value []time.Duration, usage string) *[]time.Duration {
+	p := []time.Duration{}
+	f.DurationSliceVarP(&p, name, "", value, usage)
+	return &p
+}
+
+// DurationSliceP is like DurationSlice, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) DurationSliceP(name, shorthand string, value []time.Duration, usage string) *[]time.Duration {
+	p := []time.Duration{}
+	f.DurationSliceVarP(&p, name, shorthand, value, usage)
+	return &p
+}
+
+// DurationSlice defines a []time.Duration flag with specified name, default value, and usage string.
+// The return value is the address of a []time.Duration variable that stores the value of the flag.
+func DurationSlice(name string, value []time.Duration, usage string) *[]time.Duration {
+	return CommandLine.DurationSliceP(name, "", value, usage)
+}
+
+// DurationSliceP is like DurationSlice, but accepts a shorthand letter that can be used after a single dash.
+func DurationSliceP(name, shorthand string, value []time.Duration, usage string) *[]time.Duration {
+	return CommandLine.DurationSliceP(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/flag.go b/vendor/github.com/spf13/pflag/flag.go
new file mode 100644
index 0000000..9beeda8
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/flag.go
@@ -0,0 +1,1227 @@
+// Copyright 2009 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+/*
+Package pflag is a drop-in replacement for Go's flag package, implementing
+POSIX/GNU-style --flags.
+
+pflag is compatible with the GNU extensions to the POSIX recommendations
+for command-line options. See
+http://www.gnu.org/software/libc/manual/html_node/Argument-Syntax.html
+
+Usage:
+
+pflag is a drop-in replacement of Go's native flag package. If you import
+pflag under the name "flag" then all code should continue to function
+with no changes.
+
+	import flag "github.com/spf13/pflag"
+
+There is one exception to this: if you directly instantiate the Flag struct
+there is one more field "Shorthand" that you will need to set.
+Most code never instantiates this struct directly, and instead uses
+functions such as String(), BoolVar(), and Var(), and is therefore
+unaffected.
+
+Define flags using flag.String(), Bool(), Int(), etc.
+
+This declares an integer flag, -flagname, stored in the pointer ip, with type *int.
+	var ip = flag.Int("flagname", 1234, "help message for flagname")
+If you like, you can bind the flag to a variable using the Var() functions.
+	var flagvar int
+	func init() {
+		flag.IntVar(&flagvar, "flagname", 1234, "help message for flagname")
+	}
+Or you can create custom flags that satisfy the Value interface (with
+pointer receivers) and couple them to flag parsing by
+	flag.Var(&flagVal, "name", "help message for flagname")
+For such flags, the default value is just the initial value of the variable.
+
+After all flags are defined, call
+	flag.Parse()
+to parse the command line into the defined flags.
+
+Flags may then be used directly. If you're using the flags themselves,
+they are all pointers; if you bind to variables, they're values.
+	fmt.Println("ip has value ", *ip)
+	fmt.Println("flagvar has value ", flagvar)
+
+After parsing, the arguments after the flag are available as the
+slice flag.Args() or individually as flag.Arg(i).
+The arguments are indexed from 0 through flag.NArg()-1.
+
+The pflag package also defines some new functions that are not in flag,
+that give one-letter shorthands for flags. You can use these by appending
+'P' to the name of any function that defines a flag.
+	var ip = flag.IntP("flagname", "f", 1234, "help message")
+	var flagvar bool
+	func init() {
+		flag.BoolVarP("boolname", "b", true, "help message")
+	}
+	flag.VarP(&flagVar, "varname", "v", 1234, "help message")
+Shorthand letters can be used with single dashes on the command line.
+Boolean shorthand flags can be combined with other shorthand flags.
+
+Command line flag syntax:
+	--flag    // boolean flags only
+	--flag=x
+
+Unlike the flag package, a single dash before an option means something
+different than a double dash. Single dashes signify a series of shorthand
+letters for flags. All but the last shorthand letter must be boolean flags.
+	// boolean flags
+	-f
+	-abc
+	// non-boolean flags
+	-n 1234
+	-Ifile
+	// mixed
+	-abcs "hello"
+	-abcn1234
+
+Flag parsing stops after the terminator "--". Unlike the flag package,
+flags can be interspersed with arguments anywhere on the command line
+before this terminator.
+
+Integer flags accept 1234, 0664, 0x1234 and may be negative.
+Boolean flags (in their long form) accept 1, 0, t, f, true, false,
+TRUE, FALSE, True, False.
+Duration flags accept any input valid for time.ParseDuration.
+
+The default set of command-line flags is controlled by
+top-level functions.  The FlagSet type allows one to define
+independent sets of flags, such as to implement subcommands
+in a command-line interface. The methods of FlagSet are
+analogous to the top-level functions for the command-line
+flag set.
+*/
+package pflag
+
+import (
+	"bytes"
+	"errors"
+	goflag "flag"
+	"fmt"
+	"io"
+	"os"
+	"sort"
+	"strings"
+)
+
+// ErrHelp is the error returned if the flag -help is invoked but no such flag is defined.
+var ErrHelp = errors.New("pflag: help requested")
+
+// ErrorHandling defines how to handle flag parsing errors.
+type ErrorHandling int
+
+const (
+	// ContinueOnError will return an err from Parse() if an error is found
+	ContinueOnError ErrorHandling = iota
+	// ExitOnError will call os.Exit(2) if an error is found when parsing
+	ExitOnError
+	// PanicOnError will panic() if an error is found when parsing flags
+	PanicOnError
+)
+
+// ParseErrorsWhitelist defines the parsing errors that can be ignored
+type ParseErrorsWhitelist struct {
+	// UnknownFlags will ignore unknown flags errors and continue parsing rest of the flags
+	UnknownFlags bool
+}
+
+// NormalizedName is a flag name that has been normalized according to rules
+// for the FlagSet (e.g. making '-' and '_' equivalent).
+type NormalizedName string
+
+// A FlagSet represents a set of defined flags.
+type FlagSet struct {
+	// Usage is the function called when an error occurs while parsing flags.
+	// The field is a function (not a method) that may be changed to point to
+	// a custom error handler.
+	Usage func()
+
+	// SortFlags is used to indicate, if user wants to have sorted flags in
+	// help/usage messages.
+	SortFlags bool
+
+	// ParseErrorsWhitelist is used to configure a whitelist of errors
+	ParseErrorsWhitelist ParseErrorsWhitelist
+
+	name              string
+	parsed            bool
+	actual            map[NormalizedName]*Flag
+	orderedActual     []*Flag
+	sortedActual      []*Flag
+	formal            map[NormalizedName]*Flag
+	orderedFormal     []*Flag
+	sortedFormal      []*Flag
+	shorthands        map[byte]*Flag
+	args              []string // arguments after flags
+	argsLenAtDash     int      // len(args) when a '--' was located when parsing, or -1 if no --
+	errorHandling     ErrorHandling
+	output            io.Writer // nil means stderr; use out() accessor
+	interspersed      bool      // allow interspersed option/non-option args
+	normalizeNameFunc func(f *FlagSet, name string) NormalizedName
+
+	addedGoFlagSets []*goflag.FlagSet
+}
+
+// A Flag represents the state of a flag.
+type Flag struct {
+	Name                string              // name as it appears on command line
+	Shorthand           string              // one-letter abbreviated flag
+	Usage               string              // help message
+	Value               Value               // value as set
+	DefValue            string              // default value (as text); for usage message
+	Changed             bool                // If the user set the value (or if left to default)
+	NoOptDefVal         string              // default value (as text); if the flag is on the command line without any options
+	Deprecated          string              // If this flag is deprecated, this string is the new or now thing to use
+	Hidden              bool                // used by cobra.Command to allow flags to be hidden from help/usage text
+	ShorthandDeprecated string              // If the shorthand of this flag is deprecated, this string is the new or now thing to use
+	Annotations         map[string][]string // used by cobra.Command bash autocomple code
+}
+
+// Value is the interface to the dynamic value stored in a flag.
+// (The default value is represented as a string.)
+type Value interface {
+	String() string
+	Set(string) error
+	Type() string
+}
+
+// sortFlags returns the flags as a slice in lexicographical sorted order.
+func sortFlags(flags map[NormalizedName]*Flag) []*Flag {
+	list := make(sort.StringSlice, len(flags))
+	i := 0
+	for k := range flags {
+		list[i] = string(k)
+		i++
+	}
+	list.Sort()
+	result := make([]*Flag, len(list))
+	for i, name := range list {
+		result[i] = flags[NormalizedName(name)]
+	}
+	return result
+}
+
+// SetNormalizeFunc allows you to add a function which can translate flag names.
+// Flags added to the FlagSet will be translated and then when anything tries to
+// look up the flag that will also be translated. So it would be possible to create
+// a flag named "getURL" and have it translated to "geturl".  A user could then pass
+// "--getUrl" which may also be translated to "geturl" and everything will work.
+func (f *FlagSet) SetNormalizeFunc(n func(f *FlagSet, name string) NormalizedName) {
+	f.normalizeNameFunc = n
+	f.sortedFormal = f.sortedFormal[:0]
+	for fname, flag := range f.formal {
+		nname := f.normalizeFlagName(flag.Name)
+		if fname == nname {
+			continue
+		}
+		flag.Name = string(nname)
+		delete(f.formal, fname)
+		f.formal[nname] = flag
+		if _, set := f.actual[fname]; set {
+			delete(f.actual, fname)
+			f.actual[nname] = flag
+		}
+	}
+}
+
+// GetNormalizeFunc returns the previously set NormalizeFunc of a function which
+// does no translation, if not set previously.
+func (f *FlagSet) GetNormalizeFunc() func(f *FlagSet, name string) NormalizedName {
+	if f.normalizeNameFunc != nil {
+		return f.normalizeNameFunc
+	}
+	return func(f *FlagSet, name string) NormalizedName { return NormalizedName(name) }
+}
+
+func (f *FlagSet) normalizeFlagName(name string) NormalizedName {
+	n := f.GetNormalizeFunc()
+	return n(f, name)
+}
+
+func (f *FlagSet) out() io.Writer {
+	if f.output == nil {
+		return os.Stderr
+	}
+	return f.output
+}
+
+// SetOutput sets the destination for usage and error messages.
+// If output is nil, os.Stderr is used.
+func (f *FlagSet) SetOutput(output io.Writer) {
+	f.output = output
+}
+
+// VisitAll visits the flags in lexicographical order or
+// in primordial order if f.SortFlags is false, calling fn for each.
+// It visits all flags, even those not set.
+func (f *FlagSet) VisitAll(fn func(*Flag)) {
+	if len(f.formal) == 0 {
+		return
+	}
+
+	var flags []*Flag
+	if f.SortFlags {
+		if len(f.formal) != len(f.sortedFormal) {
+			f.sortedFormal = sortFlags(f.formal)
+		}
+		flags = f.sortedFormal
+	} else {
+		flags = f.orderedFormal
+	}
+
+	for _, flag := range flags {
+		fn(flag)
+	}
+}
+
+// HasFlags returns a bool to indicate if the FlagSet has any flags defined.
+func (f *FlagSet) HasFlags() bool {
+	return len(f.formal) > 0
+}
+
+// HasAvailableFlags returns a bool to indicate if the FlagSet has any flags
+// that are not hidden.
+func (f *FlagSet) HasAvailableFlags() bool {
+	for _, flag := range f.formal {
+		if !flag.Hidden {
+			return true
+		}
+	}
+	return false
+}
+
+// VisitAll visits the command-line flags in lexicographical order or
+// in primordial order if f.SortFlags is false, calling fn for each.
+// It visits all flags, even those not set.
+func VisitAll(fn func(*Flag)) {
+	CommandLine.VisitAll(fn)
+}
+
+// Visit visits the flags in lexicographical order or
+// in primordial order if f.SortFlags is false, calling fn for each.
+// It visits only those flags that have been set.
+func (f *FlagSet) Visit(fn func(*Flag)) {
+	if len(f.actual) == 0 {
+		return
+	}
+
+	var flags []*Flag
+	if f.SortFlags {
+		if len(f.actual) != len(f.sortedActual) {
+			f.sortedActual = sortFlags(f.actual)
+		}
+		flags = f.sortedActual
+	} else {
+		flags = f.orderedActual
+	}
+
+	for _, flag := range flags {
+		fn(flag)
+	}
+}
+
+// Visit visits the command-line flags in lexicographical order or
+// in primordial order if f.SortFlags is false, calling fn for each.
+// It visits only those flags that have been set.
+func Visit(fn func(*Flag)) {
+	CommandLine.Visit(fn)
+}
+
+// Lookup returns the Flag structure of the named flag, returning nil if none exists.
+func (f *FlagSet) Lookup(name string) *Flag {
+	return f.lookup(f.normalizeFlagName(name))
+}
+
+// ShorthandLookup returns the Flag structure of the short handed flag,
+// returning nil if none exists.
+// It panics, if len(name) > 1.
+func (f *FlagSet) ShorthandLookup(name string) *Flag {
+	if name == "" {
+		return nil
+	}
+	if len(name) > 1 {
+		msg := fmt.Sprintf("can not look up shorthand which is more than one ASCII character: %q", name)
+		fmt.Fprintf(f.out(), msg)
+		panic(msg)
+	}
+	c := name[0]
+	return f.shorthands[c]
+}
+
+// lookup returns the Flag structure of the named flag, returning nil if none exists.
+func (f *FlagSet) lookup(name NormalizedName) *Flag {
+	return f.formal[name]
+}
+
+// func to return a given type for a given flag name
+func (f *FlagSet) getFlagType(name string, ftype string, convFunc func(sval string) (interface{}, error)) (interface{}, error) {
+	flag := f.Lookup(name)
+	if flag == nil {
+		err := fmt.Errorf("flag accessed but not defined: %s", name)
+		return nil, err
+	}
+
+	if flag.Value.Type() != ftype {
+		err := fmt.Errorf("trying to get %s value of flag of type %s", ftype, flag.Value.Type())
+		return nil, err
+	}
+
+	sval := flag.Value.String()
+	result, err := convFunc(sval)
+	if err != nil {
+		return nil, err
+	}
+	return result, nil
+}
+
+// ArgsLenAtDash will return the length of f.Args at the moment when a -- was
+// found during arg parsing. This allows your program to know which args were
+// before the -- and which came after.
+func (f *FlagSet) ArgsLenAtDash() int {
+	return f.argsLenAtDash
+}
+
+// MarkDeprecated indicated that a flag is deprecated in your program. It will
+// continue to function but will not show up in help or usage messages. Using
+// this flag will also print the given usageMessage.
+func (f *FlagSet) MarkDeprecated(name string, usageMessage string) error {
+	flag := f.Lookup(name)
+	if flag == nil {
+		return fmt.Errorf("flag %q does not exist", name)
+	}
+	if usageMessage == "" {
+		return fmt.Errorf("deprecated message for flag %q must be set", name)
+	}
+	flag.Deprecated = usageMessage
+	flag.Hidden = true
+	return nil
+}
+
+// MarkShorthandDeprecated will mark the shorthand of a flag deprecated in your
+// program. It will continue to function but will not show up in help or usage
+// messages. Using this flag will also print the given usageMessage.
+func (f *FlagSet) MarkShorthandDeprecated(name string, usageMessage string) error {
+	flag := f.Lookup(name)
+	if flag == nil {
+		return fmt.Errorf("flag %q does not exist", name)
+	}
+	if usageMessage == "" {
+		return fmt.Errorf("deprecated message for flag %q must be set", name)
+	}
+	flag.ShorthandDeprecated = usageMessage
+	return nil
+}
+
+// MarkHidden sets a flag to 'hidden' in your program. It will continue to
+// function but will not show up in help or usage messages.
+func (f *FlagSet) MarkHidden(name string) error {
+	flag := f.Lookup(name)
+	if flag == nil {
+		return fmt.Errorf("flag %q does not exist", name)
+	}
+	flag.Hidden = true
+	return nil
+}
+
+// Lookup returns the Flag structure of the named command-line flag,
+// returning nil if none exists.
+func Lookup(name string) *Flag {
+	return CommandLine.Lookup(name)
+}
+
+// ShorthandLookup returns the Flag structure of the short handed flag,
+// returning nil if none exists.
+func ShorthandLookup(name string) *Flag {
+	return CommandLine.ShorthandLookup(name)
+}
+
+// Set sets the value of the named flag.
+func (f *FlagSet) Set(name, value string) error {
+	normalName := f.normalizeFlagName(name)
+	flag, ok := f.formal[normalName]
+	if !ok {
+		return fmt.Errorf("no such flag -%v", name)
+	}
+
+	err := flag.Value.Set(value)
+	if err != nil {
+		var flagName string
+		if flag.Shorthand != "" && flag.ShorthandDeprecated == "" {
+			flagName = fmt.Sprintf("-%s, --%s", flag.Shorthand, flag.Name)
+		} else {
+			flagName = fmt.Sprintf("--%s", flag.Name)
+		}
+		return fmt.Errorf("invalid argument %q for %q flag: %v", value, flagName, err)
+	}
+
+	if !flag.Changed {
+		if f.actual == nil {
+			f.actual = make(map[NormalizedName]*Flag)
+		}
+		f.actual[normalName] = flag
+		f.orderedActual = append(f.orderedActual, flag)
+
+		flag.Changed = true
+	}
+
+	if flag.Deprecated != "" {
+		fmt.Fprintf(f.out(), "Flag --%s has been deprecated, %s\n", flag.Name, flag.Deprecated)
+	}
+	return nil
+}
+
+// SetAnnotation allows one to set arbitrary annotations on a flag in the FlagSet.
+// This is sometimes used by spf13/cobra programs which want to generate additional
+// bash completion information.
+func (f *FlagSet) SetAnnotation(name, key string, values []string) error {
+	normalName := f.normalizeFlagName(name)
+	flag, ok := f.formal[normalName]
+	if !ok {
+		return fmt.Errorf("no such flag -%v", name)
+	}
+	if flag.Annotations == nil {
+		flag.Annotations = map[string][]string{}
+	}
+	flag.Annotations[key] = values
+	return nil
+}
+
+// Changed returns true if the flag was explicitly set during Parse() and false
+// otherwise
+func (f *FlagSet) Changed(name string) bool {
+	flag := f.Lookup(name)
+	// If a flag doesn't exist, it wasn't changed....
+	if flag == nil {
+		return false
+	}
+	return flag.Changed
+}
+
+// Set sets the value of the named command-line flag.
+func Set(name, value string) error {
+	return CommandLine.Set(name, value)
+}
+
+// PrintDefaults prints, to standard error unless configured
+// otherwise, the default values of all defined flags in the set.
+func (f *FlagSet) PrintDefaults() {
+	usages := f.FlagUsages()
+	fmt.Fprint(f.out(), usages)
+}
+
+// defaultIsZeroValue returns true if the default value for this flag represents
+// a zero value.
+func (f *Flag) defaultIsZeroValue() bool {
+	switch f.Value.(type) {
+	case boolFlag:
+		return f.DefValue == "false"
+	case *durationValue:
+		// Beginning in Go 1.7, duration zero values are "0s"
+		return f.DefValue == "0" || f.DefValue == "0s"
+	case *intValue, *int8Value, *int32Value, *int64Value, *uintValue, *uint8Value, *uint16Value, *uint32Value, *uint64Value, *countValue, *float32Value, *float64Value:
+		return f.DefValue == "0"
+	case *stringValue:
+		return f.DefValue == ""
+	case *ipValue, *ipMaskValue, *ipNetValue:
+		return f.DefValue == "<nil>"
+	case *intSliceValue, *stringSliceValue, *stringArrayValue:
+		return f.DefValue == "[]"
+	default:
+		switch f.Value.String() {
+		case "false":
+			return true
+		case "<nil>":
+			return true
+		case "":
+			return true
+		case "0":
+			return true
+		}
+		return false
+	}
+}
+
+// UnquoteUsage extracts a back-quoted name from the usage
+// string for a flag and returns it and the un-quoted usage.
+// Given "a `name` to show" it returns ("name", "a name to show").
+// If there are no back quotes, the name is an educated guess of the
+// type of the flag's value, or the empty string if the flag is boolean.
+func UnquoteUsage(flag *Flag) (name string, usage string) {
+	// Look for a back-quoted name, but avoid the strings package.
+	usage = flag.Usage
+	for i := 0; i < len(usage); i++ {
+		if usage[i] == '`' {
+			for j := i + 1; j < len(usage); j++ {
+				if usage[j] == '`' {
+					name = usage[i+1 : j]
+					usage = usage[:i] + name + usage[j+1:]
+					return name, usage
+				}
+			}
+			break // Only one back quote; use type name.
+		}
+	}
+
+	name = flag.Value.Type()
+	switch name {
+	case "bool":
+		name = ""
+	case "float64":
+		name = "float"
+	case "int64":
+		name = "int"
+	case "uint64":
+		name = "uint"
+	case "stringSlice":
+		name = "strings"
+	case "intSlice":
+		name = "ints"
+	case "uintSlice":
+		name = "uints"
+	case "boolSlice":
+		name = "bools"
+	}
+
+	return
+}
+
+// Splits the string `s` on whitespace into an initial substring up to
+// `i` runes in length and the remainder. Will go `slop` over `i` if
+// that encompasses the entire string (which allows the caller to
+// avoid short orphan words on the final line).
+func wrapN(i, slop int, s string) (string, string) {
+	if i+slop > len(s) {
+		return s, ""
+	}
+
+	w := strings.LastIndexAny(s[:i], " \t\n")
+	if w <= 0 {
+		return s, ""
+	}
+	nlPos := strings.LastIndex(s[:i], "\n")
+	if nlPos > 0 && nlPos < w {
+		return s[:nlPos], s[nlPos+1:]
+	}
+	return s[:w], s[w+1:]
+}
+
+// Wraps the string `s` to a maximum width `w` with leading indent
+// `i`. The first line is not indented (this is assumed to be done by
+// caller). Pass `w` == 0 to do no wrapping
+func wrap(i, w int, s string) string {
+	if w == 0 {
+		return strings.Replace(s, "\n", "\n"+strings.Repeat(" ", i), -1)
+	}
+
+	// space between indent i and end of line width w into which
+	// we should wrap the text.
+	wrap := w - i
+
+	var r, l string
+
+	// Not enough space for sensible wrapping. Wrap as a block on
+	// the next line instead.
+	if wrap < 24 {
+		i = 16
+		wrap = w - i
+		r += "\n" + strings.Repeat(" ", i)
+	}
+	// If still not enough space then don't even try to wrap.
+	if wrap < 24 {
+		return strings.Replace(s, "\n", r, -1)
+	}
+
+	// Try to avoid short orphan words on the final line, by
+	// allowing wrapN to go a bit over if that would fit in the
+	// remainder of the line.
+	slop := 5
+	wrap = wrap - slop
+
+	// Handle first line, which is indented by the caller (or the
+	// special case above)
+	l, s = wrapN(wrap, slop, s)
+	r = r + strings.Replace(l, "\n", "\n"+strings.Repeat(" ", i), -1)
+
+	// Now wrap the rest
+	for s != "" {
+		var t string
+
+		t, s = wrapN(wrap, slop, s)
+		r = r + "\n" + strings.Repeat(" ", i) + strings.Replace(t, "\n", "\n"+strings.Repeat(" ", i), -1)
+	}
+
+	return r
+
+}
+
+// FlagUsagesWrapped returns a string containing the usage information
+// for all flags in the FlagSet. Wrapped to `cols` columns (0 for no
+// wrapping)
+func (f *FlagSet) FlagUsagesWrapped(cols int) string {
+	buf := new(bytes.Buffer)
+
+	lines := make([]string, 0, len(f.formal))
+
+	maxlen := 0
+	f.VisitAll(func(flag *Flag) {
+		if flag.Hidden {
+			return
+		}
+
+		line := ""
+		if flag.Shorthand != "" && flag.ShorthandDeprecated == "" {
+			line = fmt.Sprintf("  -%s, --%s", flag.Shorthand, flag.Name)
+		} else {
+			line = fmt.Sprintf("      --%s", flag.Name)
+		}
+
+		varname, usage := UnquoteUsage(flag)
+		if varname != "" {
+			line += " " + varname
+		}
+		if flag.NoOptDefVal != "" {
+			switch flag.Value.Type() {
+			case "string":
+				line += fmt.Sprintf("[=\"%s\"]", flag.NoOptDefVal)
+			case "bool":
+				if flag.NoOptDefVal != "true" {
+					line += fmt.Sprintf("[=%s]", flag.NoOptDefVal)
+				}
+			case "count":
+				if flag.NoOptDefVal != "+1" {
+					line += fmt.Sprintf("[=%s]", flag.NoOptDefVal)
+				}
+			default:
+				line += fmt.Sprintf("[=%s]", flag.NoOptDefVal)
+			}
+		}
+
+		// This special character will be replaced with spacing once the
+		// correct alignment is calculated
+		line += "\x00"
+		if len(line) > maxlen {
+			maxlen = len(line)
+		}
+
+		line += usage
+		if !flag.defaultIsZeroValue() {
+			if flag.Value.Type() == "string" {
+				line += fmt.Sprintf(" (default %q)", flag.DefValue)
+			} else {
+				line += fmt.Sprintf(" (default %s)", flag.DefValue)
+			}
+		}
+		if len(flag.Deprecated) != 0 {
+			line += fmt.Sprintf(" (DEPRECATED: %s)", flag.Deprecated)
+		}
+
+		lines = append(lines, line)
+	})
+
+	for _, line := range lines {
+		sidx := strings.Index(line, "\x00")
+		spacing := strings.Repeat(" ", maxlen-sidx)
+		// maxlen + 2 comes from + 1 for the \x00 and + 1 for the (deliberate) off-by-one in maxlen-sidx
+		fmt.Fprintln(buf, line[:sidx], spacing, wrap(maxlen+2, cols, line[sidx+1:]))
+	}
+
+	return buf.String()
+}
+
+// FlagUsages returns a string containing the usage information for all flags in
+// the FlagSet
+func (f *FlagSet) FlagUsages() string {
+	return f.FlagUsagesWrapped(0)
+}
+
+// PrintDefaults prints to standard error the default values of all defined command-line flags.
+func PrintDefaults() {
+	CommandLine.PrintDefaults()
+}
+
+// defaultUsage is the default function to print a usage message.
+func defaultUsage(f *FlagSet) {
+	fmt.Fprintf(f.out(), "Usage of %s:\n", f.name)
+	f.PrintDefaults()
+}
+
+// NOTE: Usage is not just defaultUsage(CommandLine)
+// because it serves (via godoc flag Usage) as the example
+// for how to write your own usage function.
+
+// Usage prints to standard error a usage message documenting all defined command-line flags.
+// The function is a variable that may be changed to point to a custom function.
+// By default it prints a simple header and calls PrintDefaults; for details about the
+// format of the output and how to control it, see the documentation for PrintDefaults.
+var Usage = func() {
+	fmt.Fprintf(os.Stderr, "Usage of %s:\n", os.Args[0])
+	PrintDefaults()
+}
+
+// NFlag returns the number of flags that have been set.
+func (f *FlagSet) NFlag() int { return len(f.actual) }
+
+// NFlag returns the number of command-line flags that have been set.
+func NFlag() int { return len(CommandLine.actual) }
+
+// Arg returns the i'th argument.  Arg(0) is the first remaining argument
+// after flags have been processed.
+func (f *FlagSet) Arg(i int) string {
+	if i < 0 || i >= len(f.args) {
+		return ""
+	}
+	return f.args[i]
+}
+
+// Arg returns the i'th command-line argument.  Arg(0) is the first remaining argument
+// after flags have been processed.
+func Arg(i int) string {
+	return CommandLine.Arg(i)
+}
+
+// NArg is the number of arguments remaining after flags have been processed.
+func (f *FlagSet) NArg() int { return len(f.args) }
+
+// NArg is the number of arguments remaining after flags have been processed.
+func NArg() int { return len(CommandLine.args) }
+
+// Args returns the non-flag arguments.
+func (f *FlagSet) Args() []string { return f.args }
+
+// Args returns the non-flag command-line arguments.
+func Args() []string { return CommandLine.args }
+
+// Var defines a flag with the specified name and usage string. The type and
+// value of the flag are represented by the first argument, of type Value, which
+// typically holds a user-defined implementation of Value. For instance, the
+// caller could create a flag that turns a comma-separated string into a slice
+// of strings by giving the slice the methods of Value; in particular, Set would
+// decompose the comma-separated string into the slice.
+func (f *FlagSet) Var(value Value, name string, usage string) {
+	f.VarP(value, name, "", usage)
+}
+
+// VarPF is like VarP, but returns the flag created
+func (f *FlagSet) VarPF(value Value, name, shorthand, usage string) *Flag {
+	// Remember the default value as a string; it won't change.
+	flag := &Flag{
+		Name:      name,
+		Shorthand: shorthand,
+		Usage:     usage,
+		Value:     value,
+		DefValue:  value.String(),
+	}
+	f.AddFlag(flag)
+	return flag
+}
+
+// VarP is like Var, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) VarP(value Value, name, shorthand, usage string) {
+	f.VarPF(value, name, shorthand, usage)
+}
+
+// AddFlag will add the flag to the FlagSet
+func (f *FlagSet) AddFlag(flag *Flag) {
+	normalizedFlagName := f.normalizeFlagName(flag.Name)
+
+	_, alreadyThere := f.formal[normalizedFlagName]
+	if alreadyThere {
+		msg := fmt.Sprintf("%s flag redefined: %s", f.name, flag.Name)
+		fmt.Fprintln(f.out(), msg)
+		panic(msg) // Happens only if flags are declared with identical names
+	}
+	if f.formal == nil {
+		f.formal = make(map[NormalizedName]*Flag)
+	}
+
+	flag.Name = string(normalizedFlagName)
+	f.formal[normalizedFlagName] = flag
+	f.orderedFormal = append(f.orderedFormal, flag)
+
+	if flag.Shorthand == "" {
+		return
+	}
+	if len(flag.Shorthand) > 1 {
+		msg := fmt.Sprintf("%q shorthand is more than one ASCII character", flag.Shorthand)
+		fmt.Fprintf(f.out(), msg)
+		panic(msg)
+	}
+	if f.shorthands == nil {
+		f.shorthands = make(map[byte]*Flag)
+	}
+	c := flag.Shorthand[0]
+	used, alreadyThere := f.shorthands[c]
+	if alreadyThere {
+		msg := fmt.Sprintf("unable to redefine %q shorthand in %q flagset: it's already used for %q flag", c, f.name, used.Name)
+		fmt.Fprintf(f.out(), msg)
+		panic(msg)
+	}
+	f.shorthands[c] = flag
+}
+
+// AddFlagSet adds one FlagSet to another. If a flag is already present in f
+// the flag from newSet will be ignored.
+func (f *FlagSet) AddFlagSet(newSet *FlagSet) {
+	if newSet == nil {
+		return
+	}
+	newSet.VisitAll(func(flag *Flag) {
+		if f.Lookup(flag.Name) == nil {
+			f.AddFlag(flag)
+		}
+	})
+}
+
+// Var defines a flag with the specified name and usage string. The type and
+// value of the flag are represented by the first argument, of type Value, which
+// typically holds a user-defined implementation of Value. For instance, the
+// caller could create a flag that turns a comma-separated string into a slice
+// of strings by giving the slice the methods of Value; in particular, Set would
+// decompose the comma-separated string into the slice.
+func Var(value Value, name string, usage string) {
+	CommandLine.VarP(value, name, "", usage)
+}
+
+// VarP is like Var, but accepts a shorthand letter that can be used after a single dash.
+func VarP(value Value, name, shorthand, usage string) {
+	CommandLine.VarP(value, name, shorthand, usage)
+}
+
+// failf prints to standard error a formatted error and usage message and
+// returns the error.
+func (f *FlagSet) failf(format string, a ...interface{}) error {
+	err := fmt.Errorf(format, a...)
+	if f.errorHandling != ContinueOnError {
+		fmt.Fprintln(f.out(), err)
+		f.usage()
+	}
+	return err
+}
+
+// usage calls the Usage method for the flag set, or the usage function if
+// the flag set is CommandLine.
+func (f *FlagSet) usage() {
+	if f == CommandLine {
+		Usage()
+	} else if f.Usage == nil {
+		defaultUsage(f)
+	} else {
+		f.Usage()
+	}
+}
+
+//--unknown (args will be empty)
+//--unknown --next-flag ... (args will be --next-flag ...)
+//--unknown arg ... (args will be arg ...)
+func stripUnknownFlagValue(args []string) []string {
+	if len(args) == 0 {
+		//--unknown
+		return args
+	}
+
+	first := args[0]
+	if len(first) > 0 && first[0] == '-' {
+		//--unknown --next-flag ...
+		return args
+	}
+
+	//--unknown arg ... (args will be arg ...)
+	if len(args) > 1 {
+		return args[1:]
+	}
+	return nil
+}
+
+func (f *FlagSet) parseLongArg(s string, args []string, fn parseFunc) (a []string, err error) {
+	a = args
+	name := s[2:]
+	if len(name) == 0 || name[0] == '-' || name[0] == '=' {
+		err = f.failf("bad flag syntax: %s", s)
+		return
+	}
+
+	split := strings.SplitN(name, "=", 2)
+	name = split[0]
+	flag, exists := f.formal[f.normalizeFlagName(name)]
+
+	if !exists {
+		switch {
+		case name == "help":
+			f.usage()
+			return a, ErrHelp
+		case f.ParseErrorsWhitelist.UnknownFlags:
+			// --unknown=unknownval arg ...
+			// we do not want to lose arg in this case
+			if len(split) >= 2 {
+				return a, nil
+			}
+
+			return stripUnknownFlagValue(a), nil
+		default:
+			err = f.failf("unknown flag: --%s", name)
+			return
+		}
+	}
+
+	var value string
+	if len(split) == 2 {
+		// '--flag=arg'
+		value = split[1]
+	} else if flag.NoOptDefVal != "" {
+		// '--flag' (arg was optional)
+		value = flag.NoOptDefVal
+	} else if len(a) > 0 {
+		// '--flag arg'
+		value = a[0]
+		a = a[1:]
+	} else {
+		// '--flag' (arg was required)
+		err = f.failf("flag needs an argument: %s", s)
+		return
+	}
+
+	err = fn(flag, value)
+	if err != nil {
+		f.failf(err.Error())
+	}
+	return
+}
+
+func (f *FlagSet) parseSingleShortArg(shorthands string, args []string, fn parseFunc) (outShorts string, outArgs []string, err error) {
+	outArgs = args
+
+	if strings.HasPrefix(shorthands, "test.") {
+		return
+	}
+
+	outShorts = shorthands[1:]
+	c := shorthands[0]
+
+	flag, exists := f.shorthands[c]
+	if !exists {
+		switch {
+		case c == 'h':
+			f.usage()
+			err = ErrHelp
+			return
+		case f.ParseErrorsWhitelist.UnknownFlags:
+			// '-f=arg arg ...'
+			// we do not want to lose arg in this case
+			if len(shorthands) > 2 && shorthands[1] == '=' {
+				outShorts = ""
+				return
+			}
+
+			outArgs = stripUnknownFlagValue(outArgs)
+			return
+		default:
+			err = f.failf("unknown shorthand flag: %q in -%s", c, shorthands)
+			return
+		}
+	}
+
+	var value string
+	if len(shorthands) > 2 && shorthands[1] == '=' {
+		// '-f=arg'
+		value = shorthands[2:]
+		outShorts = ""
+	} else if flag.NoOptDefVal != "" {
+		// '-f' (arg was optional)
+		value = flag.NoOptDefVal
+	} else if len(shorthands) > 1 {
+		// '-farg'
+		value = shorthands[1:]
+		outShorts = ""
+	} else if len(args) > 0 {
+		// '-f arg'
+		value = args[0]
+		outArgs = args[1:]
+	} else {
+		// '-f' (arg was required)
+		err = f.failf("flag needs an argument: %q in -%s", c, shorthands)
+		return
+	}
+
+	if flag.ShorthandDeprecated != "" {
+		fmt.Fprintf(f.out(), "Flag shorthand -%s has been deprecated, %s\n", flag.Shorthand, flag.ShorthandDeprecated)
+	}
+
+	err = fn(flag, value)
+	if err != nil {
+		f.failf(err.Error())
+	}
+	return
+}
+
+func (f *FlagSet) parseShortArg(s string, args []string, fn parseFunc) (a []string, err error) {
+	a = args
+	shorthands := s[1:]
+
+	// "shorthands" can be a series of shorthand letters of flags (e.g. "-vvv").
+	for len(shorthands) > 0 {
+		shorthands, a, err = f.parseSingleShortArg(shorthands, args, fn)
+		if err != nil {
+			return
+		}
+	}
+
+	return
+}
+
+func (f *FlagSet) parseArgs(args []string, fn parseFunc) (err error) {
+	for len(args) > 0 {
+		s := args[0]
+		args = args[1:]
+		if len(s) == 0 || s[0] != '-' || len(s) == 1 {
+			if !f.interspersed {
+				f.args = append(f.args, s)
+				f.args = append(f.args, args...)
+				return nil
+			}
+			f.args = append(f.args, s)
+			continue
+		}
+
+		if s[1] == '-' {
+			if len(s) == 2 { // "--" terminates the flags
+				f.argsLenAtDash = len(f.args)
+				f.args = append(f.args, args...)
+				break
+			}
+			args, err = f.parseLongArg(s, args, fn)
+		} else {
+			args, err = f.parseShortArg(s, args, fn)
+		}
+		if err != nil {
+			return
+		}
+	}
+	return
+}
+
+// Parse parses flag definitions from the argument list, which should not
+// include the command name.  Must be called after all flags in the FlagSet
+// are defined and before flags are accessed by the program.
+// The return value will be ErrHelp if -help was set but not defined.
+func (f *FlagSet) Parse(arguments []string) error {
+	if f.addedGoFlagSets != nil {
+		for _, goFlagSet := range f.addedGoFlagSets {
+			goFlagSet.Parse(nil)
+		}
+	}
+	f.parsed = true
+
+	if len(arguments) < 0 {
+		return nil
+	}
+
+	f.args = make([]string, 0, len(arguments))
+
+	set := func(flag *Flag, value string) error {
+		return f.Set(flag.Name, value)
+	}
+
+	err := f.parseArgs(arguments, set)
+	if err != nil {
+		switch f.errorHandling {
+		case ContinueOnError:
+			return err
+		case ExitOnError:
+			fmt.Println(err)
+			os.Exit(2)
+		case PanicOnError:
+			panic(err)
+		}
+	}
+	return nil
+}
+
+type parseFunc func(flag *Flag, value string) error
+
+// ParseAll parses flag definitions from the argument list, which should not
+// include the command name. The arguments for fn are flag and value. Must be
+// called after all flags in the FlagSet are defined and before flags are
+// accessed by the program. The return value will be ErrHelp if -help was set
+// but not defined.
+func (f *FlagSet) ParseAll(arguments []string, fn func(flag *Flag, value string) error) error {
+	f.parsed = true
+	f.args = make([]string, 0, len(arguments))
+
+	err := f.parseArgs(arguments, fn)
+	if err != nil {
+		switch f.errorHandling {
+		case ContinueOnError:
+			return err
+		case ExitOnError:
+			os.Exit(2)
+		case PanicOnError:
+			panic(err)
+		}
+	}
+	return nil
+}
+
+// Parsed reports whether f.Parse has been called.
+func (f *FlagSet) Parsed() bool {
+	return f.parsed
+}
+
+// Parse parses the command-line flags from os.Args[1:].  Must be called
+// after all flags are defined and before flags are accessed by the program.
+func Parse() {
+	// Ignore errors; CommandLine is set for ExitOnError.
+	CommandLine.Parse(os.Args[1:])
+}
+
+// ParseAll parses the command-line flags from os.Args[1:] and called fn for each.
+// The arguments for fn are flag and value. Must be called after all flags are
+// defined and before flags are accessed by the program.
+func ParseAll(fn func(flag *Flag, value string) error) {
+	// Ignore errors; CommandLine is set for ExitOnError.
+	CommandLine.ParseAll(os.Args[1:], fn)
+}
+
+// SetInterspersed sets whether to support interspersed option/non-option arguments.
+func SetInterspersed(interspersed bool) {
+	CommandLine.SetInterspersed(interspersed)
+}
+
+// Parsed returns true if the command-line flags have been parsed.
+func Parsed() bool {
+	return CommandLine.Parsed()
+}
+
+// CommandLine is the default set of command-line flags, parsed from os.Args.
+var CommandLine = NewFlagSet(os.Args[0], ExitOnError)
+
+// NewFlagSet returns a new, empty flag set with the specified name,
+// error handling property and SortFlags set to true.
+func NewFlagSet(name string, errorHandling ErrorHandling) *FlagSet {
+	f := &FlagSet{
+		name:          name,
+		errorHandling: errorHandling,
+		argsLenAtDash: -1,
+		interspersed:  true,
+		SortFlags:     true,
+	}
+	return f
+}
+
+// SetInterspersed sets whether to support interspersed option/non-option arguments.
+func (f *FlagSet) SetInterspersed(interspersed bool) {
+	f.interspersed = interspersed
+}
+
+// Init sets the name and error handling property for a flag set.
+// By default, the zero FlagSet uses an empty name and the
+// ContinueOnError error handling policy.
+func (f *FlagSet) Init(name string, errorHandling ErrorHandling) {
+	f.name = name
+	f.errorHandling = errorHandling
+	f.argsLenAtDash = -1
+}
diff --git a/vendor/github.com/spf13/pflag/float32.go b/vendor/github.com/spf13/pflag/float32.go
new file mode 100644
index 0000000..a243f81
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/float32.go
@@ -0,0 +1,88 @@
+package pflag
+
+import "strconv"
+
+// -- float32 Value
+type float32Value float32
+
+func newFloat32Value(val float32, p *float32) *float32Value {
+	*p = val
+	return (*float32Value)(p)
+}
+
+func (f *float32Value) Set(s string) error {
+	v, err := strconv.ParseFloat(s, 32)
+	*f = float32Value(v)
+	return err
+}
+
+func (f *float32Value) Type() string {
+	return "float32"
+}
+
+func (f *float32Value) String() string { return strconv.FormatFloat(float64(*f), 'g', -1, 32) }
+
+func float32Conv(sval string) (interface{}, error) {
+	v, err := strconv.ParseFloat(sval, 32)
+	if err != nil {
+		return 0, err
+	}
+	return float32(v), nil
+}
+
+// GetFloat32 return the float32 value of a flag with the given name
+func (f *FlagSet) GetFloat32(name string) (float32, error) {
+	val, err := f.getFlagType(name, "float32", float32Conv)
+	if err != nil {
+		return 0, err
+	}
+	return val.(float32), nil
+}
+
+// Float32Var defines a float32 flag with specified name, default value, and usage string.
+// The argument p points to a float32 variable in which to store the value of the flag.
+func (f *FlagSet) Float32Var(p *float32, name string, value float32, usage string) {
+	f.VarP(newFloat32Value(value, p), name, "", usage)
+}
+
+// Float32VarP is like Float32Var, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Float32VarP(p *float32, name, shorthand string, value float32, usage string) {
+	f.VarP(newFloat32Value(value, p), name, shorthand, usage)
+}
+
+// Float32Var defines a float32 flag with specified name, default value, and usage string.
+// The argument p points to a float32 variable in which to store the value of the flag.
+func Float32Var(p *float32, name string, value float32, usage string) {
+	CommandLine.VarP(newFloat32Value(value, p), name, "", usage)
+}
+
+// Float32VarP is like Float32Var, but accepts a shorthand letter that can be used after a single dash.
+func Float32VarP(p *float32, name, shorthand string, value float32, usage string) {
+	CommandLine.VarP(newFloat32Value(value, p), name, shorthand, usage)
+}
+
+// Float32 defines a float32 flag with specified name, default value, and usage string.
+// The return value is the address of a float32 variable that stores the value of the flag.
+func (f *FlagSet) Float32(name string, value float32, usage string) *float32 {
+	p := new(float32)
+	f.Float32VarP(p, name, "", value, usage)
+	return p
+}
+
+// Float32P is like Float32, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Float32P(name, shorthand string, value float32, usage string) *float32 {
+	p := new(float32)
+	f.Float32VarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// Float32 defines a float32 flag with specified name, default value, and usage string.
+// The return value is the address of a float32 variable that stores the value of the flag.
+func Float32(name string, value float32, usage string) *float32 {
+	return CommandLine.Float32P(name, "", value, usage)
+}
+
+// Float32P is like Float32, but accepts a shorthand letter that can be used after a single dash.
+func Float32P(name, shorthand string, value float32, usage string) *float32 {
+	return CommandLine.Float32P(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/float64.go b/vendor/github.com/spf13/pflag/float64.go
new file mode 100644
index 0000000..04b5492
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/float64.go
@@ -0,0 +1,84 @@
+package pflag
+
+import "strconv"
+
+// -- float64 Value
+type float64Value float64
+
+func newFloat64Value(val float64, p *float64) *float64Value {
+	*p = val
+	return (*float64Value)(p)
+}
+
+func (f *float64Value) Set(s string) error {
+	v, err := strconv.ParseFloat(s, 64)
+	*f = float64Value(v)
+	return err
+}
+
+func (f *float64Value) Type() string {
+	return "float64"
+}
+
+func (f *float64Value) String() string { return strconv.FormatFloat(float64(*f), 'g', -1, 64) }
+
+func float64Conv(sval string) (interface{}, error) {
+	return strconv.ParseFloat(sval, 64)
+}
+
+// GetFloat64 return the float64 value of a flag with the given name
+func (f *FlagSet) GetFloat64(name string) (float64, error) {
+	val, err := f.getFlagType(name, "float64", float64Conv)
+	if err != nil {
+		return 0, err
+	}
+	return val.(float64), nil
+}
+
+// Float64Var defines a float64 flag with specified name, default value, and usage string.
+// The argument p points to a float64 variable in which to store the value of the flag.
+func (f *FlagSet) Float64Var(p *float64, name string, value float64, usage string) {
+	f.VarP(newFloat64Value(value, p), name, "", usage)
+}
+
+// Float64VarP is like Float64Var, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Float64VarP(p *float64, name, shorthand string, value float64, usage string) {
+	f.VarP(newFloat64Value(value, p), name, shorthand, usage)
+}
+
+// Float64Var defines a float64 flag with specified name, default value, and usage string.
+// The argument p points to a float64 variable in which to store the value of the flag.
+func Float64Var(p *float64, name string, value float64, usage string) {
+	CommandLine.VarP(newFloat64Value(value, p), name, "", usage)
+}
+
+// Float64VarP is like Float64Var, but accepts a shorthand letter that can be used after a single dash.
+func Float64VarP(p *float64, name, shorthand string, value float64, usage string) {
+	CommandLine.VarP(newFloat64Value(value, p), name, shorthand, usage)
+}
+
+// Float64 defines a float64 flag with specified name, default value, and usage string.
+// The return value is the address of a float64 variable that stores the value of the flag.
+func (f *FlagSet) Float64(name string, value float64, usage string) *float64 {
+	p := new(float64)
+	f.Float64VarP(p, name, "", value, usage)
+	return p
+}
+
+// Float64P is like Float64, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Float64P(name, shorthand string, value float64, usage string) *float64 {
+	p := new(float64)
+	f.Float64VarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// Float64 defines a float64 flag with specified name, default value, and usage string.
+// The return value is the address of a float64 variable that stores the value of the flag.
+func Float64(name string, value float64, usage string) *float64 {
+	return CommandLine.Float64P(name, "", value, usage)
+}
+
+// Float64P is like Float64, but accepts a shorthand letter that can be used after a single dash.
+func Float64P(name, shorthand string, value float64, usage string) *float64 {
+	return CommandLine.Float64P(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/golangflag.go b/vendor/github.com/spf13/pflag/golangflag.go
new file mode 100644
index 0000000..d3dd72b
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/golangflag.go
@@ -0,0 +1,105 @@
+// Copyright 2009 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package pflag
+
+import (
+	goflag "flag"
+	"reflect"
+	"strings"
+)
+
+// flagValueWrapper implements pflag.Value around a flag.Value.  The main
+// difference here is the addition of the Type method that returns a string
+// name of the type.  As this is generally unknown, we approximate that with
+// reflection.
+type flagValueWrapper struct {
+	inner    goflag.Value
+	flagType string
+}
+
+// We are just copying the boolFlag interface out of goflag as that is what
+// they use to decide if a flag should get "true" when no arg is given.
+type goBoolFlag interface {
+	goflag.Value
+	IsBoolFlag() bool
+}
+
+func wrapFlagValue(v goflag.Value) Value {
+	// If the flag.Value happens to also be a pflag.Value, just use it directly.
+	if pv, ok := v.(Value); ok {
+		return pv
+	}
+
+	pv := &flagValueWrapper{
+		inner: v,
+	}
+
+	t := reflect.TypeOf(v)
+	if t.Kind() == reflect.Interface || t.Kind() == reflect.Ptr {
+		t = t.Elem()
+	}
+
+	pv.flagType = strings.TrimSuffix(t.Name(), "Value")
+	return pv
+}
+
+func (v *flagValueWrapper) String() string {
+	return v.inner.String()
+}
+
+func (v *flagValueWrapper) Set(s string) error {
+	return v.inner.Set(s)
+}
+
+func (v *flagValueWrapper) Type() string {
+	return v.flagType
+}
+
+// PFlagFromGoFlag will return a *pflag.Flag given a *flag.Flag
+// If the *flag.Flag.Name was a single character (ex: `v`) it will be accessiblei
+// with both `-v` and `--v` in flags. If the golang flag was more than a single
+// character (ex: `verbose`) it will only be accessible via `--verbose`
+func PFlagFromGoFlag(goflag *goflag.Flag) *Flag {
+	// Remember the default value as a string; it won't change.
+	flag := &Flag{
+		Name:  goflag.Name,
+		Usage: goflag.Usage,
+		Value: wrapFlagValue(goflag.Value),
+		// Looks like golang flags don't set DefValue correctly  :-(
+		//DefValue: goflag.DefValue,
+		DefValue: goflag.Value.String(),
+	}
+	// Ex: if the golang flag was -v, allow both -v and --v to work
+	if len(flag.Name) == 1 {
+		flag.Shorthand = flag.Name
+	}
+	if fv, ok := goflag.Value.(goBoolFlag); ok && fv.IsBoolFlag() {
+		flag.NoOptDefVal = "true"
+	}
+	return flag
+}
+
+// AddGoFlag will add the given *flag.Flag to the pflag.FlagSet
+func (f *FlagSet) AddGoFlag(goflag *goflag.Flag) {
+	if f.Lookup(goflag.Name) != nil {
+		return
+	}
+	newflag := PFlagFromGoFlag(goflag)
+	f.AddFlag(newflag)
+}
+
+// AddGoFlagSet will add the given *flag.FlagSet to the pflag.FlagSet
+func (f *FlagSet) AddGoFlagSet(newSet *goflag.FlagSet) {
+	if newSet == nil {
+		return
+	}
+	newSet.VisitAll(func(goflag *goflag.Flag) {
+		f.AddGoFlag(goflag)
+	})
+	if f.addedGoFlagSets == nil {
+		f.addedGoFlagSets = make([]*goflag.FlagSet, 0)
+	}
+	f.addedGoFlagSets = append(f.addedGoFlagSets, newSet)
+}
diff --git a/vendor/github.com/spf13/pflag/int.go b/vendor/github.com/spf13/pflag/int.go
new file mode 100644
index 0000000..1474b89
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/int.go
@@ -0,0 +1,84 @@
+package pflag
+
+import "strconv"
+
+// -- int Value
+type intValue int
+
+func newIntValue(val int, p *int) *intValue {
+	*p = val
+	return (*intValue)(p)
+}
+
+func (i *intValue) Set(s string) error {
+	v, err := strconv.ParseInt(s, 0, 64)
+	*i = intValue(v)
+	return err
+}
+
+func (i *intValue) Type() string {
+	return "int"
+}
+
+func (i *intValue) String() string { return strconv.Itoa(int(*i)) }
+
+func intConv(sval string) (interface{}, error) {
+	return strconv.Atoi(sval)
+}
+
+// GetInt return the int value of a flag with the given name
+func (f *FlagSet) GetInt(name string) (int, error) {
+	val, err := f.getFlagType(name, "int", intConv)
+	if err != nil {
+		return 0, err
+	}
+	return val.(int), nil
+}
+
+// IntVar defines an int flag with specified name, default value, and usage string.
+// The argument p points to an int variable in which to store the value of the flag.
+func (f *FlagSet) IntVar(p *int, name string, value int, usage string) {
+	f.VarP(newIntValue(value, p), name, "", usage)
+}
+
+// IntVarP is like IntVar, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) IntVarP(p *int, name, shorthand string, value int, usage string) {
+	f.VarP(newIntValue(value, p), name, shorthand, usage)
+}
+
+// IntVar defines an int flag with specified name, default value, and usage string.
+// The argument p points to an int variable in which to store the value of the flag.
+func IntVar(p *int, name string, value int, usage string) {
+	CommandLine.VarP(newIntValue(value, p), name, "", usage)
+}
+
+// IntVarP is like IntVar, but accepts a shorthand letter that can be used after a single dash.
+func IntVarP(p *int, name, shorthand string, value int, usage string) {
+	CommandLine.VarP(newIntValue(value, p), name, shorthand, usage)
+}
+
+// Int defines an int flag with specified name, default value, and usage string.
+// The return value is the address of an int variable that stores the value of the flag.
+func (f *FlagSet) Int(name string, value int, usage string) *int {
+	p := new(int)
+	f.IntVarP(p, name, "", value, usage)
+	return p
+}
+
+// IntP is like Int, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) IntP(name, shorthand string, value int, usage string) *int {
+	p := new(int)
+	f.IntVarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// Int defines an int flag with specified name, default value, and usage string.
+// The return value is the address of an int variable that stores the value of the flag.
+func Int(name string, value int, usage string) *int {
+	return CommandLine.IntP(name, "", value, usage)
+}
+
+// IntP is like Int, but accepts a shorthand letter that can be used after a single dash.
+func IntP(name, shorthand string, value int, usage string) *int {
+	return CommandLine.IntP(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/int16.go b/vendor/github.com/spf13/pflag/int16.go
new file mode 100644
index 0000000..f1a01d0
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/int16.go
@@ -0,0 +1,88 @@
+package pflag
+
+import "strconv"
+
+// -- int16 Value
+type int16Value int16
+
+func newInt16Value(val int16, p *int16) *int16Value {
+	*p = val
+	return (*int16Value)(p)
+}
+
+func (i *int16Value) Set(s string) error {
+	v, err := strconv.ParseInt(s, 0, 16)
+	*i = int16Value(v)
+	return err
+}
+
+func (i *int16Value) Type() string {
+	return "int16"
+}
+
+func (i *int16Value) String() string { return strconv.FormatInt(int64(*i), 10) }
+
+func int16Conv(sval string) (interface{}, error) {
+	v, err := strconv.ParseInt(sval, 0, 16)
+	if err != nil {
+		return 0, err
+	}
+	return int16(v), nil
+}
+
+// GetInt16 returns the int16 value of a flag with the given name
+func (f *FlagSet) GetInt16(name string) (int16, error) {
+	val, err := f.getFlagType(name, "int16", int16Conv)
+	if err != nil {
+		return 0, err
+	}
+	return val.(int16), nil
+}
+
+// Int16Var defines an int16 flag with specified name, default value, and usage string.
+// The argument p points to an int16 variable in which to store the value of the flag.
+func (f *FlagSet) Int16Var(p *int16, name string, value int16, usage string) {
+	f.VarP(newInt16Value(value, p), name, "", usage)
+}
+
+// Int16VarP is like Int16Var, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Int16VarP(p *int16, name, shorthand string, value int16, usage string) {
+	f.VarP(newInt16Value(value, p), name, shorthand, usage)
+}
+
+// Int16Var defines an int16 flag with specified name, default value, and usage string.
+// The argument p points to an int16 variable in which to store the value of the flag.
+func Int16Var(p *int16, name string, value int16, usage string) {
+	CommandLine.VarP(newInt16Value(value, p), name, "", usage)
+}
+
+// Int16VarP is like Int16Var, but accepts a shorthand letter that can be used after a single dash.
+func Int16VarP(p *int16, name, shorthand string, value int16, usage string) {
+	CommandLine.VarP(newInt16Value(value, p), name, shorthand, usage)
+}
+
+// Int16 defines an int16 flag with specified name, default value, and usage string.
+// The return value is the address of an int16 variable that stores the value of the flag.
+func (f *FlagSet) Int16(name string, value int16, usage string) *int16 {
+	p := new(int16)
+	f.Int16VarP(p, name, "", value, usage)
+	return p
+}
+
+// Int16P is like Int16, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Int16P(name, shorthand string, value int16, usage string) *int16 {
+	p := new(int16)
+	f.Int16VarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// Int16 defines an int16 flag with specified name, default value, and usage string.
+// The return value is the address of an int16 variable that stores the value of the flag.
+func Int16(name string, value int16, usage string) *int16 {
+	return CommandLine.Int16P(name, "", value, usage)
+}
+
+// Int16P is like Int16, but accepts a shorthand letter that can be used after a single dash.
+func Int16P(name, shorthand string, value int16, usage string) *int16 {
+	return CommandLine.Int16P(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/int32.go b/vendor/github.com/spf13/pflag/int32.go
new file mode 100644
index 0000000..9b95944
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/int32.go
@@ -0,0 +1,88 @@
+package pflag
+
+import "strconv"
+
+// -- int32 Value
+type int32Value int32
+
+func newInt32Value(val int32, p *int32) *int32Value {
+	*p = val
+	return (*int32Value)(p)
+}
+
+func (i *int32Value) Set(s string) error {
+	v, err := strconv.ParseInt(s, 0, 32)
+	*i = int32Value(v)
+	return err
+}
+
+func (i *int32Value) Type() string {
+	return "int32"
+}
+
+func (i *int32Value) String() string { return strconv.FormatInt(int64(*i), 10) }
+
+func int32Conv(sval string) (interface{}, error) {
+	v, err := strconv.ParseInt(sval, 0, 32)
+	if err != nil {
+		return 0, err
+	}
+	return int32(v), nil
+}
+
+// GetInt32 return the int32 value of a flag with the given name
+func (f *FlagSet) GetInt32(name string) (int32, error) {
+	val, err := f.getFlagType(name, "int32", int32Conv)
+	if err != nil {
+		return 0, err
+	}
+	return val.(int32), nil
+}
+
+// Int32Var defines an int32 flag with specified name, default value, and usage string.
+// The argument p points to an int32 variable in which to store the value of the flag.
+func (f *FlagSet) Int32Var(p *int32, name string, value int32, usage string) {
+	f.VarP(newInt32Value(value, p), name, "", usage)
+}
+
+// Int32VarP is like Int32Var, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Int32VarP(p *int32, name, shorthand string, value int32, usage string) {
+	f.VarP(newInt32Value(value, p), name, shorthand, usage)
+}
+
+// Int32Var defines an int32 flag with specified name, default value, and usage string.
+// The argument p points to an int32 variable in which to store the value of the flag.
+func Int32Var(p *int32, name string, value int32, usage string) {
+	CommandLine.VarP(newInt32Value(value, p), name, "", usage)
+}
+
+// Int32VarP is like Int32Var, but accepts a shorthand letter that can be used after a single dash.
+func Int32VarP(p *int32, name, shorthand string, value int32, usage string) {
+	CommandLine.VarP(newInt32Value(value, p), name, shorthand, usage)
+}
+
+// Int32 defines an int32 flag with specified name, default value, and usage string.
+// The return value is the address of an int32 variable that stores the value of the flag.
+func (f *FlagSet) Int32(name string, value int32, usage string) *int32 {
+	p := new(int32)
+	f.Int32VarP(p, name, "", value, usage)
+	return p
+}
+
+// Int32P is like Int32, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Int32P(name, shorthand string, value int32, usage string) *int32 {
+	p := new(int32)
+	f.Int32VarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// Int32 defines an int32 flag with specified name, default value, and usage string.
+// The return value is the address of an int32 variable that stores the value of the flag.
+func Int32(name string, value int32, usage string) *int32 {
+	return CommandLine.Int32P(name, "", value, usage)
+}
+
+// Int32P is like Int32, but accepts a shorthand letter that can be used after a single dash.
+func Int32P(name, shorthand string, value int32, usage string) *int32 {
+	return CommandLine.Int32P(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/int64.go b/vendor/github.com/spf13/pflag/int64.go
new file mode 100644
index 0000000..0026d78
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/int64.go
@@ -0,0 +1,84 @@
+package pflag
+
+import "strconv"
+
+// -- int64 Value
+type int64Value int64
+
+func newInt64Value(val int64, p *int64) *int64Value {
+	*p = val
+	return (*int64Value)(p)
+}
+
+func (i *int64Value) Set(s string) error {
+	v, err := strconv.ParseInt(s, 0, 64)
+	*i = int64Value(v)
+	return err
+}
+
+func (i *int64Value) Type() string {
+	return "int64"
+}
+
+func (i *int64Value) String() string { return strconv.FormatInt(int64(*i), 10) }
+
+func int64Conv(sval string) (interface{}, error) {
+	return strconv.ParseInt(sval, 0, 64)
+}
+
+// GetInt64 return the int64 value of a flag with the given name
+func (f *FlagSet) GetInt64(name string) (int64, error) {
+	val, err := f.getFlagType(name, "int64", int64Conv)
+	if err != nil {
+		return 0, err
+	}
+	return val.(int64), nil
+}
+
+// Int64Var defines an int64 flag with specified name, default value, and usage string.
+// The argument p points to an int64 variable in which to store the value of the flag.
+func (f *FlagSet) Int64Var(p *int64, name string, value int64, usage string) {
+	f.VarP(newInt64Value(value, p), name, "", usage)
+}
+
+// Int64VarP is like Int64Var, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Int64VarP(p *int64, name, shorthand string, value int64, usage string) {
+	f.VarP(newInt64Value(value, p), name, shorthand, usage)
+}
+
+// Int64Var defines an int64 flag with specified name, default value, and usage string.
+// The argument p points to an int64 variable in which to store the value of the flag.
+func Int64Var(p *int64, name string, value int64, usage string) {
+	CommandLine.VarP(newInt64Value(value, p), name, "", usage)
+}
+
+// Int64VarP is like Int64Var, but accepts a shorthand letter that can be used after a single dash.
+func Int64VarP(p *int64, name, shorthand string, value int64, usage string) {
+	CommandLine.VarP(newInt64Value(value, p), name, shorthand, usage)
+}
+
+// Int64 defines an int64 flag with specified name, default value, and usage string.
+// The return value is the address of an int64 variable that stores the value of the flag.
+func (f *FlagSet) Int64(name string, value int64, usage string) *int64 {
+	p := new(int64)
+	f.Int64VarP(p, name, "", value, usage)
+	return p
+}
+
+// Int64P is like Int64, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Int64P(name, shorthand string, value int64, usage string) *int64 {
+	p := new(int64)
+	f.Int64VarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// Int64 defines an int64 flag with specified name, default value, and usage string.
+// The return value is the address of an int64 variable that stores the value of the flag.
+func Int64(name string, value int64, usage string) *int64 {
+	return CommandLine.Int64P(name, "", value, usage)
+}
+
+// Int64P is like Int64, but accepts a shorthand letter that can be used after a single dash.
+func Int64P(name, shorthand string, value int64, usage string) *int64 {
+	return CommandLine.Int64P(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/int8.go b/vendor/github.com/spf13/pflag/int8.go
new file mode 100644
index 0000000..4da9222
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/int8.go
@@ -0,0 +1,88 @@
+package pflag
+
+import "strconv"
+
+// -- int8 Value
+type int8Value int8
+
+func newInt8Value(val int8, p *int8) *int8Value {
+	*p = val
+	return (*int8Value)(p)
+}
+
+func (i *int8Value) Set(s string) error {
+	v, err := strconv.ParseInt(s, 0, 8)
+	*i = int8Value(v)
+	return err
+}
+
+func (i *int8Value) Type() string {
+	return "int8"
+}
+
+func (i *int8Value) String() string { return strconv.FormatInt(int64(*i), 10) }
+
+func int8Conv(sval string) (interface{}, error) {
+	v, err := strconv.ParseInt(sval, 0, 8)
+	if err != nil {
+		return 0, err
+	}
+	return int8(v), nil
+}
+
+// GetInt8 return the int8 value of a flag with the given name
+func (f *FlagSet) GetInt8(name string) (int8, error) {
+	val, err := f.getFlagType(name, "int8", int8Conv)
+	if err != nil {
+		return 0, err
+	}
+	return val.(int8), nil
+}
+
+// Int8Var defines an int8 flag with specified name, default value, and usage string.
+// The argument p points to an int8 variable in which to store the value of the flag.
+func (f *FlagSet) Int8Var(p *int8, name string, value int8, usage string) {
+	f.VarP(newInt8Value(value, p), name, "", usage)
+}
+
+// Int8VarP is like Int8Var, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Int8VarP(p *int8, name, shorthand string, value int8, usage string) {
+	f.VarP(newInt8Value(value, p), name, shorthand, usage)
+}
+
+// Int8Var defines an int8 flag with specified name, default value, and usage string.
+// The argument p points to an int8 variable in which to store the value of the flag.
+func Int8Var(p *int8, name string, value int8, usage string) {
+	CommandLine.VarP(newInt8Value(value, p), name, "", usage)
+}
+
+// Int8VarP is like Int8Var, but accepts a shorthand letter that can be used after a single dash.
+func Int8VarP(p *int8, name, shorthand string, value int8, usage string) {
+	CommandLine.VarP(newInt8Value(value, p), name, shorthand, usage)
+}
+
+// Int8 defines an int8 flag with specified name, default value, and usage string.
+// The return value is the address of an int8 variable that stores the value of the flag.
+func (f *FlagSet) Int8(name string, value int8, usage string) *int8 {
+	p := new(int8)
+	f.Int8VarP(p, name, "", value, usage)
+	return p
+}
+
+// Int8P is like Int8, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Int8P(name, shorthand string, value int8, usage string) *int8 {
+	p := new(int8)
+	f.Int8VarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// Int8 defines an int8 flag with specified name, default value, and usage string.
+// The return value is the address of an int8 variable that stores the value of the flag.
+func Int8(name string, value int8, usage string) *int8 {
+	return CommandLine.Int8P(name, "", value, usage)
+}
+
+// Int8P is like Int8, but accepts a shorthand letter that can be used after a single dash.
+func Int8P(name, shorthand string, value int8, usage string) *int8 {
+	return CommandLine.Int8P(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/int_slice.go b/vendor/github.com/spf13/pflag/int_slice.go
new file mode 100644
index 0000000..1e7c9ed
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/int_slice.go
@@ -0,0 +1,128 @@
+package pflag
+
+import (
+	"fmt"
+	"strconv"
+	"strings"
+)
+
+// -- intSlice Value
+type intSliceValue struct {
+	value   *[]int
+	changed bool
+}
+
+func newIntSliceValue(val []int, p *[]int) *intSliceValue {
+	isv := new(intSliceValue)
+	isv.value = p
+	*isv.value = val
+	return isv
+}
+
+func (s *intSliceValue) Set(val string) error {
+	ss := strings.Split(val, ",")
+	out := make([]int, len(ss))
+	for i, d := range ss {
+		var err error
+		out[i], err = strconv.Atoi(d)
+		if err != nil {
+			return err
+		}
+
+	}
+	if !s.changed {
+		*s.value = out
+	} else {
+		*s.value = append(*s.value, out...)
+	}
+	s.changed = true
+	return nil
+}
+
+func (s *intSliceValue) Type() string {
+	return "intSlice"
+}
+
+func (s *intSliceValue) String() string {
+	out := make([]string, len(*s.value))
+	for i, d := range *s.value {
+		out[i] = fmt.Sprintf("%d", d)
+	}
+	return "[" + strings.Join(out, ",") + "]"
+}
+
+func intSliceConv(val string) (interface{}, error) {
+	val = strings.Trim(val, "[]")
+	// Empty string would cause a slice with one (empty) entry
+	if len(val) == 0 {
+		return []int{}, nil
+	}
+	ss := strings.Split(val, ",")
+	out := make([]int, len(ss))
+	for i, d := range ss {
+		var err error
+		out[i], err = strconv.Atoi(d)
+		if err != nil {
+			return nil, err
+		}
+
+	}
+	return out, nil
+}
+
+// GetIntSlice return the []int value of a flag with the given name
+func (f *FlagSet) GetIntSlice(name string) ([]int, error) {
+	val, err := f.getFlagType(name, "intSlice", intSliceConv)
+	if err != nil {
+		return []int{}, err
+	}
+	return val.([]int), nil
+}
+
+// IntSliceVar defines a intSlice flag with specified name, default value, and usage string.
+// The argument p points to a []int variable in which to store the value of the flag.
+func (f *FlagSet) IntSliceVar(p *[]int, name string, value []int, usage string) {
+	f.VarP(newIntSliceValue(value, p), name, "", usage)
+}
+
+// IntSliceVarP is like IntSliceVar, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) IntSliceVarP(p *[]int, name, shorthand string, value []int, usage string) {
+	f.VarP(newIntSliceValue(value, p), name, shorthand, usage)
+}
+
+// IntSliceVar defines a int[] flag with specified name, default value, and usage string.
+// The argument p points to a int[] variable in which to store the value of the flag.
+func IntSliceVar(p *[]int, name string, value []int, usage string) {
+	CommandLine.VarP(newIntSliceValue(value, p), name, "", usage)
+}
+
+// IntSliceVarP is like IntSliceVar, but accepts a shorthand letter that can be used after a single dash.
+func IntSliceVarP(p *[]int, name, shorthand string, value []int, usage string) {
+	CommandLine.VarP(newIntSliceValue(value, p), name, shorthand, usage)
+}
+
+// IntSlice defines a []int flag with specified name, default value, and usage string.
+// The return value is the address of a []int variable that stores the value of the flag.
+func (f *FlagSet) IntSlice(name string, value []int, usage string) *[]int {
+	p := []int{}
+	f.IntSliceVarP(&p, name, "", value, usage)
+	return &p
+}
+
+// IntSliceP is like IntSlice, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) IntSliceP(name, shorthand string, value []int, usage string) *[]int {
+	p := []int{}
+	f.IntSliceVarP(&p, name, shorthand, value, usage)
+	return &p
+}
+
+// IntSlice defines a []int flag with specified name, default value, and usage string.
+// The return value is the address of a []int variable that stores the value of the flag.
+func IntSlice(name string, value []int, usage string) *[]int {
+	return CommandLine.IntSliceP(name, "", value, usage)
+}
+
+// IntSliceP is like IntSlice, but accepts a shorthand letter that can be used after a single dash.
+func IntSliceP(name, shorthand string, value []int, usage string) *[]int {
+	return CommandLine.IntSliceP(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/ip.go b/vendor/github.com/spf13/pflag/ip.go
new file mode 100644
index 0000000..3d414ba
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/ip.go
@@ -0,0 +1,94 @@
+package pflag
+
+import (
+	"fmt"
+	"net"
+	"strings"
+)
+
+// -- net.IP value
+type ipValue net.IP
+
+func newIPValue(val net.IP, p *net.IP) *ipValue {
+	*p = val
+	return (*ipValue)(p)
+}
+
+func (i *ipValue) String() string { return net.IP(*i).String() }
+func (i *ipValue) Set(s string) error {
+	ip := net.ParseIP(strings.TrimSpace(s))
+	if ip == nil {
+		return fmt.Errorf("failed to parse IP: %q", s)
+	}
+	*i = ipValue(ip)
+	return nil
+}
+
+func (i *ipValue) Type() string {
+	return "ip"
+}
+
+func ipConv(sval string) (interface{}, error) {
+	ip := net.ParseIP(sval)
+	if ip != nil {
+		return ip, nil
+	}
+	return nil, fmt.Errorf("invalid string being converted to IP address: %s", sval)
+}
+
+// GetIP return the net.IP value of a flag with the given name
+func (f *FlagSet) GetIP(name string) (net.IP, error) {
+	val, err := f.getFlagType(name, "ip", ipConv)
+	if err != nil {
+		return nil, err
+	}
+	return val.(net.IP), nil
+}
+
+// IPVar defines an net.IP flag with specified name, default value, and usage string.
+// The argument p points to an net.IP variable in which to store the value of the flag.
+func (f *FlagSet) IPVar(p *net.IP, name string, value net.IP, usage string) {
+	f.VarP(newIPValue(value, p), name, "", usage)
+}
+
+// IPVarP is like IPVar, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) IPVarP(p *net.IP, name, shorthand string, value net.IP, usage string) {
+	f.VarP(newIPValue(value, p), name, shorthand, usage)
+}
+
+// IPVar defines an net.IP flag with specified name, default value, and usage string.
+// The argument p points to an net.IP variable in which to store the value of the flag.
+func IPVar(p *net.IP, name string, value net.IP, usage string) {
+	CommandLine.VarP(newIPValue(value, p), name, "", usage)
+}
+
+// IPVarP is like IPVar, but accepts a shorthand letter that can be used after a single dash.
+func IPVarP(p *net.IP, name, shorthand string, value net.IP, usage string) {
+	CommandLine.VarP(newIPValue(value, p), name, shorthand, usage)
+}
+
+// IP defines an net.IP flag with specified name, default value, and usage string.
+// The return value is the address of an net.IP variable that stores the value of the flag.
+func (f *FlagSet) IP(name string, value net.IP, usage string) *net.IP {
+	p := new(net.IP)
+	f.IPVarP(p, name, "", value, usage)
+	return p
+}
+
+// IPP is like IP, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) IPP(name, shorthand string, value net.IP, usage string) *net.IP {
+	p := new(net.IP)
+	f.IPVarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// IP defines an net.IP flag with specified name, default value, and usage string.
+// The return value is the address of an net.IP variable that stores the value of the flag.
+func IP(name string, value net.IP, usage string) *net.IP {
+	return CommandLine.IPP(name, "", value, usage)
+}
+
+// IPP is like IP, but accepts a shorthand letter that can be used after a single dash.
+func IPP(name, shorthand string, value net.IP, usage string) *net.IP {
+	return CommandLine.IPP(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/ip_slice.go b/vendor/github.com/spf13/pflag/ip_slice.go
new file mode 100644
index 0000000..7dd196f
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/ip_slice.go
@@ -0,0 +1,148 @@
+package pflag
+
+import (
+	"fmt"
+	"io"
+	"net"
+	"strings"
+)
+
+// -- ipSlice Value
+type ipSliceValue struct {
+	value   *[]net.IP
+	changed bool
+}
+
+func newIPSliceValue(val []net.IP, p *[]net.IP) *ipSliceValue {
+	ipsv := new(ipSliceValue)
+	ipsv.value = p
+	*ipsv.value = val
+	return ipsv
+}
+
+// Set converts, and assigns, the comma-separated IP argument string representation as the []net.IP value of this flag.
+// If Set is called on a flag that already has a []net.IP assigned, the newly converted values will be appended.
+func (s *ipSliceValue) Set(val string) error {
+
+	// remove all quote characters
+	rmQuote := strings.NewReplacer(`"`, "", `'`, "", "`", "")
+
+	// read flag arguments with CSV parser
+	ipStrSlice, err := readAsCSV(rmQuote.Replace(val))
+	if err != nil && err != io.EOF {
+		return err
+	}
+
+	// parse ip values into slice
+	out := make([]net.IP, 0, len(ipStrSlice))
+	for _, ipStr := range ipStrSlice {
+		ip := net.ParseIP(strings.TrimSpace(ipStr))
+		if ip == nil {
+			return fmt.Errorf("invalid string being converted to IP address: %s", ipStr)
+		}
+		out = append(out, ip)
+	}
+
+	if !s.changed {
+		*s.value = out
+	} else {
+		*s.value = append(*s.value, out...)
+	}
+
+	s.changed = true
+
+	return nil
+}
+
+// Type returns a string that uniquely represents this flag's type.
+func (s *ipSliceValue) Type() string {
+	return "ipSlice"
+}
+
+// String defines a "native" format for this net.IP slice flag value.
+func (s *ipSliceValue) String() string {
+
+	ipStrSlice := make([]string, len(*s.value))
+	for i, ip := range *s.value {
+		ipStrSlice[i] = ip.String()
+	}
+
+	out, _ := writeAsCSV(ipStrSlice)
+
+	return "[" + out + "]"
+}
+
+func ipSliceConv(val string) (interface{}, error) {
+	val = strings.Trim(val, "[]")
+	// Emtpy string would cause a slice with one (empty) entry
+	if len(val) == 0 {
+		return []net.IP{}, nil
+	}
+	ss := strings.Split(val, ",")
+	out := make([]net.IP, len(ss))
+	for i, sval := range ss {
+		ip := net.ParseIP(strings.TrimSpace(sval))
+		if ip == nil {
+			return nil, fmt.Errorf("invalid string being converted to IP address: %s", sval)
+		}
+		out[i] = ip
+	}
+	return out, nil
+}
+
+// GetIPSlice returns the []net.IP value of a flag with the given name
+func (f *FlagSet) GetIPSlice(name string) ([]net.IP, error) {
+	val, err := f.getFlagType(name, "ipSlice", ipSliceConv)
+	if err != nil {
+		return []net.IP{}, err
+	}
+	return val.([]net.IP), nil
+}
+
+// IPSliceVar defines a ipSlice flag with specified name, default value, and usage string.
+// The argument p points to a []net.IP variable in which to store the value of the flag.
+func (f *FlagSet) IPSliceVar(p *[]net.IP, name string, value []net.IP, usage string) {
+	f.VarP(newIPSliceValue(value, p), name, "", usage)
+}
+
+// IPSliceVarP is like IPSliceVar, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) IPSliceVarP(p *[]net.IP, name, shorthand string, value []net.IP, usage string) {
+	f.VarP(newIPSliceValue(value, p), name, shorthand, usage)
+}
+
+// IPSliceVar defines a []net.IP flag with specified name, default value, and usage string.
+// The argument p points to a []net.IP variable in which to store the value of the flag.
+func IPSliceVar(p *[]net.IP, name string, value []net.IP, usage string) {
+	CommandLine.VarP(newIPSliceValue(value, p), name, "", usage)
+}
+
+// IPSliceVarP is like IPSliceVar, but accepts a shorthand letter that can be used after a single dash.
+func IPSliceVarP(p *[]net.IP, name, shorthand string, value []net.IP, usage string) {
+	CommandLine.VarP(newIPSliceValue(value, p), name, shorthand, usage)
+}
+
+// IPSlice defines a []net.IP flag with specified name, default value, and usage string.
+// The return value is the address of a []net.IP variable that stores the value of that flag.
+func (f *FlagSet) IPSlice(name string, value []net.IP, usage string) *[]net.IP {
+	p := []net.IP{}
+	f.IPSliceVarP(&p, name, "", value, usage)
+	return &p
+}
+
+// IPSliceP is like IPSlice, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) IPSliceP(name, shorthand string, value []net.IP, usage string) *[]net.IP {
+	p := []net.IP{}
+	f.IPSliceVarP(&p, name, shorthand, value, usage)
+	return &p
+}
+
+// IPSlice defines a []net.IP flag with specified name, default value, and usage string.
+// The return value is the address of a []net.IP variable that stores the value of the flag.
+func IPSlice(name string, value []net.IP, usage string) *[]net.IP {
+	return CommandLine.IPSliceP(name, "", value, usage)
+}
+
+// IPSliceP is like IPSlice, but accepts a shorthand letter that can be used after a single dash.
+func IPSliceP(name, shorthand string, value []net.IP, usage string) *[]net.IP {
+	return CommandLine.IPSliceP(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/ipmask.go b/vendor/github.com/spf13/pflag/ipmask.go
new file mode 100644
index 0000000..5bd44bd
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/ipmask.go
@@ -0,0 +1,122 @@
+package pflag
+
+import (
+	"fmt"
+	"net"
+	"strconv"
+)
+
+// -- net.IPMask value
+type ipMaskValue net.IPMask
+
+func newIPMaskValue(val net.IPMask, p *net.IPMask) *ipMaskValue {
+	*p = val
+	return (*ipMaskValue)(p)
+}
+
+func (i *ipMaskValue) String() string { return net.IPMask(*i).String() }
+func (i *ipMaskValue) Set(s string) error {
+	ip := ParseIPv4Mask(s)
+	if ip == nil {
+		return fmt.Errorf("failed to parse IP mask: %q", s)
+	}
+	*i = ipMaskValue(ip)
+	return nil
+}
+
+func (i *ipMaskValue) Type() string {
+	return "ipMask"
+}
+
+// ParseIPv4Mask written in IP form (e.g. 255.255.255.0).
+// This function should really belong to the net package.
+func ParseIPv4Mask(s string) net.IPMask {
+	mask := net.ParseIP(s)
+	if mask == nil {
+		if len(s) != 8 {
+			return nil
+		}
+		// net.IPMask.String() actually outputs things like ffffff00
+		// so write a horrible parser for that as well  :-(
+		m := []int{}
+		for i := 0; i < 4; i++ {
+			b := "0x" + s[2*i:2*i+2]
+			d, err := strconv.ParseInt(b, 0, 0)
+			if err != nil {
+				return nil
+			}
+			m = append(m, int(d))
+		}
+		s := fmt.Sprintf("%d.%d.%d.%d", m[0], m[1], m[2], m[3])
+		mask = net.ParseIP(s)
+		if mask == nil {
+			return nil
+		}
+	}
+	return net.IPv4Mask(mask[12], mask[13], mask[14], mask[15])
+}
+
+func parseIPv4Mask(sval string) (interface{}, error) {
+	mask := ParseIPv4Mask(sval)
+	if mask == nil {
+		return nil, fmt.Errorf("unable to parse %s as net.IPMask", sval)
+	}
+	return mask, nil
+}
+
+// GetIPv4Mask return the net.IPv4Mask value of a flag with the given name
+func (f *FlagSet) GetIPv4Mask(name string) (net.IPMask, error) {
+	val, err := f.getFlagType(name, "ipMask", parseIPv4Mask)
+	if err != nil {
+		return nil, err
+	}
+	return val.(net.IPMask), nil
+}
+
+// IPMaskVar defines an net.IPMask flag with specified name, default value, and usage string.
+// The argument p points to an net.IPMask variable in which to store the value of the flag.
+func (f *FlagSet) IPMaskVar(p *net.IPMask, name string, value net.IPMask, usage string) {
+	f.VarP(newIPMaskValue(value, p), name, "", usage)
+}
+
+// IPMaskVarP is like IPMaskVar, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) IPMaskVarP(p *net.IPMask, name, shorthand string, value net.IPMask, usage string) {
+	f.VarP(newIPMaskValue(value, p), name, shorthand, usage)
+}
+
+// IPMaskVar defines an net.IPMask flag with specified name, default value, and usage string.
+// The argument p points to an net.IPMask variable in which to store the value of the flag.
+func IPMaskVar(p *net.IPMask, name string, value net.IPMask, usage string) {
+	CommandLine.VarP(newIPMaskValue(value, p), name, "", usage)
+}
+
+// IPMaskVarP is like IPMaskVar, but accepts a shorthand letter that can be used after a single dash.
+func IPMaskVarP(p *net.IPMask, name, shorthand string, value net.IPMask, usage string) {
+	CommandLine.VarP(newIPMaskValue(value, p), name, shorthand, usage)
+}
+
+// IPMask defines an net.IPMask flag with specified name, default value, and usage string.
+// The return value is the address of an net.IPMask variable that stores the value of the flag.
+func (f *FlagSet) IPMask(name string, value net.IPMask, usage string) *net.IPMask {
+	p := new(net.IPMask)
+	f.IPMaskVarP(p, name, "", value, usage)
+	return p
+}
+
+// IPMaskP is like IPMask, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) IPMaskP(name, shorthand string, value net.IPMask, usage string) *net.IPMask {
+	p := new(net.IPMask)
+	f.IPMaskVarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// IPMask defines an net.IPMask flag with specified name, default value, and usage string.
+// The return value is the address of an net.IPMask variable that stores the value of the flag.
+func IPMask(name string, value net.IPMask, usage string) *net.IPMask {
+	return CommandLine.IPMaskP(name, "", value, usage)
+}
+
+// IPMaskP is like IP, but accepts a shorthand letter that can be used after a single dash.
+func IPMaskP(name, shorthand string, value net.IPMask, usage string) *net.IPMask {
+	return CommandLine.IPMaskP(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/ipnet.go b/vendor/github.com/spf13/pflag/ipnet.go
new file mode 100644
index 0000000..e2c1b8b
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/ipnet.go
@@ -0,0 +1,98 @@
+package pflag
+
+import (
+	"fmt"
+	"net"
+	"strings"
+)
+
+// IPNet adapts net.IPNet for use as a flag.
+type ipNetValue net.IPNet
+
+func (ipnet ipNetValue) String() string {
+	n := net.IPNet(ipnet)
+	return n.String()
+}
+
+func (ipnet *ipNetValue) Set(value string) error {
+	_, n, err := net.ParseCIDR(strings.TrimSpace(value))
+	if err != nil {
+		return err
+	}
+	*ipnet = ipNetValue(*n)
+	return nil
+}
+
+func (*ipNetValue) Type() string {
+	return "ipNet"
+}
+
+func newIPNetValue(val net.IPNet, p *net.IPNet) *ipNetValue {
+	*p = val
+	return (*ipNetValue)(p)
+}
+
+func ipNetConv(sval string) (interface{}, error) {
+	_, n, err := net.ParseCIDR(strings.TrimSpace(sval))
+	if err == nil {
+		return *n, nil
+	}
+	return nil, fmt.Errorf("invalid string being converted to IPNet: %s", sval)
+}
+
+// GetIPNet return the net.IPNet value of a flag with the given name
+func (f *FlagSet) GetIPNet(name string) (net.IPNet, error) {
+	val, err := f.getFlagType(name, "ipNet", ipNetConv)
+	if err != nil {
+		return net.IPNet{}, err
+	}
+	return val.(net.IPNet), nil
+}
+
+// IPNetVar defines an net.IPNet flag with specified name, default value, and usage string.
+// The argument p points to an net.IPNet variable in which to store the value of the flag.
+func (f *FlagSet) IPNetVar(p *net.IPNet, name string, value net.IPNet, usage string) {
+	f.VarP(newIPNetValue(value, p), name, "", usage)
+}
+
+// IPNetVarP is like IPNetVar, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) IPNetVarP(p *net.IPNet, name, shorthand string, value net.IPNet, usage string) {
+	f.VarP(newIPNetValue(value, p), name, shorthand, usage)
+}
+
+// IPNetVar defines an net.IPNet flag with specified name, default value, and usage string.
+// The argument p points to an net.IPNet variable in which to store the value of the flag.
+func IPNetVar(p *net.IPNet, name string, value net.IPNet, usage string) {
+	CommandLine.VarP(newIPNetValue(value, p), name, "", usage)
+}
+
+// IPNetVarP is like IPNetVar, but accepts a shorthand letter that can be used after a single dash.
+func IPNetVarP(p *net.IPNet, name, shorthand string, value net.IPNet, usage string) {
+	CommandLine.VarP(newIPNetValue(value, p), name, shorthand, usage)
+}
+
+// IPNet defines an net.IPNet flag with specified name, default value, and usage string.
+// The return value is the address of an net.IPNet variable that stores the value of the flag.
+func (f *FlagSet) IPNet(name string, value net.IPNet, usage string) *net.IPNet {
+	p := new(net.IPNet)
+	f.IPNetVarP(p, name, "", value, usage)
+	return p
+}
+
+// IPNetP is like IPNet, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) IPNetP(name, shorthand string, value net.IPNet, usage string) *net.IPNet {
+	p := new(net.IPNet)
+	f.IPNetVarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// IPNet defines an net.IPNet flag with specified name, default value, and usage string.
+// The return value is the address of an net.IPNet variable that stores the value of the flag.
+func IPNet(name string, value net.IPNet, usage string) *net.IPNet {
+	return CommandLine.IPNetP(name, "", value, usage)
+}
+
+// IPNetP is like IPNet, but accepts a shorthand letter that can be used after a single dash.
+func IPNetP(name, shorthand string, value net.IPNet, usage string) *net.IPNet {
+	return CommandLine.IPNetP(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/string.go b/vendor/github.com/spf13/pflag/string.go
new file mode 100644
index 0000000..04e0a26
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/string.go
@@ -0,0 +1,80 @@
+package pflag
+
+// -- string Value
+type stringValue string
+
+func newStringValue(val string, p *string) *stringValue {
+	*p = val
+	return (*stringValue)(p)
+}
+
+func (s *stringValue) Set(val string) error {
+	*s = stringValue(val)
+	return nil
+}
+func (s *stringValue) Type() string {
+	return "string"
+}
+
+func (s *stringValue) String() string { return string(*s) }
+
+func stringConv(sval string) (interface{}, error) {
+	return sval, nil
+}
+
+// GetString return the string value of a flag with the given name
+func (f *FlagSet) GetString(name string) (string, error) {
+	val, err := f.getFlagType(name, "string", stringConv)
+	if err != nil {
+		return "", err
+	}
+	return val.(string), nil
+}
+
+// StringVar defines a string flag with specified name, default value, and usage string.
+// The argument p points to a string variable in which to store the value of the flag.
+func (f *FlagSet) StringVar(p *string, name string, value string, usage string) {
+	f.VarP(newStringValue(value, p), name, "", usage)
+}
+
+// StringVarP is like StringVar, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) StringVarP(p *string, name, shorthand string, value string, usage string) {
+	f.VarP(newStringValue(value, p), name, shorthand, usage)
+}
+
+// StringVar defines a string flag with specified name, default value, and usage string.
+// The argument p points to a string variable in which to store the value of the flag.
+func StringVar(p *string, name string, value string, usage string) {
+	CommandLine.VarP(newStringValue(value, p), name, "", usage)
+}
+
+// StringVarP is like StringVar, but accepts a shorthand letter that can be used after a single dash.
+func StringVarP(p *string, name, shorthand string, value string, usage string) {
+	CommandLine.VarP(newStringValue(value, p), name, shorthand, usage)
+}
+
+// String defines a string flag with specified name, default value, and usage string.
+// The return value is the address of a string variable that stores the value of the flag.
+func (f *FlagSet) String(name string, value string, usage string) *string {
+	p := new(string)
+	f.StringVarP(p, name, "", value, usage)
+	return p
+}
+
+// StringP is like String, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) StringP(name, shorthand string, value string, usage string) *string {
+	p := new(string)
+	f.StringVarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// String defines a string flag with specified name, default value, and usage string.
+// The return value is the address of a string variable that stores the value of the flag.
+func String(name string, value string, usage string) *string {
+	return CommandLine.StringP(name, "", value, usage)
+}
+
+// StringP is like String, but accepts a shorthand letter that can be used after a single dash.
+func StringP(name, shorthand string, value string, usage string) *string {
+	return CommandLine.StringP(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/string_array.go b/vendor/github.com/spf13/pflag/string_array.go
new file mode 100644
index 0000000..fa7bc60
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/string_array.go
@@ -0,0 +1,103 @@
+package pflag
+
+// -- stringArray Value
+type stringArrayValue struct {
+	value   *[]string
+	changed bool
+}
+
+func newStringArrayValue(val []string, p *[]string) *stringArrayValue {
+	ssv := new(stringArrayValue)
+	ssv.value = p
+	*ssv.value = val
+	return ssv
+}
+
+func (s *stringArrayValue) Set(val string) error {
+	if !s.changed {
+		*s.value = []string{val}
+		s.changed = true
+	} else {
+		*s.value = append(*s.value, val)
+	}
+	return nil
+}
+
+func (s *stringArrayValue) Type() string {
+	return "stringArray"
+}
+
+func (s *stringArrayValue) String() string {
+	str, _ := writeAsCSV(*s.value)
+	return "[" + str + "]"
+}
+
+func stringArrayConv(sval string) (interface{}, error) {
+	sval = sval[1 : len(sval)-1]
+	// An empty string would cause a array with one (empty) string
+	if len(sval) == 0 {
+		return []string{}, nil
+	}
+	return readAsCSV(sval)
+}
+
+// GetStringArray return the []string value of a flag with the given name
+func (f *FlagSet) GetStringArray(name string) ([]string, error) {
+	val, err := f.getFlagType(name, "stringArray", stringArrayConv)
+	if err != nil {
+		return []string{}, err
+	}
+	return val.([]string), nil
+}
+
+// StringArrayVar defines a string flag with specified name, default value, and usage string.
+// The argument p points to a []string variable in which to store the values of the multiple flags.
+// The value of each argument will not try to be separated by comma. Use a StringSlice for that.
+func (f *FlagSet) StringArrayVar(p *[]string, name string, value []string, usage string) {
+	f.VarP(newStringArrayValue(value, p), name, "", usage)
+}
+
+// StringArrayVarP is like StringArrayVar, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) StringArrayVarP(p *[]string, name, shorthand string, value []string, usage string) {
+	f.VarP(newStringArrayValue(value, p), name, shorthand, usage)
+}
+
+// StringArrayVar defines a string flag with specified name, default value, and usage string.
+// The argument p points to a []string variable in which to store the value of the flag.
+// The value of each argument will not try to be separated by comma. Use a StringSlice for that.
+func StringArrayVar(p *[]string, name string, value []string, usage string) {
+	CommandLine.VarP(newStringArrayValue(value, p), name, "", usage)
+}
+
+// StringArrayVarP is like StringArrayVar, but accepts a shorthand letter that can be used after a single dash.
+func StringArrayVarP(p *[]string, name, shorthand string, value []string, usage string) {
+	CommandLine.VarP(newStringArrayValue(value, p), name, shorthand, usage)
+}
+
+// StringArray defines a string flag with specified name, default value, and usage string.
+// The return value is the address of a []string variable that stores the value of the flag.
+// The value of each argument will not try to be separated by comma. Use a StringSlice for that.
+func (f *FlagSet) StringArray(name string, value []string, usage string) *[]string {
+	p := []string{}
+	f.StringArrayVarP(&p, name, "", value, usage)
+	return &p
+}
+
+// StringArrayP is like StringArray, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) StringArrayP(name, shorthand string, value []string, usage string) *[]string {
+	p := []string{}
+	f.StringArrayVarP(&p, name, shorthand, value, usage)
+	return &p
+}
+
+// StringArray defines a string flag with specified name, default value, and usage string.
+// The return value is the address of a []string variable that stores the value of the flag.
+// The value of each argument will not try to be separated by comma. Use a StringSlice for that.
+func StringArray(name string, value []string, usage string) *[]string {
+	return CommandLine.StringArrayP(name, "", value, usage)
+}
+
+// StringArrayP is like StringArray, but accepts a shorthand letter that can be used after a single dash.
+func StringArrayP(name, shorthand string, value []string, usage string) *[]string {
+	return CommandLine.StringArrayP(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/string_slice.go b/vendor/github.com/spf13/pflag/string_slice.go
new file mode 100644
index 0000000..0cd3ccc
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/string_slice.go
@@ -0,0 +1,149 @@
+package pflag
+
+import (
+	"bytes"
+	"encoding/csv"
+	"strings"
+)
+
+// -- stringSlice Value
+type stringSliceValue struct {
+	value   *[]string
+	changed bool
+}
+
+func newStringSliceValue(val []string, p *[]string) *stringSliceValue {
+	ssv := new(stringSliceValue)
+	ssv.value = p
+	*ssv.value = val
+	return ssv
+}
+
+func readAsCSV(val string) ([]string, error) {
+	if val == "" {
+		return []string{}, nil
+	}
+	stringReader := strings.NewReader(val)
+	csvReader := csv.NewReader(stringReader)
+	return csvReader.Read()
+}
+
+func writeAsCSV(vals []string) (string, error) {
+	b := &bytes.Buffer{}
+	w := csv.NewWriter(b)
+	err := w.Write(vals)
+	if err != nil {
+		return "", err
+	}
+	w.Flush()
+	return strings.TrimSuffix(b.String(), "\n"), nil
+}
+
+func (s *stringSliceValue) Set(val string) error {
+	v, err := readAsCSV(val)
+	if err != nil {
+		return err
+	}
+	if !s.changed {
+		*s.value = v
+	} else {
+		*s.value = append(*s.value, v...)
+	}
+	s.changed = true
+	return nil
+}
+
+func (s *stringSliceValue) Type() string {
+	return "stringSlice"
+}
+
+func (s *stringSliceValue) String() string {
+	str, _ := writeAsCSV(*s.value)
+	return "[" + str + "]"
+}
+
+func stringSliceConv(sval string) (interface{}, error) {
+	sval = sval[1 : len(sval)-1]
+	// An empty string would cause a slice with one (empty) string
+	if len(sval) == 0 {
+		return []string{}, nil
+	}
+	return readAsCSV(sval)
+}
+
+// GetStringSlice return the []string value of a flag with the given name
+func (f *FlagSet) GetStringSlice(name string) ([]string, error) {
+	val, err := f.getFlagType(name, "stringSlice", stringSliceConv)
+	if err != nil {
+		return []string{}, err
+	}
+	return val.([]string), nil
+}
+
+// StringSliceVar defines a string flag with specified name, default value, and usage string.
+// The argument p points to a []string variable in which to store the value of the flag.
+// Compared to StringArray flags, StringSlice flags take comma-separated value as arguments and split them accordingly.
+// For example:
+//   --ss="v1,v2" -ss="v3"
+// will result in
+//   []string{"v1", "v2", "v3"}
+func (f *FlagSet) StringSliceVar(p *[]string, name string, value []string, usage string) {
+	f.VarP(newStringSliceValue(value, p), name, "", usage)
+}
+
+// StringSliceVarP is like StringSliceVar, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) StringSliceVarP(p *[]string, name, shorthand string, value []string, usage string) {
+	f.VarP(newStringSliceValue(value, p), name, shorthand, usage)
+}
+
+// StringSliceVar defines a string flag with specified name, default value, and usage string.
+// The argument p points to a []string variable in which to store the value of the flag.
+// Compared to StringArray flags, StringSlice flags take comma-separated value as arguments and split them accordingly.
+// For example:
+//   --ss="v1,v2" -ss="v3"
+// will result in
+//   []string{"v1", "v2", "v3"}
+func StringSliceVar(p *[]string, name string, value []string, usage string) {
+	CommandLine.VarP(newStringSliceValue(value, p), name, "", usage)
+}
+
+// StringSliceVarP is like StringSliceVar, but accepts a shorthand letter that can be used after a single dash.
+func StringSliceVarP(p *[]string, name, shorthand string, value []string, usage string) {
+	CommandLine.VarP(newStringSliceValue(value, p), name, shorthand, usage)
+}
+
+// StringSlice defines a string flag with specified name, default value, and usage string.
+// The return value is the address of a []string variable that stores the value of the flag.
+// Compared to StringArray flags, StringSlice flags take comma-separated value as arguments and split them accordingly.
+// For example:
+//   --ss="v1,v2" -ss="v3"
+// will result in
+//   []string{"v1", "v2", "v3"}
+func (f *FlagSet) StringSlice(name string, value []string, usage string) *[]string {
+	p := []string{}
+	f.StringSliceVarP(&p, name, "", value, usage)
+	return &p
+}
+
+// StringSliceP is like StringSlice, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) StringSliceP(name, shorthand string, value []string, usage string) *[]string {
+	p := []string{}
+	f.StringSliceVarP(&p, name, shorthand, value, usage)
+	return &p
+}
+
+// StringSlice defines a string flag with specified name, default value, and usage string.
+// The return value is the address of a []string variable that stores the value of the flag.
+// Compared to StringArray flags, StringSlice flags take comma-separated value as arguments and split them accordingly.
+// For example:
+//   --ss="v1,v2" -ss="v3"
+// will result in
+//   []string{"v1", "v2", "v3"}
+func StringSlice(name string, value []string, usage string) *[]string {
+	return CommandLine.StringSliceP(name, "", value, usage)
+}
+
+// StringSliceP is like StringSlice, but accepts a shorthand letter that can be used after a single dash.
+func StringSliceP(name, shorthand string, value []string, usage string) *[]string {
+	return CommandLine.StringSliceP(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/string_to_int.go b/vendor/github.com/spf13/pflag/string_to_int.go
new file mode 100644
index 0000000..5ceda39
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/string_to_int.go
@@ -0,0 +1,149 @@
+package pflag
+
+import (
+	"bytes"
+	"fmt"
+	"strconv"
+	"strings"
+)
+
+// -- stringToInt Value
+type stringToIntValue struct {
+	value   *map[string]int
+	changed bool
+}
+
+func newStringToIntValue(val map[string]int, p *map[string]int) *stringToIntValue {
+	ssv := new(stringToIntValue)
+	ssv.value = p
+	*ssv.value = val
+	return ssv
+}
+
+// Format: a=1,b=2
+func (s *stringToIntValue) Set(val string) error {
+	ss := strings.Split(val, ",")
+	out := make(map[string]int, len(ss))
+	for _, pair := range ss {
+		kv := strings.SplitN(pair, "=", 2)
+		if len(kv) != 2 {
+			return fmt.Errorf("%s must be formatted as key=value", pair)
+		}
+		var err error
+		out[kv[0]], err = strconv.Atoi(kv[1])
+		if err != nil {
+			return err
+		}
+	}
+	if !s.changed {
+		*s.value = out
+	} else {
+		for k, v := range out {
+			(*s.value)[k] = v
+		}
+	}
+	s.changed = true
+	return nil
+}
+
+func (s *stringToIntValue) Type() string {
+	return "stringToInt"
+}
+
+func (s *stringToIntValue) String() string {
+	var buf bytes.Buffer
+	i := 0
+	for k, v := range *s.value {
+		if i > 0 {
+			buf.WriteRune(',')
+		}
+		buf.WriteString(k)
+		buf.WriteRune('=')
+		buf.WriteString(strconv.Itoa(v))
+		i++
+	}
+	return "[" + buf.String() + "]"
+}
+
+func stringToIntConv(val string) (interface{}, error) {
+	val = strings.Trim(val, "[]")
+	// An empty string would cause an empty map
+	if len(val) == 0 {
+		return map[string]int{}, nil
+	}
+	ss := strings.Split(val, ",")
+	out := make(map[string]int, len(ss))
+	for _, pair := range ss {
+		kv := strings.SplitN(pair, "=", 2)
+		if len(kv) != 2 {
+			return nil, fmt.Errorf("%s must be formatted as key=value", pair)
+		}
+		var err error
+		out[kv[0]], err = strconv.Atoi(kv[1])
+		if err != nil {
+			return nil, err
+		}
+	}
+	return out, nil
+}
+
+// GetStringToInt return the map[string]int value of a flag with the given name
+func (f *FlagSet) GetStringToInt(name string) (map[string]int, error) {
+	val, err := f.getFlagType(name, "stringToInt", stringToIntConv)
+	if err != nil {
+		return map[string]int{}, err
+	}
+	return val.(map[string]int), nil
+}
+
+// StringToIntVar defines a string flag with specified name, default value, and usage string.
+// The argument p points to a map[string]int variable in which to store the values of the multiple flags.
+// The value of each argument will not try to be separated by comma
+func (f *FlagSet) StringToIntVar(p *map[string]int, name string, value map[string]int, usage string) {
+	f.VarP(newStringToIntValue(value, p), name, "", usage)
+}
+
+// StringToIntVarP is like StringToIntVar, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) StringToIntVarP(p *map[string]int, name, shorthand string, value map[string]int, usage string) {
+	f.VarP(newStringToIntValue(value, p), name, shorthand, usage)
+}
+
+// StringToIntVar defines a string flag with specified name, default value, and usage string.
+// The argument p points to a map[string]int variable in which to store the value of the flag.
+// The value of each argument will not try to be separated by comma
+func StringToIntVar(p *map[string]int, name string, value map[string]int, usage string) {
+	CommandLine.VarP(newStringToIntValue(value, p), name, "", usage)
+}
+
+// StringToIntVarP is like StringToIntVar, but accepts a shorthand letter that can be used after a single dash.
+func StringToIntVarP(p *map[string]int, name, shorthand string, value map[string]int, usage string) {
+	CommandLine.VarP(newStringToIntValue(value, p), name, shorthand, usage)
+}
+
+// StringToInt defines a string flag with specified name, default value, and usage string.
+// The return value is the address of a map[string]int variable that stores the value of the flag.
+// The value of each argument will not try to be separated by comma
+func (f *FlagSet) StringToInt(name string, value map[string]int, usage string) *map[string]int {
+	p := map[string]int{}
+	f.StringToIntVarP(&p, name, "", value, usage)
+	return &p
+}
+
+// StringToIntP is like StringToInt, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) StringToIntP(name, shorthand string, value map[string]int, usage string) *map[string]int {
+	p := map[string]int{}
+	f.StringToIntVarP(&p, name, shorthand, value, usage)
+	return &p
+}
+
+// StringToInt defines a string flag with specified name, default value, and usage string.
+// The return value is the address of a map[string]int variable that stores the value of the flag.
+// The value of each argument will not try to be separated by comma
+func StringToInt(name string, value map[string]int, usage string) *map[string]int {
+	return CommandLine.StringToIntP(name, "", value, usage)
+}
+
+// StringToIntP is like StringToInt, but accepts a shorthand letter that can be used after a single dash.
+func StringToIntP(name, shorthand string, value map[string]int, usage string) *map[string]int {
+	return CommandLine.StringToIntP(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/string_to_string.go b/vendor/github.com/spf13/pflag/string_to_string.go
new file mode 100644
index 0000000..890a01a
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/string_to_string.go
@@ -0,0 +1,160 @@
+package pflag
+
+import (
+	"bytes"
+	"encoding/csv"
+	"fmt"
+	"strings"
+)
+
+// -- stringToString Value
+type stringToStringValue struct {
+	value   *map[string]string
+	changed bool
+}
+
+func newStringToStringValue(val map[string]string, p *map[string]string) *stringToStringValue {
+	ssv := new(stringToStringValue)
+	ssv.value = p
+	*ssv.value = val
+	return ssv
+}
+
+// Format: a=1,b=2
+func (s *stringToStringValue) Set(val string) error {
+	var ss []string
+	n := strings.Count(val, "=")
+	switch n {
+	case 0:
+		return fmt.Errorf("%s must be formatted as key=value", val)
+	case 1:
+		ss = append(ss, strings.Trim(val, `"`))
+	default:
+		r := csv.NewReader(strings.NewReader(val))
+		var err error
+		ss, err = r.Read()
+		if err != nil {
+			return err
+		}
+	}
+
+	out := make(map[string]string, len(ss))
+	for _, pair := range ss {
+		kv := strings.SplitN(pair, "=", 2)
+		if len(kv) != 2 {
+			return fmt.Errorf("%s must be formatted as key=value", pair)
+		}
+		out[kv[0]] = kv[1]
+	}
+	if !s.changed {
+		*s.value = out
+	} else {
+		for k, v := range out {
+			(*s.value)[k] = v
+		}
+	}
+	s.changed = true
+	return nil
+}
+
+func (s *stringToStringValue) Type() string {
+	return "stringToString"
+}
+
+func (s *stringToStringValue) String() string {
+	records := make([]string, 0, len(*s.value)>>1)
+	for k, v := range *s.value {
+		records = append(records, k+"="+v)
+	}
+
+	var buf bytes.Buffer
+	w := csv.NewWriter(&buf)
+	if err := w.Write(records); err != nil {
+		panic(err)
+	}
+	w.Flush()
+	return "[" + strings.TrimSpace(buf.String()) + "]"
+}
+
+func stringToStringConv(val string) (interface{}, error) {
+	val = strings.Trim(val, "[]")
+	// An empty string would cause an empty map
+	if len(val) == 0 {
+		return map[string]string{}, nil
+	}
+	r := csv.NewReader(strings.NewReader(val))
+	ss, err := r.Read()
+	if err != nil {
+		return nil, err
+	}
+	out := make(map[string]string, len(ss))
+	for _, pair := range ss {
+		kv := strings.SplitN(pair, "=", 2)
+		if len(kv) != 2 {
+			return nil, fmt.Errorf("%s must be formatted as key=value", pair)
+		}
+		out[kv[0]] = kv[1]
+	}
+	return out, nil
+}
+
+// GetStringToString return the map[string]string value of a flag with the given name
+func (f *FlagSet) GetStringToString(name string) (map[string]string, error) {
+	val, err := f.getFlagType(name, "stringToString", stringToStringConv)
+	if err != nil {
+		return map[string]string{}, err
+	}
+	return val.(map[string]string), nil
+}
+
+// StringToStringVar defines a string flag with specified name, default value, and usage string.
+// The argument p points to a map[string]string variable in which to store the values of the multiple flags.
+// The value of each argument will not try to be separated by comma
+func (f *FlagSet) StringToStringVar(p *map[string]string, name string, value map[string]string, usage string) {
+	f.VarP(newStringToStringValue(value, p), name, "", usage)
+}
+
+// StringToStringVarP is like StringToStringVar, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) StringToStringVarP(p *map[string]string, name, shorthand string, value map[string]string, usage string) {
+	f.VarP(newStringToStringValue(value, p), name, shorthand, usage)
+}
+
+// StringToStringVar defines a string flag with specified name, default value, and usage string.
+// The argument p points to a map[string]string variable in which to store the value of the flag.
+// The value of each argument will not try to be separated by comma
+func StringToStringVar(p *map[string]string, name string, value map[string]string, usage string) {
+	CommandLine.VarP(newStringToStringValue(value, p), name, "", usage)
+}
+
+// StringToStringVarP is like StringToStringVar, but accepts a shorthand letter that can be used after a single dash.
+func StringToStringVarP(p *map[string]string, name, shorthand string, value map[string]string, usage string) {
+	CommandLine.VarP(newStringToStringValue(value, p), name, shorthand, usage)
+}
+
+// StringToString defines a string flag with specified name, default value, and usage string.
+// The return value is the address of a map[string]string variable that stores the value of the flag.
+// The value of each argument will not try to be separated by comma
+func (f *FlagSet) StringToString(name string, value map[string]string, usage string) *map[string]string {
+	p := map[string]string{}
+	f.StringToStringVarP(&p, name, "", value, usage)
+	return &p
+}
+
+// StringToStringP is like StringToString, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) StringToStringP(name, shorthand string, value map[string]string, usage string) *map[string]string {
+	p := map[string]string{}
+	f.StringToStringVarP(&p, name, shorthand, value, usage)
+	return &p
+}
+
+// StringToString defines a string flag with specified name, default value, and usage string.
+// The return value is the address of a map[string]string variable that stores the value of the flag.
+// The value of each argument will not try to be separated by comma
+func StringToString(name string, value map[string]string, usage string) *map[string]string {
+	return CommandLine.StringToStringP(name, "", value, usage)
+}
+
+// StringToStringP is like StringToString, but accepts a shorthand letter that can be used after a single dash.
+func StringToStringP(name, shorthand string, value map[string]string, usage string) *map[string]string {
+	return CommandLine.StringToStringP(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/uint.go b/vendor/github.com/spf13/pflag/uint.go
new file mode 100644
index 0000000..dcbc2b7
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/uint.go
@@ -0,0 +1,88 @@
+package pflag
+
+import "strconv"
+
+// -- uint Value
+type uintValue uint
+
+func newUintValue(val uint, p *uint) *uintValue {
+	*p = val
+	return (*uintValue)(p)
+}
+
+func (i *uintValue) Set(s string) error {
+	v, err := strconv.ParseUint(s, 0, 64)
+	*i = uintValue(v)
+	return err
+}
+
+func (i *uintValue) Type() string {
+	return "uint"
+}
+
+func (i *uintValue) String() string { return strconv.FormatUint(uint64(*i), 10) }
+
+func uintConv(sval string) (interface{}, error) {
+	v, err := strconv.ParseUint(sval, 0, 0)
+	if err != nil {
+		return 0, err
+	}
+	return uint(v), nil
+}
+
+// GetUint return the uint value of a flag with the given name
+func (f *FlagSet) GetUint(name string) (uint, error) {
+	val, err := f.getFlagType(name, "uint", uintConv)
+	if err != nil {
+		return 0, err
+	}
+	return val.(uint), nil
+}
+
+// UintVar defines a uint flag with specified name, default value, and usage string.
+// The argument p points to a uint variable in which to store the value of the flag.
+func (f *FlagSet) UintVar(p *uint, name string, value uint, usage string) {
+	f.VarP(newUintValue(value, p), name, "", usage)
+}
+
+// UintVarP is like UintVar, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) UintVarP(p *uint, name, shorthand string, value uint, usage string) {
+	f.VarP(newUintValue(value, p), name, shorthand, usage)
+}
+
+// UintVar defines a uint flag with specified name, default value, and usage string.
+// The argument p points to a uint  variable in which to store the value of the flag.
+func UintVar(p *uint, name string, value uint, usage string) {
+	CommandLine.VarP(newUintValue(value, p), name, "", usage)
+}
+
+// UintVarP is like UintVar, but accepts a shorthand letter that can be used after a single dash.
+func UintVarP(p *uint, name, shorthand string, value uint, usage string) {
+	CommandLine.VarP(newUintValue(value, p), name, shorthand, usage)
+}
+
+// Uint defines a uint flag with specified name, default value, and usage string.
+// The return value is the address of a uint  variable that stores the value of the flag.
+func (f *FlagSet) Uint(name string, value uint, usage string) *uint {
+	p := new(uint)
+	f.UintVarP(p, name, "", value, usage)
+	return p
+}
+
+// UintP is like Uint, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) UintP(name, shorthand string, value uint, usage string) *uint {
+	p := new(uint)
+	f.UintVarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// Uint defines a uint flag with specified name, default value, and usage string.
+// The return value is the address of a uint  variable that stores the value of the flag.
+func Uint(name string, value uint, usage string) *uint {
+	return CommandLine.UintP(name, "", value, usage)
+}
+
+// UintP is like Uint, but accepts a shorthand letter that can be used after a single dash.
+func UintP(name, shorthand string, value uint, usage string) *uint {
+	return CommandLine.UintP(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/uint16.go b/vendor/github.com/spf13/pflag/uint16.go
new file mode 100644
index 0000000..7e9914e
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/uint16.go
@@ -0,0 +1,88 @@
+package pflag
+
+import "strconv"
+
+// -- uint16 value
+type uint16Value uint16
+
+func newUint16Value(val uint16, p *uint16) *uint16Value {
+	*p = val
+	return (*uint16Value)(p)
+}
+
+func (i *uint16Value) Set(s string) error {
+	v, err := strconv.ParseUint(s, 0, 16)
+	*i = uint16Value(v)
+	return err
+}
+
+func (i *uint16Value) Type() string {
+	return "uint16"
+}
+
+func (i *uint16Value) String() string { return strconv.FormatUint(uint64(*i), 10) }
+
+func uint16Conv(sval string) (interface{}, error) {
+	v, err := strconv.ParseUint(sval, 0, 16)
+	if err != nil {
+		return 0, err
+	}
+	return uint16(v), nil
+}
+
+// GetUint16 return the uint16 value of a flag with the given name
+func (f *FlagSet) GetUint16(name string) (uint16, error) {
+	val, err := f.getFlagType(name, "uint16", uint16Conv)
+	if err != nil {
+		return 0, err
+	}
+	return val.(uint16), nil
+}
+
+// Uint16Var defines a uint flag with specified name, default value, and usage string.
+// The argument p points to a uint variable in which to store the value of the flag.
+func (f *FlagSet) Uint16Var(p *uint16, name string, value uint16, usage string) {
+	f.VarP(newUint16Value(value, p), name, "", usage)
+}
+
+// Uint16VarP is like Uint16Var, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Uint16VarP(p *uint16, name, shorthand string, value uint16, usage string) {
+	f.VarP(newUint16Value(value, p), name, shorthand, usage)
+}
+
+// Uint16Var defines a uint flag with specified name, default value, and usage string.
+// The argument p points to a uint  variable in which to store the value of the flag.
+func Uint16Var(p *uint16, name string, value uint16, usage string) {
+	CommandLine.VarP(newUint16Value(value, p), name, "", usage)
+}
+
+// Uint16VarP is like Uint16Var, but accepts a shorthand letter that can be used after a single dash.
+func Uint16VarP(p *uint16, name, shorthand string, value uint16, usage string) {
+	CommandLine.VarP(newUint16Value(value, p), name, shorthand, usage)
+}
+
+// Uint16 defines a uint flag with specified name, default value, and usage string.
+// The return value is the address of a uint  variable that stores the value of the flag.
+func (f *FlagSet) Uint16(name string, value uint16, usage string) *uint16 {
+	p := new(uint16)
+	f.Uint16VarP(p, name, "", value, usage)
+	return p
+}
+
+// Uint16P is like Uint16, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Uint16P(name, shorthand string, value uint16, usage string) *uint16 {
+	p := new(uint16)
+	f.Uint16VarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// Uint16 defines a uint flag with specified name, default value, and usage string.
+// The return value is the address of a uint  variable that stores the value of the flag.
+func Uint16(name string, value uint16, usage string) *uint16 {
+	return CommandLine.Uint16P(name, "", value, usage)
+}
+
+// Uint16P is like Uint16, but accepts a shorthand letter that can be used after a single dash.
+func Uint16P(name, shorthand string, value uint16, usage string) *uint16 {
+	return CommandLine.Uint16P(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/uint32.go b/vendor/github.com/spf13/pflag/uint32.go
new file mode 100644
index 0000000..d802453
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/uint32.go
@@ -0,0 +1,88 @@
+package pflag
+
+import "strconv"
+
+// -- uint32 value
+type uint32Value uint32
+
+func newUint32Value(val uint32, p *uint32) *uint32Value {
+	*p = val
+	return (*uint32Value)(p)
+}
+
+func (i *uint32Value) Set(s string) error {
+	v, err := strconv.ParseUint(s, 0, 32)
+	*i = uint32Value(v)
+	return err
+}
+
+func (i *uint32Value) Type() string {
+	return "uint32"
+}
+
+func (i *uint32Value) String() string { return strconv.FormatUint(uint64(*i), 10) }
+
+func uint32Conv(sval string) (interface{}, error) {
+	v, err := strconv.ParseUint(sval, 0, 32)
+	if err != nil {
+		return 0, err
+	}
+	return uint32(v), nil
+}
+
+// GetUint32 return the uint32 value of a flag with the given name
+func (f *FlagSet) GetUint32(name string) (uint32, error) {
+	val, err := f.getFlagType(name, "uint32", uint32Conv)
+	if err != nil {
+		return 0, err
+	}
+	return val.(uint32), nil
+}
+
+// Uint32Var defines a uint32 flag with specified name, default value, and usage string.
+// The argument p points to a uint32 variable in which to store the value of the flag.
+func (f *FlagSet) Uint32Var(p *uint32, name string, value uint32, usage string) {
+	f.VarP(newUint32Value(value, p), name, "", usage)
+}
+
+// Uint32VarP is like Uint32Var, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Uint32VarP(p *uint32, name, shorthand string, value uint32, usage string) {
+	f.VarP(newUint32Value(value, p), name, shorthand, usage)
+}
+
+// Uint32Var defines a uint32 flag with specified name, default value, and usage string.
+// The argument p points to a uint32  variable in which to store the value of the flag.
+func Uint32Var(p *uint32, name string, value uint32, usage string) {
+	CommandLine.VarP(newUint32Value(value, p), name, "", usage)
+}
+
+// Uint32VarP is like Uint32Var, but accepts a shorthand letter that can be used after a single dash.
+func Uint32VarP(p *uint32, name, shorthand string, value uint32, usage string) {
+	CommandLine.VarP(newUint32Value(value, p), name, shorthand, usage)
+}
+
+// Uint32 defines a uint32 flag with specified name, default value, and usage string.
+// The return value is the address of a uint32  variable that stores the value of the flag.
+func (f *FlagSet) Uint32(name string, value uint32, usage string) *uint32 {
+	p := new(uint32)
+	f.Uint32VarP(p, name, "", value, usage)
+	return p
+}
+
+// Uint32P is like Uint32, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Uint32P(name, shorthand string, value uint32, usage string) *uint32 {
+	p := new(uint32)
+	f.Uint32VarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// Uint32 defines a uint32 flag with specified name, default value, and usage string.
+// The return value is the address of a uint32  variable that stores the value of the flag.
+func Uint32(name string, value uint32, usage string) *uint32 {
+	return CommandLine.Uint32P(name, "", value, usage)
+}
+
+// Uint32P is like Uint32, but accepts a shorthand letter that can be used after a single dash.
+func Uint32P(name, shorthand string, value uint32, usage string) *uint32 {
+	return CommandLine.Uint32P(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/uint64.go b/vendor/github.com/spf13/pflag/uint64.go
new file mode 100644
index 0000000..f62240f
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/uint64.go
@@ -0,0 +1,88 @@
+package pflag
+
+import "strconv"
+
+// -- uint64 Value
+type uint64Value uint64
+
+func newUint64Value(val uint64, p *uint64) *uint64Value {
+	*p = val
+	return (*uint64Value)(p)
+}
+
+func (i *uint64Value) Set(s string) error {
+	v, err := strconv.ParseUint(s, 0, 64)
+	*i = uint64Value(v)
+	return err
+}
+
+func (i *uint64Value) Type() string {
+	return "uint64"
+}
+
+func (i *uint64Value) String() string { return strconv.FormatUint(uint64(*i), 10) }
+
+func uint64Conv(sval string) (interface{}, error) {
+	v, err := strconv.ParseUint(sval, 0, 64)
+	if err != nil {
+		return 0, err
+	}
+	return uint64(v), nil
+}
+
+// GetUint64 return the uint64 value of a flag with the given name
+func (f *FlagSet) GetUint64(name string) (uint64, error) {
+	val, err := f.getFlagType(name, "uint64", uint64Conv)
+	if err != nil {
+		return 0, err
+	}
+	return val.(uint64), nil
+}
+
+// Uint64Var defines a uint64 flag with specified name, default value, and usage string.
+// The argument p points to a uint64 variable in which to store the value of the flag.
+func (f *FlagSet) Uint64Var(p *uint64, name string, value uint64, usage string) {
+	f.VarP(newUint64Value(value, p), name, "", usage)
+}
+
+// Uint64VarP is like Uint64Var, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Uint64VarP(p *uint64, name, shorthand string, value uint64, usage string) {
+	f.VarP(newUint64Value(value, p), name, shorthand, usage)
+}
+
+// Uint64Var defines a uint64 flag with specified name, default value, and usage string.
+// The argument p points to a uint64 variable in which to store the value of the flag.
+func Uint64Var(p *uint64, name string, value uint64, usage string) {
+	CommandLine.VarP(newUint64Value(value, p), name, "", usage)
+}
+
+// Uint64VarP is like Uint64Var, but accepts a shorthand letter that can be used after a single dash.
+func Uint64VarP(p *uint64, name, shorthand string, value uint64, usage string) {
+	CommandLine.VarP(newUint64Value(value, p), name, shorthand, usage)
+}
+
+// Uint64 defines a uint64 flag with specified name, default value, and usage string.
+// The return value is the address of a uint64 variable that stores the value of the flag.
+func (f *FlagSet) Uint64(name string, value uint64, usage string) *uint64 {
+	p := new(uint64)
+	f.Uint64VarP(p, name, "", value, usage)
+	return p
+}
+
+// Uint64P is like Uint64, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Uint64P(name, shorthand string, value uint64, usage string) *uint64 {
+	p := new(uint64)
+	f.Uint64VarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// Uint64 defines a uint64 flag with specified name, default value, and usage string.
+// The return value is the address of a uint64 variable that stores the value of the flag.
+func Uint64(name string, value uint64, usage string) *uint64 {
+	return CommandLine.Uint64P(name, "", value, usage)
+}
+
+// Uint64P is like Uint64, but accepts a shorthand letter that can be used after a single dash.
+func Uint64P(name, shorthand string, value uint64, usage string) *uint64 {
+	return CommandLine.Uint64P(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/uint8.go b/vendor/github.com/spf13/pflag/uint8.go
new file mode 100644
index 0000000..bb0e83c
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/uint8.go
@@ -0,0 +1,88 @@
+package pflag
+
+import "strconv"
+
+// -- uint8 Value
+type uint8Value uint8
+
+func newUint8Value(val uint8, p *uint8) *uint8Value {
+	*p = val
+	return (*uint8Value)(p)
+}
+
+func (i *uint8Value) Set(s string) error {
+	v, err := strconv.ParseUint(s, 0, 8)
+	*i = uint8Value(v)
+	return err
+}
+
+func (i *uint8Value) Type() string {
+	return "uint8"
+}
+
+func (i *uint8Value) String() string { return strconv.FormatUint(uint64(*i), 10) }
+
+func uint8Conv(sval string) (interface{}, error) {
+	v, err := strconv.ParseUint(sval, 0, 8)
+	if err != nil {
+		return 0, err
+	}
+	return uint8(v), nil
+}
+
+// GetUint8 return the uint8 value of a flag with the given name
+func (f *FlagSet) GetUint8(name string) (uint8, error) {
+	val, err := f.getFlagType(name, "uint8", uint8Conv)
+	if err != nil {
+		return 0, err
+	}
+	return val.(uint8), nil
+}
+
+// Uint8Var defines a uint8 flag with specified name, default value, and usage string.
+// The argument p points to a uint8 variable in which to store the value of the flag.
+func (f *FlagSet) Uint8Var(p *uint8, name string, value uint8, usage string) {
+	f.VarP(newUint8Value(value, p), name, "", usage)
+}
+
+// Uint8VarP is like Uint8Var, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Uint8VarP(p *uint8, name, shorthand string, value uint8, usage string) {
+	f.VarP(newUint8Value(value, p), name, shorthand, usage)
+}
+
+// Uint8Var defines a uint8 flag with specified name, default value, and usage string.
+// The argument p points to a uint8 variable in which to store the value of the flag.
+func Uint8Var(p *uint8, name string, value uint8, usage string) {
+	CommandLine.VarP(newUint8Value(value, p), name, "", usage)
+}
+
+// Uint8VarP is like Uint8Var, but accepts a shorthand letter that can be used after a single dash.
+func Uint8VarP(p *uint8, name, shorthand string, value uint8, usage string) {
+	CommandLine.VarP(newUint8Value(value, p), name, shorthand, usage)
+}
+
+// Uint8 defines a uint8 flag with specified name, default value, and usage string.
+// The return value is the address of a uint8 variable that stores the value of the flag.
+func (f *FlagSet) Uint8(name string, value uint8, usage string) *uint8 {
+	p := new(uint8)
+	f.Uint8VarP(p, name, "", value, usage)
+	return p
+}
+
+// Uint8P is like Uint8, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) Uint8P(name, shorthand string, value uint8, usage string) *uint8 {
+	p := new(uint8)
+	f.Uint8VarP(p, name, shorthand, value, usage)
+	return p
+}
+
+// Uint8 defines a uint8 flag with specified name, default value, and usage string.
+// The return value is the address of a uint8 variable that stores the value of the flag.
+func Uint8(name string, value uint8, usage string) *uint8 {
+	return CommandLine.Uint8P(name, "", value, usage)
+}
+
+// Uint8P is like Uint8, but accepts a shorthand letter that can be used after a single dash.
+func Uint8P(name, shorthand string, value uint8, usage string) *uint8 {
+	return CommandLine.Uint8P(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/spf13/pflag/uint_slice.go b/vendor/github.com/spf13/pflag/uint_slice.go
new file mode 100644
index 0000000..edd94c6
--- /dev/null
+++ b/vendor/github.com/spf13/pflag/uint_slice.go
@@ -0,0 +1,126 @@
+package pflag
+
+import (
+	"fmt"
+	"strconv"
+	"strings"
+)
+
+// -- uintSlice Value
+type uintSliceValue struct {
+	value   *[]uint
+	changed bool
+}
+
+func newUintSliceValue(val []uint, p *[]uint) *uintSliceValue {
+	uisv := new(uintSliceValue)
+	uisv.value = p
+	*uisv.value = val
+	return uisv
+}
+
+func (s *uintSliceValue) Set(val string) error {
+	ss := strings.Split(val, ",")
+	out := make([]uint, len(ss))
+	for i, d := range ss {
+		u, err := strconv.ParseUint(d, 10, 0)
+		if err != nil {
+			return err
+		}
+		out[i] = uint(u)
+	}
+	if !s.changed {
+		*s.value = out
+	} else {
+		*s.value = append(*s.value, out...)
+	}
+	s.changed = true
+	return nil
+}
+
+func (s *uintSliceValue) Type() string {
+	return "uintSlice"
+}
+
+func (s *uintSliceValue) String() string {
+	out := make([]string, len(*s.value))
+	for i, d := range *s.value {
+		out[i] = fmt.Sprintf("%d", d)
+	}
+	return "[" + strings.Join(out, ",") + "]"
+}
+
+func uintSliceConv(val string) (interface{}, error) {
+	val = strings.Trim(val, "[]")
+	// Empty string would cause a slice with one (empty) entry
+	if len(val) == 0 {
+		return []uint{}, nil
+	}
+	ss := strings.Split(val, ",")
+	out := make([]uint, len(ss))
+	for i, d := range ss {
+		u, err := strconv.ParseUint(d, 10, 0)
+		if err != nil {
+			return nil, err
+		}
+		out[i] = uint(u)
+	}
+	return out, nil
+}
+
+// GetUintSlice returns the []uint value of a flag with the given name.
+func (f *FlagSet) GetUintSlice(name string) ([]uint, error) {
+	val, err := f.getFlagType(name, "uintSlice", uintSliceConv)
+	if err != nil {
+		return []uint{}, err
+	}
+	return val.([]uint), nil
+}
+
+// UintSliceVar defines a uintSlice flag with specified name, default value, and usage string.
+// The argument p points to a []uint variable in which to store the value of the flag.
+func (f *FlagSet) UintSliceVar(p *[]uint, name string, value []uint, usage string) {
+	f.VarP(newUintSliceValue(value, p), name, "", usage)
+}
+
+// UintSliceVarP is like UintSliceVar, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) UintSliceVarP(p *[]uint, name, shorthand string, value []uint, usage string) {
+	f.VarP(newUintSliceValue(value, p), name, shorthand, usage)
+}
+
+// UintSliceVar defines a uint[] flag with specified name, default value, and usage string.
+// The argument p points to a uint[] variable in which to store the value of the flag.
+func UintSliceVar(p *[]uint, name string, value []uint, usage string) {
+	CommandLine.VarP(newUintSliceValue(value, p), name, "", usage)
+}
+
+// UintSliceVarP is like the UintSliceVar, but accepts a shorthand letter that can be used after a single dash.
+func UintSliceVarP(p *[]uint, name, shorthand string, value []uint, usage string) {
+	CommandLine.VarP(newUintSliceValue(value, p), name, shorthand, usage)
+}
+
+// UintSlice defines a []uint flag with specified name, default value, and usage string.
+// The return value is the address of a []uint variable that stores the value of the flag.
+func (f *FlagSet) UintSlice(name string, value []uint, usage string) *[]uint {
+	p := []uint{}
+	f.UintSliceVarP(&p, name, "", value, usage)
+	return &p
+}
+
+// UintSliceP is like UintSlice, but accepts a shorthand letter that can be used after a single dash.
+func (f *FlagSet) UintSliceP(name, shorthand string, value []uint, usage string) *[]uint {
+	p := []uint{}
+	f.UintSliceVarP(&p, name, shorthand, value, usage)
+	return &p
+}
+
+// UintSlice defines a []uint flag with specified name, default value, and usage string.
+// The return value is the address of a []uint variable that stores the value of the flag.
+func UintSlice(name string, value []uint, usage string) *[]uint {
+	return CommandLine.UintSliceP(name, "", value, usage)
+}
+
+// UintSliceP is like UintSlice, but accepts a shorthand letter that can be used after a single dash.
+func UintSliceP(name, shorthand string, value []uint, usage string) *[]uint {
+	return CommandLine.UintSliceP(name, shorthand, value, usage)
+}
diff --git a/vendor/github.com/tmc/grpc-websocket-proxy/LICENSE b/vendor/github.com/tmc/grpc-websocket-proxy/LICENSE
new file mode 100644
index 0000000..95d0bc8
--- /dev/null
+++ b/vendor/github.com/tmc/grpc-websocket-proxy/LICENSE
@@ -0,0 +1,7 @@
+Copyright (C) 2016 Travis Cline
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/vendor/github.com/tmc/grpc-websocket-proxy/wsproxy/doc.go b/vendor/github.com/tmc/grpc-websocket-proxy/wsproxy/doc.go
new file mode 100644
index 0000000..baf9545
--- /dev/null
+++ b/vendor/github.com/tmc/grpc-websocket-proxy/wsproxy/doc.go
@@ -0,0 +1,2 @@
+// Package wsproxy implements a websocket proxy for grpc-gateway backed services
+package wsproxy
diff --git a/vendor/github.com/tmc/grpc-websocket-proxy/wsproxy/websocket_proxy.go b/vendor/github.com/tmc/grpc-websocket-proxy/wsproxy/websocket_proxy.go
new file mode 100644
index 0000000..0fca05a
--- /dev/null
+++ b/vendor/github.com/tmc/grpc-websocket-proxy/wsproxy/websocket_proxy.go
@@ -0,0 +1,254 @@
+package wsproxy
+
+import (
+	"bufio"
+	"io"
+	"net/http"
+	"strings"
+
+	"github.com/gorilla/websocket"
+	"github.com/sirupsen/logrus"
+	"golang.org/x/net/context"
+)
+
+// MethodOverrideParam defines the special URL parameter that is translated into the subsequent proxied streaming http request's method.
+//
+// Deprecated: it is preferable to use the Options parameters to WebSocketProxy to supply parameters.
+var MethodOverrideParam = "method"
+
+// TokenCookieName defines the cookie name that is translated to an 'Authorization: Bearer' header in the streaming http request's headers.
+//
+// Deprecated: it is preferable to use the Options parameters to WebSocketProxy to supply parameters.
+var TokenCookieName = "token"
+
+// RequestMutatorFunc can supply an alternate outgoing request.
+type RequestMutatorFunc func(incoming *http.Request, outgoing *http.Request) *http.Request
+
+// Proxy provides websocket transport upgrade to compatible endpoints.
+type Proxy struct {
+	h                   http.Handler
+	logger              Logger
+	methodOverrideParam string
+	tokenCookieName     string
+	requestMutator      RequestMutatorFunc
+}
+
+// Logger collects log messages.
+type Logger interface {
+	Warnln(...interface{})
+	Debugln(...interface{})
+}
+
+func (p *Proxy) ServeHTTP(w http.ResponseWriter, r *http.Request) {
+	if !websocket.IsWebSocketUpgrade(r) {
+		p.h.ServeHTTP(w, r)
+		return
+	}
+	p.proxy(w, r)
+}
+
+// Option allows customization of the proxy.
+type Option func(*Proxy)
+
+// WithMethodParamOverride allows specification of the special http parameter that is used in the proxied streaming request.
+func WithMethodParamOverride(param string) Option {
+	return func(p *Proxy) {
+		p.methodOverrideParam = param
+	}
+}
+
+// WithTokenCookieName allows specification of the cookie that is supplied as an upstream 'Authorization: Bearer' http header.
+func WithTokenCookieName(param string) Option {
+	return func(p *Proxy) {
+		p.tokenCookieName = param
+	}
+}
+
+// WithRequestMutator allows a custom RequestMutatorFunc to be supplied.
+func WithRequestMutator(fn RequestMutatorFunc) Option {
+	return func(p *Proxy) {
+		p.requestMutator = fn
+	}
+}
+
+// WithLogger allows a custom FieldLogger to be supplied
+func WithLogger(logger Logger) Option {
+	return func(p *Proxy) {
+		p.logger = logger
+	}
+}
+
+// WebsocketProxy attempts to expose the underlying handler as a bidi websocket stream with newline-delimited
+// JSON as the content encoding.
+//
+// The HTTP Authorization header is either populated from the Sec-Websocket-Protocol field or by a cookie.
+// The cookie name is specified by the TokenCookieName value.
+//
+// example:
+//   Sec-Websocket-Protocol: Bearer, foobar
+// is converted to:
+//   Authorization: Bearer foobar
+//
+// Method can be overwritten with the MethodOverrideParam get parameter in the requested URL
+func WebsocketProxy(h http.Handler, opts ...Option) http.Handler {
+	p := &Proxy{
+		h:                   h,
+		logger:              logrus.New(),
+		methodOverrideParam: MethodOverrideParam,
+		tokenCookieName:     TokenCookieName,
+	}
+	for _, o := range opts {
+		o(p)
+	}
+	return p
+}
+
+// TODO(tmc): allow modification of upgrader settings?
+var upgrader = websocket.Upgrader{
+	ReadBufferSize:  1024,
+	WriteBufferSize: 1024,
+	CheckOrigin:     func(r *http.Request) bool { return true },
+}
+
+func isClosedConnError(err error) bool {
+	str := err.Error()
+	if strings.Contains(str, "use of closed network connection") {
+		return true
+	}
+	return websocket.IsCloseError(err, websocket.CloseNormalClosure, websocket.CloseGoingAway)
+}
+
+func (p *Proxy) proxy(w http.ResponseWriter, r *http.Request) {
+	var responseHeader http.Header
+	// If Sec-WebSocket-Protocol starts with "Bearer", respond in kind.
+	// TODO(tmc): consider customizability/extension point here.
+	if strings.HasPrefix(r.Header.Get("Sec-WebSocket-Protocol"), "Bearer") {
+		responseHeader = http.Header{
+			"Sec-WebSocket-Protocol": []string{"Bearer"},
+		}
+	}
+	conn, err := upgrader.Upgrade(w, r, responseHeader)
+	if err != nil {
+		p.logger.Warnln("error upgrading websocket:", err)
+		return
+	}
+	defer conn.Close()
+
+	ctx, cancelFn := context.WithCancel(context.Background())
+	defer cancelFn()
+
+	requestBodyR, requestBodyW := io.Pipe()
+	request, err := http.NewRequest(r.Method, r.URL.String(), requestBodyR)
+	if err != nil {
+		p.logger.Warnln("error preparing request:", err)
+		return
+	}
+	if swsp := r.Header.Get("Sec-WebSocket-Protocol"); swsp != "" {
+		request.Header.Set("Authorization", strings.Replace(swsp, "Bearer, ", "Bearer ", 1))
+	}
+	// If token cookie is present, populate Authorization header from the cookie instead.
+	if cookie, err := r.Cookie(p.tokenCookieName); err == nil {
+		request.Header.Set("Authorization", "Bearer "+cookie.Value)
+	}
+	if m := r.URL.Query().Get(p.methodOverrideParam); m != "" {
+		request.Method = m
+	}
+
+	if p.requestMutator != nil {
+		request = p.requestMutator(r, request)
+	}
+
+	responseBodyR, responseBodyW := io.Pipe()
+	response := newInMemoryResponseWriter(responseBodyW)
+	go func() {
+		<-ctx.Done()
+		p.logger.Debugln("closing pipes")
+		requestBodyW.CloseWithError(io.EOF)
+		responseBodyW.CloseWithError(io.EOF)
+		response.closed <- true
+	}()
+
+	go func() {
+		defer cancelFn()
+		p.h.ServeHTTP(response, request)
+	}()
+
+	// read loop -- take messages from websocket and write to http request
+	go func() {
+		defer func() {
+			cancelFn()
+		}()
+		for {
+			select {
+			case <-ctx.Done():
+				p.logger.Debugln("read loop done")
+				return
+			default:
+			}
+			p.logger.Debugln("[read] reading from socket.")
+			_, payload, err := conn.ReadMessage()
+			if err != nil {
+				if isClosedConnError(err) {
+					p.logger.Debugln("[read] websocket closed:", err)
+					return
+				}
+				p.logger.Warnln("error reading websocket message:", err)
+				return
+			}
+			p.logger.Debugln("[read] read payload:", string(payload))
+			p.logger.Debugln("[read] writing to requestBody:")
+			n, err := requestBodyW.Write(payload)
+			requestBodyW.Write([]byte("\n"))
+			p.logger.Debugln("[read] wrote to requestBody", n)
+			if err != nil {
+				p.logger.Warnln("[read] error writing message to upstream http server:", err)
+				return
+			}
+		}
+	}()
+	// write loop -- take messages from response and write to websocket
+	scanner := bufio.NewScanner(responseBodyR)
+	for scanner.Scan() {
+		if len(scanner.Bytes()) == 0 {
+			p.logger.Warnln("[write] empty scan", scanner.Err())
+			continue
+		}
+		p.logger.Debugln("[write] scanned", scanner.Text())
+		if err = conn.WriteMessage(websocket.TextMessage, scanner.Bytes()); err != nil {
+			p.logger.Warnln("[write] error writing websocket message:", err)
+			return
+		}
+	}
+	if err := scanner.Err(); err != nil {
+		p.logger.Warnln("scanner err:", err)
+	}
+}
+
+type inMemoryResponseWriter struct {
+	io.Writer
+	header http.Header
+	code   int
+	closed chan bool
+}
+
+func newInMemoryResponseWriter(w io.Writer) *inMemoryResponseWriter {
+	return &inMemoryResponseWriter{
+		Writer: w,
+		header: http.Header{},
+		closed: make(chan bool, 1),
+	}
+}
+
+func (w *inMemoryResponseWriter) Write(b []byte) (int, error) {
+	return w.Writer.Write(b)
+}
+func (w *inMemoryResponseWriter) Header() http.Header {
+	return w.header
+}
+func (w *inMemoryResponseWriter) WriteHeader(code int) {
+	w.code = code
+}
+func (w *inMemoryResponseWriter) CloseNotify() <-chan bool {
+	return w.closed
+}
+func (w *inMemoryResponseWriter) Flush() {}
diff --git a/vendor/github.com/xiang90/probing/.gitignore b/vendor/github.com/xiang90/probing/.gitignore
new file mode 100644
index 0000000..daf913b
--- /dev/null
+++ b/vendor/github.com/xiang90/probing/.gitignore
@@ -0,0 +1,24 @@
+# Compiled Object files, Static and Dynamic libs (Shared Objects)
+*.o
+*.a
+*.so
+
+# Folders
+_obj
+_test
+
+# Architecture specific extensions/prefixes
+*.[568vq]
+[568vq].out
+
+*.cgo1.go
+*.cgo2.c
+_cgo_defun.c
+_cgo_gotypes.go
+_cgo_export.*
+
+_testmain.go
+
+*.exe
+*.test
+*.prof
diff --git a/vendor/github.com/xiang90/probing/LICENSE b/vendor/github.com/xiang90/probing/LICENSE
new file mode 100644
index 0000000..cde8b8b
--- /dev/null
+++ b/vendor/github.com/xiang90/probing/LICENSE
@@ -0,0 +1,22 @@
+The MIT License (MIT)
+
+Copyright (c) 2015 Xiang Li
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
diff --git a/vendor/github.com/xiang90/probing/README.md b/vendor/github.com/xiang90/probing/README.md
new file mode 100644
index 0000000..2ff6820
--- /dev/null
+++ b/vendor/github.com/xiang90/probing/README.md
@@ -0,0 +1,39 @@
+## Getting Started
+
+### Install the handler
+
+We first need to serve the probing HTTP handler.
+
+```go
+    http.HandleFunc("/health", probing.NewHandler())
+    err := http.ListenAndServe(":12345", nil)
+	if err != nil {
+		log.Fatal("ListenAndServe: ", err)
+	}
+```
+
+### Start to probe
+
+Now we can start to probe the endpoint.
+
+``` go
+    id := "example"
+    probingInterval = 5 * time.Second
+    url := "http://example.com:12345/health"
+    p.AddHTTP(id, probingInterval, url)
+
+	time.Sleep(13 * time.Second)
+	status, err := p.Status(id)
+ 	fmt.Printf("Total Probing: %d, Total Loss: %d, Estimated RTT: %v, Estimated Clock Difference: %v\n",
+		status.Total(), status.Loss(), status.SRTT(), status.ClockDiff())
+	// Total Probing: 2, Total Loss: 0, Estimated RTT: 320.771µs, Estimated Clock Difference: -35.869µs
+```
+
+### TODOs:
+
+- TCP probing
+- UDP probing
+- Gossip based probing
+- More accurate RTT estimation
+- More accurate Clock difference estimation
+- Use a clock interface rather than the real clock
diff --git a/vendor/github.com/xiang90/probing/prober.go b/vendor/github.com/xiang90/probing/prober.go
new file mode 100644
index 0000000..9431c10
--- /dev/null
+++ b/vendor/github.com/xiang90/probing/prober.go
@@ -0,0 +1,139 @@
+package probing
+
+import (
+	"encoding/json"
+	"errors"
+	"fmt"
+	"net/http"
+	"sync"
+	"time"
+)
+
+var (
+	ErrNotFound = errors.New("probing: id not found")
+	ErrExist    = errors.New("probing: id exists")
+)
+
+type Prober interface {
+	AddHTTP(id string, probingInterval time.Duration, endpoints []string) error
+	Remove(id string) error
+	RemoveAll()
+	Reset(id string) error
+	Status(id string) (Status, error)
+}
+
+type prober struct {
+	mu      sync.Mutex
+	targets map[string]*status
+	tr      http.RoundTripper
+}
+
+func NewProber(tr http.RoundTripper) Prober {
+	p := &prober{targets: make(map[string]*status)}
+	if tr == nil {
+		p.tr = http.DefaultTransport
+	} else {
+		p.tr = tr
+	}
+	return p
+}
+
+func (p *prober) AddHTTP(id string, probingInterval time.Duration, endpoints []string) error {
+	p.mu.Lock()
+	defer p.mu.Unlock()
+	if _, ok := p.targets[id]; ok {
+		return ErrExist
+	}
+
+	s := &status{stopC: make(chan struct{})}
+	p.targets[id] = s
+
+	ticker := time.NewTicker(probingInterval)
+
+	go func() {
+		pinned := 0
+		for {
+			select {
+			case <-ticker.C:
+				start := time.Now()
+				req, err := http.NewRequest("GET", endpoints[pinned], nil)
+				if err != nil {
+					panic(err)
+				}
+				resp, err := p.tr.RoundTrip(req)
+				if err == nil && resp.StatusCode != http.StatusOK {
+					err = fmt.Errorf("got unexpected HTTP status code %s from %s", resp.Status, endpoints[pinned])
+					resp.Body.Close()
+				}
+				if err != nil {
+					s.recordFailure(err)
+					pinned = (pinned + 1) % len(endpoints)
+					continue
+				}
+
+				var hh Health
+				d := json.NewDecoder(resp.Body)
+				err = d.Decode(&hh)
+				resp.Body.Close()
+				if err != nil || !hh.OK {
+					s.recordFailure(err)
+					pinned = (pinned + 1) % len(endpoints)
+					continue
+				}
+
+				s.record(time.Since(start), hh.Now)
+			case <-s.stopC:
+				ticker.Stop()
+				return
+			}
+		}
+	}()
+
+	return nil
+}
+
+func (p *prober) Remove(id string) error {
+	p.mu.Lock()
+	defer p.mu.Unlock()
+
+	s, ok := p.targets[id]
+	if !ok {
+		return ErrNotFound
+	}
+	close(s.stopC)
+	delete(p.targets, id)
+	return nil
+}
+
+func (p *prober) RemoveAll() {
+	p.mu.Lock()
+	defer p.mu.Unlock()
+
+	for _, s := range p.targets {
+		close(s.stopC)
+	}
+	p.targets = make(map[string]*status)
+}
+
+func (p *prober) Reset(id string) error {
+	p.mu.Lock()
+	defer p.mu.Unlock()
+
+	s, ok := p.targets[id]
+	if !ok {
+		return ErrNotFound
+	}
+	s.reset()
+	return nil
+}
+
+func (p *prober) Status(id string) (Status, error) {
+	p.mu.Lock()
+	defer p.mu.Unlock()
+
+	s, ok := p.targets[id]
+	if !ok {
+		return nil, ErrNotFound
+	}
+	return s, nil
+}
diff --git a/vendor/github.com/xiang90/probing/server.go b/vendor/github.com/xiang90/probing/server.go
new file mode 100644
index 0000000..0e7b797
--- /dev/null
+++ b/vendor/github.com/xiang90/probing/server.go
@@ -0,0 +1,25 @@
+package probing
+
+import (
+	"encoding/json"
+	"net/http"
+	"time"
+)
+
+func NewHandler() http.Handler {
+	return &httpHealth{}
+}
+
+type httpHealth struct {
+}
+
+type Health struct {
+	OK  bool
+	Now time.Time
+}
+
+func (h *httpHealth) ServeHTTP(w http.ResponseWriter, r *http.Request) {
+	health := Health{OK: true, Now: time.Now()}
+	e := json.NewEncoder(w)
+	e.Encode(health)
+}
diff --git a/vendor/github.com/xiang90/probing/status.go b/vendor/github.com/xiang90/probing/status.go
new file mode 100644
index 0000000..bb5f659
--- /dev/null
+++ b/vendor/github.com/xiang90/probing/status.go
@@ -0,0 +1,108 @@
+package probing
+
+import (
+	"sync"
+	"time"
+)
+
+var (
+	// weight factor
+	α = 0.125
+)
+
+type Status interface {
+	Total() int64
+	Loss() int64
+	Health() bool
+	Err() error
+	// Estimated smoothed round trip time
+	SRTT() time.Duration
+	// Estimated clock difference
+	ClockDiff() time.Duration
+	StopNotify() <-chan struct{}
+}
+
+type status struct {
+	mu        sync.Mutex
+	srtt      time.Duration
+	total     int64
+	loss      int64
+	health    bool
+	err       error
+	clockdiff time.Duration
+	stopC     chan struct{}
+}
+
+// SRTT = (1-α) * SRTT + α * RTT
+func (s *status) SRTT() time.Duration {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+	return s.srtt
+}
+
+func (s *status) Total() int64 {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+	return s.total
+}
+
+func (s *status) Loss() int64 {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+	return s.loss
+}
+
+func (s *status) Health() bool {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+	return s.health
+}
+
+func (s *status) Err() error {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+	return s.err
+}
+
+func (s *status) ClockDiff() time.Duration {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+	return s.clockdiff
+}
+
+func (s *status) StopNotify() <-chan struct{} {
+	return s.stopC
+}
+
+func (s *status) record(rtt time.Duration, when time.Time) {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+
+	s.total += 1
+	s.health = true
+	s.srtt = time.Duration((1-α)*float64(s.srtt) + α*float64(rtt))
+	s.clockdiff = time.Now().Sub(when) - s.srtt/2
+	s.err = nil
+}
+
+func (s *status) recordFailure(err error) {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+
+	s.total++
+	s.health = false
+	s.loss += 1
+	s.err = err
+}
+
+func (s *status) reset() {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+
+	s.srtt = 0
+	s.total = 0
+	s.loss = 0
+	s.health = false
+	s.clockdiff = 0
+	s.err = nil
+}
diff --git a/vendor/go.etcd.io/bbolt/.gitignore b/vendor/go.etcd.io/bbolt/.gitignore
new file mode 100644
index 0000000..3bcd8cb
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/.gitignore
@@ -0,0 +1,5 @@
+*.prof
+*.test
+*.swp
+/bin/
+cover.out
diff --git a/vendor/go.etcd.io/bbolt/.travis.yml b/vendor/go.etcd.io/bbolt/.travis.yml
new file mode 100644
index 0000000..a60300c
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/.travis.yml
@@ -0,0 +1,17 @@
+language: go
+go_import_path: go.etcd.io/bbolt
+
+sudo: false
+
+go:
+- 1.11
+
+before_install:
+- go get -v honnef.co/go/tools/...
+- go get -v github.com/kisielk/errcheck
+
+script:
+- make fmt
+- make test
+- make race
+# - make errcheck
diff --git a/vendor/go.etcd.io/bbolt/LICENSE b/vendor/go.etcd.io/bbolt/LICENSE
new file mode 100644
index 0000000..004e77f
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/LICENSE
@@ -0,0 +1,20 @@
+The MIT License (MIT)
+
+Copyright (c) 2013 Ben Johnson
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of
+this software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
+the Software, and to permit persons to whom the Software is furnished to do so,
+subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS
+FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
+COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER
+IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
+CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/vendor/go.etcd.io/bbolt/Makefile b/vendor/go.etcd.io/bbolt/Makefile
new file mode 100644
index 0000000..2968aaa
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/Makefile
@@ -0,0 +1,38 @@
+BRANCH=`git rev-parse --abbrev-ref HEAD`
+COMMIT=`git rev-parse --short HEAD`
+GOLDFLAGS="-X main.branch $(BRANCH) -X main.commit $(COMMIT)"
+
+default: build
+
+race:
+	@TEST_FREELIST_TYPE=hashmap go test -v -race -test.run="TestSimulate_(100op|1000op)"
+	@echo "array freelist test"
+	@TEST_FREELIST_TYPE=array go test -v -race -test.run="TestSimulate_(100op|1000op)"
+
+fmt:
+	!(gofmt -l -s -d $(shell find . -name \*.go) | grep '[a-z]')
+
+# go get honnef.co/go/tools/simple
+gosimple:
+	gosimple ./...
+
+# go get honnef.co/go/tools/unused
+unused:
+	unused ./...
+
+# go get github.com/kisielk/errcheck
+errcheck:
+	@errcheck -ignorepkg=bytes -ignore=os:Remove go.etcd.io/bbolt
+
+test:
+	TEST_FREELIST_TYPE=hashmap go test -timeout 20m -v -coverprofile cover.out -covermode atomic
+	# Note: gets "program not an importable package" in out of path builds
+	TEST_FREELIST_TYPE=hashmap go test -v ./cmd/bbolt
+
+	@echo "array freelist test"
+
+	@TEST_FREELIST_TYPE=array go test -timeout 20m -v -coverprofile cover.out -covermode atomic
+	# Note: gets "program not an importable package" in out of path builds
+	@TEST_FREELIST_TYPE=array go test -v ./cmd/bbolt
+
+.PHONY: race fmt errcheck test gosimple unused
diff --git a/vendor/go.etcd.io/bbolt/README.md b/vendor/go.etcd.io/bbolt/README.md
new file mode 100644
index 0000000..e9989ef
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/README.md
@@ -0,0 +1,954 @@
+bbolt
+=====
+
+[![Go Report Card](https://goreportcard.com/badge/github.com/etcd-io/bbolt?style=flat-square)](https://goreportcard.com/report/github.com/etcd-io/bbolt)
+[![Coverage](https://codecov.io/gh/etcd-io/bbolt/branch/master/graph/badge.svg)](https://codecov.io/gh/etcd-io/bbolt)
+[![Build Status Travis](https://img.shields.io/travis/etcd-io/bboltlabs.svg?style=flat-square&&branch=master)](https://travis-ci.com/etcd-io/bbolt)
+[![Godoc](http://img.shields.io/badge/go-documentation-blue.svg?style=flat-square)](https://godoc.org/github.com/etcd-io/bbolt)
+[![Releases](https://img.shields.io/github/release/etcd-io/bbolt/all.svg?style=flat-square)](https://github.com/etcd-io/bbolt/releases)
+[![LICENSE](https://img.shields.io/github/license/etcd-io/bbolt.svg?style=flat-square)](https://github.com/etcd-io/bbolt/blob/master/LICENSE)
+
+bbolt is a fork of [Ben Johnson's][gh_ben] [Bolt][bolt] key/value
+store. The purpose of this fork is to provide the Go community with an active
+maintenance and development target for Bolt; the goal is improved reliability
+and stability. bbolt includes bug fixes, performance enhancements, and features
+not found in Bolt while preserving backwards compatibility with the Bolt API.
+
+Bolt is a pure Go key/value store inspired by [Howard Chu's][hyc_symas]
+[LMDB project][lmdb]. The goal of the project is to provide a simple,
+fast, and reliable database for projects that don't require a full database
+server such as Postgres or MySQL.
+
+Since Bolt is meant to be used as such a low-level piece of functionality,
+simplicity is key. The API will be small and only focus on getting values
+and setting values. That's it.
+
+[gh_ben]: https://github.com/benbjohnson
+[bolt]: https://github.com/boltdb/bolt
+[hyc_symas]: https://twitter.com/hyc_symas
+[lmdb]: http://symas.com/mdb/
+
+## Project Status
+
+Bolt is stable, the API is fixed, and the file format is fixed. Full unit
+test coverage and randomized black box testing are used to ensure database
+consistency and thread safety. Bolt is currently used in high-load production
+environments serving databases as large as 1TB. Many companies such as
+Shopify and Heroku use Bolt-backed services every day.
+
+## Project versioning
+
+bbolt uses [semantic versioning](http://semver.org).
+API should not change between patch and minor releases.
+New minor versions may add additional features to the API.
+
+## Table of Contents
+
+  - [Getting Started](#getting-started)
+    - [Installing](#installing)
+    - [Opening a database](#opening-a-database)
+    - [Transactions](#transactions)
+      - [Read-write transactions](#read-write-transactions)
+      - [Read-only transactions](#read-only-transactions)
+      - [Batch read-write transactions](#batch-read-write-transactions)
+      - [Managing transactions manually](#managing-transactions-manually)
+    - [Using buckets](#using-buckets)
+    - [Using key/value pairs](#using-keyvalue-pairs)
+    - [Autoincrementing integer for the bucket](#autoincrementing-integer-for-the-bucket)
+    - [Iterating over keys](#iterating-over-keys)
+      - [Prefix scans](#prefix-scans)
+      - [Range scans](#range-scans)
+      - [ForEach()](#foreach)
+    - [Nested buckets](#nested-buckets)
+    - [Database backups](#database-backups)
+    - [Statistics](#statistics)
+    - [Read-Only Mode](#read-only-mode)
+    - [Mobile Use (iOS/Android)](#mobile-use-iosandroid)
+  - [Resources](#resources)
+  - [Comparison with other databases](#comparison-with-other-databases)
+    - [Postgres, MySQL, & other relational databases](#postgres-mysql--other-relational-databases)
+    - [LevelDB, RocksDB](#leveldb-rocksdb)
+    - [LMDB](#lmdb)
+  - [Caveats & Limitations](#caveats--limitations)
+  - [Reading the Source](#reading-the-source)
+  - [Other Projects Using Bolt](#other-projects-using-bolt)
+
+## Getting Started
+
+### Installing
+
+To start using Bolt, install Go and run `go get`:
+
+```sh
+$ go get go.etcd.io/bbolt/...
+```
+
+This will retrieve the library and install the `bolt` command line utility into
+your `$GOBIN` path.
+
+
+### Importing bbolt
+
+To use bbolt as an embedded key-value store, import as:
+
+```go
+import bolt "go.etcd.io/bbolt"
+
+db, err := bolt.Open(path, 0666, nil)
+if err != nil {
+  return err
+}
+defer db.Close()
+```
+
+
+### Opening a database
+
+The top-level object in Bolt is a `DB`. It is represented as a single file on
+your disk and represents a consistent snapshot of your data.
+
+To open your database, simply use the `bolt.Open()` function:
+
+```go
+package main
+
+import (
+	"log"
+
+	bolt "go.etcd.io/bbolt"
+)
+
+func main() {
+	// Open the my.db data file in your current directory.
+	// It will be created if it doesn't exist.
+	db, err := bolt.Open("my.db", 0600, nil)
+	if err != nil {
+		log.Fatal(err)
+	}
+	defer db.Close()
+
+	...
+}
+```
+
+Please note that Bolt obtains a file lock on the data file so multiple processes
+cannot open the same database at the same time. Opening an already open Bolt
+database will cause it to hang until the other process closes it. To prevent
+an indefinite wait you can pass a timeout option to the `Open()` function:
+
+```go
+db, err := bolt.Open("my.db", 0600, &bolt.Options{Timeout: 1 * time.Second})
+```
+
+
+### Transactions
+
+Bolt allows only one read-write transaction at a time but allows as many
+read-only transactions as you want at a time. Each transaction has a consistent
+view of the data as it existed when the transaction started.
+
+Individual transactions and all objects created from them (e.g. buckets, keys)
+are not thread safe. To work with data in multiple goroutines you must start
+a transaction for each one or use locking to ensure only one goroutine accesses
+a transaction at a time. Creating transaction from the `DB` is thread safe.
+
+Read-only transactions and read-write transactions should not depend on one
+another and generally shouldn't be opened simultaneously in the same goroutine.
+This can cause a deadlock as the read-write transaction needs to periodically
+re-map the data file but it cannot do so while a read-only transaction is open.
+
+
+#### Read-write transactions
+
+To start a read-write transaction, you can use the `DB.Update()` function:
+
+```go
+err := db.Update(func(tx *bolt.Tx) error {
+	...
+	return nil
+})
+```
+
+Inside the closure, you have a consistent view of the database. You commit the
+transaction by returning `nil` at the end. You can also rollback the transaction
+at any point by returning an error. All database operations are allowed inside
+a read-write transaction.
+
+Always check the return error as it will report any disk failures that can cause
+your transaction to not complete. If you return an error within your closure
+it will be passed through.
+
+
+#### Read-only transactions
+
+To start a read-only transaction, you can use the `DB.View()` function:
+
+```go
+err := db.View(func(tx *bolt.Tx) error {
+	...
+	return nil
+})
+```
+
+You also get a consistent view of the database within this closure, however,
+no mutating operations are allowed within a read-only transaction. You can only
+retrieve buckets, retrieve values, and copy the database within a read-only
+transaction.
+
+
+#### Batch read-write transactions
+
+Each `DB.Update()` waits for disk to commit the writes. This overhead
+can be minimized by combining multiple updates with the `DB.Batch()`
+function:
+
+```go
+err := db.Batch(func(tx *bolt.Tx) error {
+	...
+	return nil
+})
+```
+
+Concurrent Batch calls are opportunistically combined into larger
+transactions. Batch is only useful when there are multiple goroutines
+calling it.
+
+The trade-off is that `Batch` can call the given
+function multiple times, if parts of the transaction fail. The
+function must be idempotent and side effects must take effect only
+after a successful return from `DB.Batch()`.
+
+For example: don't display messages from inside the function, instead
+set variables in the enclosing scope:
+
+```go
+var id uint64
+err := db.Batch(func(tx *bolt.Tx) error {
+	// Find last key in bucket, decode as bigendian uint64, increment
+	// by one, encode back to []byte, and add new key.
+	...
+	id = newValue
+	return nil
+})
+if err != nil {
+	return ...
+}
+fmt.Println("Allocated ID %d", id)
+```
+
+
+#### Managing transactions manually
+
+The `DB.View()` and `DB.Update()` functions are wrappers around the `DB.Begin()`
+function. These helper functions will start the transaction, execute a function,
+and then safely close your transaction if an error is returned. This is the
+recommended way to use Bolt transactions.
+
+However, sometimes you may want to manually start and end your transactions.
+You can use the `DB.Begin()` function directly but **please** be sure to close
+the transaction.
+
+```go
+// Start a writable transaction.
+tx, err := db.Begin(true)
+if err != nil {
+    return err
+}
+defer tx.Rollback()
+
+// Use the transaction...
+_, err := tx.CreateBucket([]byte("MyBucket"))
+if err != nil {
+    return err
+}
+
+// Commit the transaction and check for error.
+if err := tx.Commit(); err != nil {
+    return err
+}
+```
+
+The first argument to `DB.Begin()` is a boolean stating if the transaction
+should be writable.
+
+
+### Using buckets
+
+Buckets are collections of key/value pairs within the database. All keys in a
+bucket must be unique. You can create a bucket using the `DB.CreateBucket()`
+function:
+
+```go
+db.Update(func(tx *bolt.Tx) error {
+	b, err := tx.CreateBucket([]byte("MyBucket"))
+	if err != nil {
+		return fmt.Errorf("create bucket: %s", err)
+	}
+	return nil
+})
+```
+
+You can also create a bucket only if it doesn't exist by using the
+`Tx.CreateBucketIfNotExists()` function. It's a common pattern to call this
+function for all your top-level buckets after you open your database so you can
+guarantee that they exist for future transactions.
+
+To delete a bucket, simply call the `Tx.DeleteBucket()` function.
+
+
+### Using key/value pairs
+
+To save a key/value pair to a bucket, use the `Bucket.Put()` function:
+
+```go
+db.Update(func(tx *bolt.Tx) error {
+	b := tx.Bucket([]byte("MyBucket"))
+	err := b.Put([]byte("answer"), []byte("42"))
+	return err
+})
+```
+
+This will set the value of the `"answer"` key to `"42"` in the `MyBucket`
+bucket. To retrieve this value, we can use the `Bucket.Get()` function:
+
+```go
+db.View(func(tx *bolt.Tx) error {
+	b := tx.Bucket([]byte("MyBucket"))
+	v := b.Get([]byte("answer"))
+	fmt.Printf("The answer is: %s\n", v)
+	return nil
+})
+```
+
+The `Get()` function does not return an error because its operation is
+guaranteed to work (unless there is some kind of system failure). If the key
+exists then it will return its byte slice value. If it doesn't exist then it
+will return `nil`. It's important to note that you can have a zero-length value
+set to a key which is different than the key not existing.
+
+Use the `Bucket.Delete()` function to delete a key from the bucket.
+
+Please note that values returned from `Get()` are only valid while the
+transaction is open. If you need to use a value outside of the transaction
+then you must use `copy()` to copy it to another byte slice.
+
+
+### Autoincrementing integer for the bucket
+By using the `NextSequence()` function, you can let Bolt determine a sequence
+which can be used as the unique identifier for your key/value pairs. See the
+example below.
+
+```go
+// CreateUser saves u to the store. The new user ID is set on u once the data is persisted.
+func (s *Store) CreateUser(u *User) error {
+    return s.db.Update(func(tx *bolt.Tx) error {
+        // Retrieve the users bucket.
+        // This should be created when the DB is first opened.
+        b := tx.Bucket([]byte("users"))
+
+        // Generate ID for the user.
+        // This returns an error only if the Tx is closed or not writeable.
+        // That can't happen in an Update() call so I ignore the error check.
+        id, _ := b.NextSequence()
+        u.ID = int(id)
+
+        // Marshal user data into bytes.
+        buf, err := json.Marshal(u)
+        if err != nil {
+            return err
+        }
+
+        // Persist bytes to users bucket.
+        return b.Put(itob(u.ID), buf)
+    })
+}
+
+// itob returns an 8-byte big endian representation of v.
+func itob(v int) []byte {
+    b := make([]byte, 8)
+    binary.BigEndian.PutUint64(b, uint64(v))
+    return b
+}
+
+type User struct {
+    ID int
+    ...
+}
+```
+
+### Iterating over keys
+
+Bolt stores its keys in byte-sorted order within a bucket. This makes sequential
+iteration over these keys extremely fast. To iterate over keys we'll use a
+`Cursor`:
+
+```go
+db.View(func(tx *bolt.Tx) error {
+	// Assume bucket exists and has keys
+	b := tx.Bucket([]byte("MyBucket"))
+
+	c := b.Cursor()
+
+	for k, v := c.First(); k != nil; k, v = c.Next() {
+		fmt.Printf("key=%s, value=%s\n", k, v)
+	}
+
+	return nil
+})
+```
+
+The cursor allows you to move to a specific point in the list of keys and move
+forward or backward through the keys one at a time.
+
+The following functions are available on the cursor:
+
+```
+First()  Move to the first key.
+Last()   Move to the last key.
+Seek()   Move to a specific key.
+Next()   Move to the next key.
+Prev()   Move to the previous key.
+```
+
+Each of those functions has a return signature of `(key []byte, value []byte)`.
+When you have iterated to the end of the cursor then `Next()` will return a
+`nil` key.  You must seek to a position using `First()`, `Last()`, or `Seek()`
+before calling `Next()` or `Prev()`. If you do not seek to a position then
+these functions will return a `nil` key.
+
+During iteration, if the key is non-`nil` but the value is `nil`, that means
+the key refers to a bucket rather than a value.  Use `Bucket.Bucket()` to
+access the sub-bucket.
+
+
+#### Prefix scans
+
+To iterate over a key prefix, you can combine `Seek()` and `bytes.HasPrefix()`:
+
+```go
+db.View(func(tx *bolt.Tx) error {
+	// Assume bucket exists and has keys
+	c := tx.Bucket([]byte("MyBucket")).Cursor()
+
+	prefix := []byte("1234")
+	for k, v := c.Seek(prefix); k != nil && bytes.HasPrefix(k, prefix); k, v = c.Next() {
+		fmt.Printf("key=%s, value=%s\n", k, v)
+	}
+
+	return nil
+})
+```
+
+#### Range scans
+
+Another common use case is scanning over a range such as a time range. If you
+use a sortable time encoding such as RFC3339 then you can query a specific
+date range like this:
+
+```go
+db.View(func(tx *bolt.Tx) error {
+	// Assume our events bucket exists and has RFC3339 encoded time keys.
+	c := tx.Bucket([]byte("Events")).Cursor()
+
+	// Our time range spans the 90's decade.
+	min := []byte("1990-01-01T00:00:00Z")
+	max := []byte("2000-01-01T00:00:00Z")
+
+	// Iterate over the 90's.
+	for k, v := c.Seek(min); k != nil && bytes.Compare(k, max) <= 0; k, v = c.Next() {
+		fmt.Printf("%s: %s\n", k, v)
+	}
+
+	return nil
+})
+```
+
+Note that, while RFC3339 is sortable, the Golang implementation of RFC3339Nano does not use a fixed number of digits after the decimal point and is therefore not sortable.
+
+
+#### ForEach()
+
+You can also use the function `ForEach()` if you know you'll be iterating over
+all the keys in a bucket:
+
+```go
+db.View(func(tx *bolt.Tx) error {
+	// Assume bucket exists and has keys
+	b := tx.Bucket([]byte("MyBucket"))
+
+	b.ForEach(func(k, v []byte) error {
+		fmt.Printf("key=%s, value=%s\n", k, v)
+		return nil
+	})
+	return nil
+})
+```
+
+Please note that keys and values in `ForEach()` are only valid while
+the transaction is open. If you need to use a key or value outside of
+the transaction, you must use `copy()` to copy it to another byte
+slice.
+
+### Nested buckets
+
+You can also store a bucket in a key to create nested buckets. The API is the
+same as the bucket management API on the `DB` object:
+
+```go
+func (*Bucket) CreateBucket(key []byte) (*Bucket, error)
+func (*Bucket) CreateBucketIfNotExists(key []byte) (*Bucket, error)
+func (*Bucket) DeleteBucket(key []byte) error
+```
+
+Say you had a multi-tenant application where the root level bucket was the account bucket. Inside of this bucket was a sequence of accounts which themselves are buckets. And inside the sequence bucket you could have many buckets pertaining to the Account itself (Users, Notes, etc) isolating the information into logical groupings.
+
+```go
+
+// createUser creates a new user in the given account.
+func createUser(accountID int, u *User) error {
+    // Start the transaction.
+    tx, err := db.Begin(true)
+    if err != nil {
+        return err
+    }
+    defer tx.Rollback()
+
+    // Retrieve the root bucket for the account.
+    // Assume this has already been created when the account was set up.
+    root := tx.Bucket([]byte(strconv.FormatUint(accountID, 10)))
+
+    // Setup the users bucket.
+    bkt, err := root.CreateBucketIfNotExists([]byte("USERS"))
+    if err != nil {
+        return err
+    }
+
+    // Generate an ID for the new user.
+    userID, err := bkt.NextSequence()
+    if err != nil {
+        return err
+    }
+    u.ID = userID
+
+    // Marshal and save the encoded user.
+    if buf, err := json.Marshal(u); err != nil {
+        return err
+    } else if err := bkt.Put([]byte(strconv.FormatUint(u.ID, 10)), buf); err != nil {
+        return err
+    }
+
+    // Commit the transaction.
+    if err := tx.Commit(); err != nil {
+        return err
+    }
+
+    return nil
+}
+
+```
+
+
+
+
+### Database backups
+
+Bolt is a single file so it's easy to backup. You can use the `Tx.WriteTo()`
+function to write a consistent view of the database to a writer. If you call
+this from a read-only transaction, it will perform a hot backup and not block
+your other database reads and writes.
+
+By default, it will use a regular file handle which will utilize the operating
+system's page cache. See the [`Tx`](https://godoc.org/go.etcd.io/bbolt#Tx)
+documentation for information about optimizing for larger-than-RAM datasets.
+
+One common use case is to backup over HTTP so you can use tools like `cURL` to
+do database backups:
+
+```go
+func BackupHandleFunc(w http.ResponseWriter, req *http.Request) {
+	err := db.View(func(tx *bolt.Tx) error {
+		w.Header().Set("Content-Type", "application/octet-stream")
+		w.Header().Set("Content-Disposition", `attachment; filename="my.db"`)
+		w.Header().Set("Content-Length", strconv.Itoa(int(tx.Size())))
+		_, err := tx.WriteTo(w)
+		return err
+	})
+	if err != nil {
+		http.Error(w, err.Error(), http.StatusInternalServerError)
+	}
+}
+```
+
+Then you can backup using this command:
+
+```sh
+$ curl http://localhost/backup > my.db
+```
+
+Or you can open your browser to `http://localhost/backup` and it will download
+automatically.
+
+If you want to backup to another file you can use the `Tx.CopyFile()` helper
+function.
+
+
+### Statistics
+
+The database keeps a running count of many of the internal operations it
+performs so you can better understand what's going on. By grabbing a snapshot
+of these stats at two points in time we can see what operations were performed
+in that time range.
+
+For example, we could start a goroutine to log stats every 10 seconds:
+
+```go
+go func() {
+	// Grab the initial stats.
+	prev := db.Stats()
+
+	for {
+		// Wait for 10s.
+		time.Sleep(10 * time.Second)
+
+		// Grab the current stats and diff them.
+		stats := db.Stats()
+		diff := stats.Sub(&prev)
+
+		// Encode stats to JSON and print to STDERR.
+		json.NewEncoder(os.Stderr).Encode(diff)
+
+		// Save stats for the next loop.
+		prev = stats
+	}
+}()
+```
+
+It's also useful to pipe these stats to a service such as statsd for monitoring
+or to provide an HTTP endpoint that will perform a fixed-length sample.
+
+
+### Read-Only Mode
+
+Sometimes it is useful to create a shared, read-only Bolt database. To this,
+set the `Options.ReadOnly` flag when opening your database. Read-only mode
+uses a shared lock to allow multiple processes to read from the database but
+it will block any processes from opening the database in read-write mode.
+
+```go
+db, err := bolt.Open("my.db", 0666, &bolt.Options{ReadOnly: true})
+if err != nil {
+	log.Fatal(err)
+}
+```
+
+### Mobile Use (iOS/Android)
+
+Bolt is able to run on mobile devices by leveraging the binding feature of the
+[gomobile](https://github.com/golang/mobile) tool. Create a struct that will
+contain your database logic and a reference to a `*bolt.DB` with a initializing
+constructor that takes in a filepath where the database file will be stored.
+Neither Android nor iOS require extra permissions or cleanup from using this method.
+
+```go
+func NewBoltDB(filepath string) *BoltDB {
+	db, err := bolt.Open(filepath+"/demo.db", 0600, nil)
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	return &BoltDB{db}
+}
+
+type BoltDB struct {
+	db *bolt.DB
+	...
+}
+
+func (b *BoltDB) Path() string {
+	return b.db.Path()
+}
+
+func (b *BoltDB) Close() {
+	b.db.Close()
+}
+```
+
+Database logic should be defined as methods on this wrapper struct.
+
+To initialize this struct from the native language (both platforms now sync
+their local storage to the cloud. These snippets disable that functionality for the
+database file):
+
+#### Android
+
+```java
+String path;
+if (android.os.Build.VERSION.SDK_INT >=android.os.Build.VERSION_CODES.LOLLIPOP){
+    path = getNoBackupFilesDir().getAbsolutePath();
+} else{
+    path = getFilesDir().getAbsolutePath();
+}
+Boltmobiledemo.BoltDB boltDB = Boltmobiledemo.NewBoltDB(path)
+```
+
+#### iOS
+
+```objc
+- (void)demo {
+    NSString* path = [NSSearchPathForDirectoriesInDomains(NSLibraryDirectory,
+                                                          NSUserDomainMask,
+                                                          YES) objectAtIndex:0];
+	GoBoltmobiledemoBoltDB * demo = GoBoltmobiledemoNewBoltDB(path);
+	[self addSkipBackupAttributeToItemAtPath:demo.path];
+	//Some DB Logic would go here
+	[demo close];
+}
+
+- (BOOL)addSkipBackupAttributeToItemAtPath:(NSString *) filePathString
+{
+    NSURL* URL= [NSURL fileURLWithPath: filePathString];
+    assert([[NSFileManager defaultManager] fileExistsAtPath: [URL path]]);
+
+    NSError *error = nil;
+    BOOL success = [URL setResourceValue: [NSNumber numberWithBool: YES]
+                                  forKey: NSURLIsExcludedFromBackupKey error: &error];
+    if(!success){
+        NSLog(@"Error excluding %@ from backup %@", [URL lastPathComponent], error);
+    }
+    return success;
+}
+
+```
+
+## Resources
+
+For more information on getting started with Bolt, check out the following articles:
+
+* [Intro to BoltDB: Painless Performant Persistence](http://npf.io/2014/07/intro-to-boltdb-painless-performant-persistence/) by [Nate Finch](https://github.com/natefinch).
+* [Bolt -- an embedded key/value database for Go](https://www.progville.com/go/bolt-embedded-db-golang/) by Progville
+
+
+## Comparison with other databases
+
+### Postgres, MySQL, & other relational databases
+
+Relational databases structure data into rows and are only accessible through
+the use of SQL. This approach provides flexibility in how you store and query
+your data but also incurs overhead in parsing and planning SQL statements. Bolt
+accesses all data by a byte slice key. This makes Bolt fast to read and write
+data by key but provides no built-in support for joining values together.
+
+Most relational databases (with the exception of SQLite) are standalone servers
+that run separately from your application. This gives your systems
+flexibility to connect multiple application servers to a single database
+server but also adds overhead in serializing and transporting data over the
+network. Bolt runs as a library included in your application so all data access
+has to go through your application's process. This brings data closer to your
+application but limits multi-process access to the data.
+
+
+### LevelDB, RocksDB
+
+LevelDB and its derivatives (RocksDB, HyperLevelDB) are similar to Bolt in that
+they are libraries bundled into the application, however, their underlying
+structure is a log-structured merge-tree (LSM tree). An LSM tree optimizes
+random writes by using a write ahead log and multi-tiered, sorted files called
+SSTables. Bolt uses a B+tree internally and only a single file. Both approaches
+have trade-offs.
+
+If you require a high random write throughput (>10,000 w/sec) or you need to use
+spinning disks then LevelDB could be a good choice. If your application is
+read-heavy or does a lot of range scans then Bolt could be a good choice.
+
+One other important consideration is that LevelDB does not have transactions.
+It supports batch writing of key/values pairs and it supports read snapshots
+but it will not give you the ability to do a compare-and-swap operation safely.
+Bolt supports fully serializable ACID transactions.
+
+
+### LMDB
+
+Bolt was originally a port of LMDB so it is architecturally similar. Both use
+a B+tree, have ACID semantics with fully serializable transactions, and support
+lock-free MVCC using a single writer and multiple readers.
+
+The two projects have somewhat diverged. LMDB heavily focuses on raw performance
+while Bolt has focused on simplicity and ease of use. For example, LMDB allows
+several unsafe actions such as direct writes for the sake of performance. Bolt
+opts to disallow actions which can leave the database in a corrupted state. The
+only exception to this in Bolt is `DB.NoSync`.
+
+There are also a few differences in API. LMDB requires a maximum mmap size when
+opening an `mdb_env` whereas Bolt will handle incremental mmap resizing
+automatically. LMDB overloads the getter and setter functions with multiple
+flags whereas Bolt splits these specialized cases into their own functions.
+
+
+## Caveats & Limitations
+
+It's important to pick the right tool for the job and Bolt is no exception.
+Here are a few things to note when evaluating and using Bolt:
+
+* Bolt is good for read intensive workloads. Sequential write performance is
+  also fast but random writes can be slow. You can use `DB.Batch()` or add a
+  write-ahead log to help mitigate this issue.
+
+* Bolt uses a B+tree internally so there can be a lot of random page access.
+  SSDs provide a significant performance boost over spinning disks.
+
+* Try to avoid long running read transactions. Bolt uses copy-on-write so
+  old pages cannot be reclaimed while an old transaction is using them.
+
+* Byte slices returned from Bolt are only valid during a transaction. Once the
+  transaction has been committed or rolled back then the memory they point to
+  can be reused by a new page or can be unmapped from virtual memory and you'll
+  see an `unexpected fault address` panic when accessing it.
+
+* Bolt uses an exclusive write lock on the database file so it cannot be
+  shared by multiple processes.
+
+* Be careful when using `Bucket.FillPercent`. Setting a high fill percent for
+  buckets that have random inserts will cause your database to have very poor
+  page utilization.
+
+* Use larger buckets in general. Smaller buckets causes poor page utilization
+  once they become larger than the page size (typically 4KB).
+
+* Bulk loading a lot of random writes into a new bucket can be slow as the
+  page will not split until the transaction is committed. Randomly inserting
+  more than 100,000 key/value pairs into a single new bucket in a single
+  transaction is not advised.
+
+* Bolt uses a memory-mapped file so the underlying operating system handles the
+  caching of the data. Typically, the OS will cache as much of the file as it
+  can in memory and will release memory as needed to other processes. This means
+  that Bolt can show very high memory usage when working with large databases.
+  However, this is expected and the OS will release memory as needed. Bolt can
+  handle databases much larger than the available physical RAM, provided its
+  memory-map fits in the process virtual address space. It may be problematic
+  on 32-bits systems.
+
+* The data structures in the Bolt database are memory mapped so the data file
+  will be endian specific. This means that you cannot copy a Bolt file from a
+  little endian machine to a big endian machine and have it work. For most
+  users this is not a concern since most modern CPUs are little endian.
+
+* Because of the way pages are laid out on disk, Bolt cannot truncate data files
+  and return free pages back to the disk. Instead, Bolt maintains a free list
+  of unused pages within its data file. These free pages can be reused by later
+  transactions. This works well for many use cases as databases generally tend
+  to grow. However, it's important to note that deleting large chunks of data
+  will not allow you to reclaim that space on disk.
+
+  For more information on page allocation, [see this comment][page-allocation].
+
+[page-allocation]: https://github.com/boltdb/bolt/issues/308#issuecomment-74811638
+
+
+## Reading the Source
+
+Bolt is a relatively small code base (<5KLOC) for an embedded, serializable,
+transactional key/value database so it can be a good starting point for people
+interested in how databases work.
+
+The best places to start are the main entry points into Bolt:
+
+- `Open()` - Initializes the reference to the database. It's responsible for
+  creating the database if it doesn't exist, obtaining an exclusive lock on the
+  file, reading the meta pages, & memory-mapping the file.
+
+- `DB.Begin()` - Starts a read-only or read-write transaction depending on the
+  value of the `writable` argument. This requires briefly obtaining the "meta"
+  lock to keep track of open transactions. Only one read-write transaction can
+  exist at a time so the "rwlock" is acquired during the life of a read-write
+  transaction.
+
+- `Bucket.Put()` - Writes a key/value pair into a bucket. After validating the
+  arguments, a cursor is used to traverse the B+tree to the page and position
+  where they key & value will be written. Once the position is found, the bucket
+  materializes the underlying page and the page's parent pages into memory as
+  "nodes". These nodes are where mutations occur during read-write transactions.
+  These changes get flushed to disk during commit.
+
+- `Bucket.Get()` - Retrieves a key/value pair from a bucket. This uses a cursor
+  to move to the page & position of a key/value pair. During a read-only
+  transaction, the key and value data is returned as a direct reference to the
+  underlying mmap file so there's no allocation overhead. For read-write
+  transactions, this data may reference the mmap file or one of the in-memory
+  node values.
+
+- `Cursor` - This object is simply for traversing the B+tree of on-disk pages
+  or in-memory nodes. It can seek to a specific key, move to the first or last
+  value, or it can move forward or backward. The cursor handles the movement up
+  and down the B+tree transparently to the end user.
+
+- `Tx.Commit()` - Converts the in-memory dirty nodes and the list of free pages
+  into pages to be written to disk. Writing to disk then occurs in two phases.
+  First, the dirty pages are written to disk and an `fsync()` occurs. Second, a
+  new meta page with an incremented transaction ID is written and another
+  `fsync()` occurs. This two phase write ensures that partially written data
+  pages are ignored in the event of a crash since the meta page pointing to them
+  is never written. Partially written meta pages are invalidated because they
+  are written with a checksum.
+
+If you have additional notes that could be helpful for others, please submit
+them via pull request.
+
+
+## Other Projects Using Bolt
+
+Below is a list of public, open source projects that use Bolt:
+
+* [Algernon](https://github.com/xyproto/algernon) - A HTTP/2 web server with built-in support for Lua. Uses BoltDB as the default database backend.
+* [Bazil](https://bazil.org/) - A file system that lets your data reside where it is most convenient for it to reside.
+* [bolter](https://github.com/hasit/bolter) - Command-line app for viewing BoltDB file in your terminal.
+* [boltcli](https://github.com/spacewander/boltcli) - the redis-cli for boltdb with Lua script support.
+* [BoltHold](https://github.com/timshannon/bolthold) - An embeddable NoSQL store for Go types built on BoltDB
+* [BoltStore](https://github.com/yosssi/boltstore) - Session store using Bolt.
+* [Boltdb Boilerplate](https://github.com/bobintornado/boltdb-boilerplate) - Boilerplate wrapper around bolt aiming to make simple calls one-liners.
+* [BoltDbWeb](https://github.com/evnix/boltdbweb) - A web based GUI for BoltDB files.
+* [bleve](http://www.blevesearch.com/) - A pure Go search engine similar to ElasticSearch that uses Bolt as the default storage backend.
+* [btcwallet](https://github.com/btcsuite/btcwallet) - A bitcoin wallet.
+* [buckets](https://github.com/joyrexus/buckets) - a bolt wrapper streamlining
+  simple tx and key scans.
+* [cayley](https://github.com/google/cayley) - Cayley is an open-source graph database using Bolt as optional backend.
+* [ChainStore](https://github.com/pressly/chainstore) - Simple key-value interface to a variety of storage engines organized as a chain of operations.
+* [Consul](https://github.com/hashicorp/consul) - Consul is service discovery and configuration made easy. Distributed, highly available, and datacenter-aware.
+* [DVID](https://github.com/janelia-flyem/dvid) - Added Bolt as optional storage engine and testing it against Basho-tuned leveldb.
+* [dcrwallet](https://github.com/decred/dcrwallet) - A wallet for the Decred cryptocurrency.
+* [drive](https://github.com/odeke-em/drive) - drive is an unofficial Google Drive command line client for \*NIX operating systems.
+* [event-shuttle](https://github.com/sclasen/event-shuttle) - A Unix system service to collect and reliably deliver messages to Kafka.
+* [Freehold](http://tshannon.bitbucket.org/freehold/) - An open, secure, and lightweight platform for your files and data.
+* [Go Report Card](https://goreportcard.com/) - Go code quality report cards as a (free and open source) service.
+* [GoWebApp](https://github.com/josephspurrier/gowebapp) - A basic MVC web application in Go using BoltDB.
+* [GoShort](https://github.com/pankajkhairnar/goShort) - GoShort is a URL shortener written in Golang and BoltDB for persistent key/value storage and for routing it's using high performent HTTPRouter.
+* [gopherpit](https://github.com/gopherpit/gopherpit) - A web service to manage Go remote import paths with custom domains
+* [Gitchain](https://github.com/gitchain/gitchain) - Decentralized, peer-to-peer Git repositories aka "Git meets Bitcoin".
+* [InfluxDB](https://influxdata.com) - Scalable datastore for metrics, events, and real-time analytics.
+* [ipLocator](https://github.com/AndreasBriese/ipLocator) - A fast ip-geo-location-server using bolt with bloom filters.
+* [ipxed](https://github.com/kelseyhightower/ipxed) - Web interface and api for ipxed.
+* [Ironsmith](https://github.com/timshannon/ironsmith) - A simple, script-driven continuous integration (build - > test -> release) tool, with no external dependencies
+* [Kala](https://github.com/ajvb/kala) - Kala is a modern job scheduler optimized to run on a single node. It is persistent, JSON over HTTP API, ISO 8601 duration notation, and dependent jobs.
+* [Key Value Access Langusge (KVAL)](https://github.com/kval-access-language) - A proposed grammar for key-value datastores offering a bbolt binding.
+* [LedisDB](https://github.com/siddontang/ledisdb) - A high performance NoSQL, using Bolt as optional storage.
+* [lru](https://github.com/crowdriff/lru) - Easy to use Bolt-backed Least-Recently-Used (LRU) read-through cache with chainable remote stores.
+* [mbuckets](https://github.com/abhigupta912/mbuckets) - A Bolt wrapper that allows easy operations on multi level (nested) buckets.
+* [MetricBase](https://github.com/msiebuhr/MetricBase) - Single-binary version of Graphite.
+* [MuLiFS](https://github.com/dankomiocevic/mulifs) - Music Library Filesystem creates a filesystem to organise your music files.
+* [Operation Go: A Routine Mission](http://gocode.io) - An online programming game for Golang using Bolt for user accounts and a leaderboard.
+* [photosite/session](https://godoc.org/bitbucket.org/kardianos/photosite/session) - Sessions for a photo viewing site.
+* [Prometheus Annotation Server](https://github.com/oliver006/prom_annotation_server) - Annotation server for PromDash & Prometheus service monitoring system.
+* [reef-pi](https://github.com/reef-pi/reef-pi) - reef-pi is an award winning, modular, DIY reef tank controller using easy to learn electronics based on a Raspberry Pi.
+* [Request Baskets](https://github.com/darklynx/request-baskets) - A web service to collect arbitrary HTTP requests and inspect them via REST API or simple web UI, similar to [RequestBin](http://requestb.in/) service
+* [Seaweed File System](https://github.com/chrislusf/seaweedfs) - Highly scalable distributed key~file system with O(1) disk read.
+* [stow](https://github.com/djherbis/stow) -  a persistence manager for objects
+  backed by boltdb.
+* [Storm](https://github.com/asdine/storm) - Simple and powerful ORM for BoltDB.
+* [SimpleBolt](https://github.com/xyproto/simplebolt) - A simple way to use BoltDB. Deals mainly with strings.
+* [Skybox Analytics](https://github.com/skybox/skybox) - A standalone funnel analysis tool for web analytics.
+* [Scuttlebutt](https://github.com/benbjohnson/scuttlebutt) - Uses Bolt to store and process all Twitter mentions of GitHub projects.
+* [tentacool](https://github.com/optiflows/tentacool) - REST api server to manage system stuff (IP, DNS, Gateway...) on a linux server.
+* [torrent](https://github.com/anacrolix/torrent) - Full-featured BitTorrent client package and utilities in Go. BoltDB is a storage backend in development.
+* [Wiki](https://github.com/peterhellberg/wiki) - A tiny wiki using Goji, BoltDB and Blackfriday.
+
+If you are using Bolt in a project please send a pull request to add it to the list.
diff --git a/vendor/go.etcd.io/bbolt/bolt_386.go b/vendor/go.etcd.io/bbolt/bolt_386.go
new file mode 100644
index 0000000..4d35ee7
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/bolt_386.go
@@ -0,0 +1,10 @@
+package bbolt
+
+// maxMapSize represents the largest mmap size supported by Bolt.
+const maxMapSize = 0x7FFFFFFF // 2GB
+
+// maxAllocSize is the size used when creating array pointers.
+const maxAllocSize = 0xFFFFFFF
+
+// Are unaligned load/stores broken on this arch?
+var brokenUnaligned = false
diff --git a/vendor/go.etcd.io/bbolt/bolt_amd64.go b/vendor/go.etcd.io/bbolt/bolt_amd64.go
new file mode 100644
index 0000000..60a52da
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/bolt_amd64.go
@@ -0,0 +1,10 @@
+package bbolt
+
+// maxMapSize represents the largest mmap size supported by Bolt.
+const maxMapSize = 0xFFFFFFFFFFFF // 256TB
+
+// maxAllocSize is the size used when creating array pointers.
+const maxAllocSize = 0x7FFFFFFF
+
+// Are unaligned load/stores broken on this arch?
+var brokenUnaligned = false
diff --git a/vendor/go.etcd.io/bbolt/bolt_arm.go b/vendor/go.etcd.io/bbolt/bolt_arm.go
new file mode 100644
index 0000000..105d27d
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/bolt_arm.go
@@ -0,0 +1,28 @@
+package bbolt
+
+import "unsafe"
+
+// maxMapSize represents the largest mmap size supported by Bolt.
+const maxMapSize = 0x7FFFFFFF // 2GB
+
+// maxAllocSize is the size used when creating array pointers.
+const maxAllocSize = 0xFFFFFFF
+
+// Are unaligned load/stores broken on this arch?
+var brokenUnaligned bool
+
+func init() {
+	// Simple check to see whether this arch handles unaligned load/stores
+	// correctly.
+
+	// ARM9 and older devices require load/stores to be from/to aligned
+	// addresses. If not, the lower 2 bits are cleared and that address is
+	// read in a jumbled up order.
+
+	// See http://infocenter.arm.com/help/index.jsp?topic=/com.arm.doc.faqs/ka15414.html
+
+	raw := [6]byte{0xfe, 0xef, 0x11, 0x22, 0x22, 0x11}
+	val := *(*uint32)(unsafe.Pointer(uintptr(unsafe.Pointer(&raw)) + 2))
+
+	brokenUnaligned = val != 0x11222211
+}
diff --git a/vendor/go.etcd.io/bbolt/bolt_arm64.go b/vendor/go.etcd.io/bbolt/bolt_arm64.go
new file mode 100644
index 0000000..f5aa2a5
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/bolt_arm64.go
@@ -0,0 +1,12 @@
+// +build arm64
+
+package bbolt
+
+// maxMapSize represents the largest mmap size supported by Bolt.
+const maxMapSize = 0xFFFFFFFFFFFF // 256TB
+
+// maxAllocSize is the size used when creating array pointers.
+const maxAllocSize = 0x7FFFFFFF
+
+// Are unaligned load/stores broken on this arch?
+var brokenUnaligned = false
diff --git a/vendor/go.etcd.io/bbolt/bolt_linux.go b/vendor/go.etcd.io/bbolt/bolt_linux.go
new file mode 100644
index 0000000..7707bca
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/bolt_linux.go
@@ -0,0 +1,10 @@
+package bbolt
+
+import (
+	"syscall"
+)
+
+// fdatasync flushes written data to a file descriptor.
+func fdatasync(db *DB) error {
+	return syscall.Fdatasync(int(db.file.Fd()))
+}
diff --git a/vendor/go.etcd.io/bbolt/bolt_mips64x.go b/vendor/go.etcd.io/bbolt/bolt_mips64x.go
new file mode 100644
index 0000000..baeb289
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/bolt_mips64x.go
@@ -0,0 +1,12 @@
+// +build mips64 mips64le
+
+package bbolt
+
+// maxMapSize represents the largest mmap size supported by Bolt.
+const maxMapSize = 0x8000000000 // 512GB
+
+// maxAllocSize is the size used when creating array pointers.
+const maxAllocSize = 0x7FFFFFFF
+
+// Are unaligned load/stores broken on this arch?
+var brokenUnaligned = false
diff --git a/vendor/go.etcd.io/bbolt/bolt_mipsx.go b/vendor/go.etcd.io/bbolt/bolt_mipsx.go
new file mode 100644
index 0000000..2d9b1a9
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/bolt_mipsx.go
@@ -0,0 +1,12 @@
+// +build mips mipsle
+
+package bbolt
+
+// maxMapSize represents the largest mmap size supported by Bolt.
+const maxMapSize = 0x40000000 // 1GB
+
+// maxAllocSize is the size used when creating array pointers.
+const maxAllocSize = 0xFFFFFFF
+
+// Are unaligned load/stores broken on this arch?
+var brokenUnaligned = false
diff --git a/vendor/go.etcd.io/bbolt/bolt_openbsd.go b/vendor/go.etcd.io/bbolt/bolt_openbsd.go
new file mode 100644
index 0000000..d7f5035
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/bolt_openbsd.go
@@ -0,0 +1,27 @@
+package bbolt
+
+import (
+	"syscall"
+	"unsafe"
+)
+
+const (
+	msAsync      = 1 << iota // perform asynchronous writes
+	msSync                   // perform synchronous writes
+	msInvalidate             // invalidate cached data
+)
+
+func msync(db *DB) error {
+	_, _, errno := syscall.Syscall(syscall.SYS_MSYNC, uintptr(unsafe.Pointer(db.data)), uintptr(db.datasz), msInvalidate)
+	if errno != 0 {
+		return errno
+	}
+	return nil
+}
+
+func fdatasync(db *DB) error {
+	if db.data != nil {
+		return msync(db)
+	}
+	return db.file.Sync()
+}
diff --git a/vendor/go.etcd.io/bbolt/bolt_ppc.go b/vendor/go.etcd.io/bbolt/bolt_ppc.go
new file mode 100644
index 0000000..6980471
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/bolt_ppc.go
@@ -0,0 +1,12 @@
+// +build ppc
+
+package bbolt
+
+// maxMapSize represents the largest mmap size supported by Bolt.
+const maxMapSize = 0x7FFFFFFF // 2GB
+
+// maxAllocSize is the size used when creating array pointers.
+const maxAllocSize = 0xFFFFFFF
+
+// Are unaligned load/stores broken on this arch?
+var brokenUnaligned = false
diff --git a/vendor/go.etcd.io/bbolt/bolt_ppc64.go b/vendor/go.etcd.io/bbolt/bolt_ppc64.go
new file mode 100644
index 0000000..3565908
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/bolt_ppc64.go
@@ -0,0 +1,12 @@
+// +build ppc64
+
+package bbolt
+
+// maxMapSize represents the largest mmap size supported by Bolt.
+const maxMapSize = 0xFFFFFFFFFFFF // 256TB
+
+// maxAllocSize is the size used when creating array pointers.
+const maxAllocSize = 0x7FFFFFFF
+
+// Are unaligned load/stores broken on this arch?
+var brokenUnaligned = false
diff --git a/vendor/go.etcd.io/bbolt/bolt_ppc64le.go b/vendor/go.etcd.io/bbolt/bolt_ppc64le.go
new file mode 100644
index 0000000..422c7c6
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/bolt_ppc64le.go
@@ -0,0 +1,12 @@
+// +build ppc64le
+
+package bbolt
+
+// maxMapSize represents the largest mmap size supported by Bolt.
+const maxMapSize = 0xFFFFFFFFFFFF // 256TB
+
+// maxAllocSize is the size used when creating array pointers.
+const maxAllocSize = 0x7FFFFFFF
+
+// Are unaligned load/stores broken on this arch?
+var brokenUnaligned = false
diff --git a/vendor/go.etcd.io/bbolt/bolt_riscv64.go b/vendor/go.etcd.io/bbolt/bolt_riscv64.go
new file mode 100644
index 0000000..07b4b47
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/bolt_riscv64.go
@@ -0,0 +1,12 @@
+// +build riscv64
+
+package bbolt
+
+// maxMapSize represents the largest mmap size supported by Bolt.
+const maxMapSize = 0xFFFFFFFFFFFF // 256TB
+
+// maxAllocSize is the size used when creating array pointers.
+const maxAllocSize = 0x7FFFFFFF
+
+// Are unaligned load/stores broken on this arch?
+var brokenUnaligned = true
diff --git a/vendor/go.etcd.io/bbolt/bolt_s390x.go b/vendor/go.etcd.io/bbolt/bolt_s390x.go
new file mode 100644
index 0000000..6d3fcb8
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/bolt_s390x.go
@@ -0,0 +1,12 @@
+// +build s390x
+
+package bbolt
+
+// maxMapSize represents the largest mmap size supported by Bolt.
+const maxMapSize = 0xFFFFFFFFFFFF // 256TB
+
+// maxAllocSize is the size used when creating array pointers.
+const maxAllocSize = 0x7FFFFFFF
+
+// Are unaligned load/stores broken on this arch?
+var brokenUnaligned = false
diff --git a/vendor/go.etcd.io/bbolt/bolt_unix.go b/vendor/go.etcd.io/bbolt/bolt_unix.go
new file mode 100644
index 0000000..5f2bb51
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/bolt_unix.go
@@ -0,0 +1,93 @@
+// +build !windows,!plan9,!solaris
+
+package bbolt
+
+import (
+	"fmt"
+	"syscall"
+	"time"
+	"unsafe"
+)
+
+// flock acquires an advisory lock on a file descriptor.
+func flock(db *DB, exclusive bool, timeout time.Duration) error {
+	var t time.Time
+	if timeout != 0 {
+		t = time.Now()
+	}
+	fd := db.file.Fd()
+	flag := syscall.LOCK_NB
+	if exclusive {
+		flag |= syscall.LOCK_EX
+	} else {
+		flag |= syscall.LOCK_SH
+	}
+	for {
+		// Attempt to obtain an exclusive lock.
+		err := syscall.Flock(int(fd), flag)
+		if err == nil {
+			return nil
+		} else if err != syscall.EWOULDBLOCK {
+			return err
+		}
+
+		// If we timed out then return an error.
+		if timeout != 0 && time.Since(t) > timeout-flockRetryTimeout {
+			return ErrTimeout
+		}
+
+		// Wait for a bit and try again.
+		time.Sleep(flockRetryTimeout)
+	}
+}
+
+// funlock releases an advisory lock on a file descriptor.
+func funlock(db *DB) error {
+	return syscall.Flock(int(db.file.Fd()), syscall.LOCK_UN)
+}
+
+// mmap memory maps a DB's data file.
+func mmap(db *DB, sz int) error {
+	// Map the data file to memory.
+	b, err := syscall.Mmap(int(db.file.Fd()), 0, sz, syscall.PROT_READ, syscall.MAP_SHARED|db.MmapFlags)
+	if err != nil {
+		return err
+	}
+
+	// Advise the kernel that the mmap is accessed randomly.
+	err = madvise(b, syscall.MADV_RANDOM)
+	if err != nil && err != syscall.ENOSYS {
+		// Ignore not implemented error in kernel because it still works.
+		return fmt.Errorf("madvise: %s", err)
+	}
+
+	// Save the original byte slice and convert to a byte array pointer.
+	db.dataref = b
+	db.data = (*[maxMapSize]byte)(unsafe.Pointer(&b[0]))
+	db.datasz = sz
+	return nil
+}
+
+// munmap unmaps a DB's data file from memory.
+func munmap(db *DB) error {
+	// Ignore the unmap if we have no mapped data.
+	if db.dataref == nil {
+		return nil
+	}
+
+	// Unmap using the original byte slice.
+	err := syscall.Munmap(db.dataref)
+	db.dataref = nil
+	db.data = nil
+	db.datasz = 0
+	return err
+}
+
+// NOTE: This function is copied from stdlib because it is not available on darwin.
+func madvise(b []byte, advice int) (err error) {
+	_, _, e1 := syscall.Syscall(syscall.SYS_MADVISE, uintptr(unsafe.Pointer(&b[0])), uintptr(len(b)), uintptr(advice))
+	if e1 != 0 {
+		err = e1
+	}
+	return
+}
diff --git a/vendor/go.etcd.io/bbolt/bolt_unix_solaris.go b/vendor/go.etcd.io/bbolt/bolt_unix_solaris.go
new file mode 100644
index 0000000..babad65
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/bolt_unix_solaris.go
@@ -0,0 +1,88 @@
+package bbolt
+
+import (
+	"fmt"
+	"syscall"
+	"time"
+	"unsafe"
+
+	"golang.org/x/sys/unix"
+)
+
+// flock acquires an advisory lock on a file descriptor.
+func flock(db *DB, exclusive bool, timeout time.Duration) error {
+	var t time.Time
+	if timeout != 0 {
+		t = time.Now()
+	}
+	fd := db.file.Fd()
+	var lockType int16
+	if exclusive {
+		lockType = syscall.F_WRLCK
+	} else {
+		lockType = syscall.F_RDLCK
+	}
+	for {
+		// Attempt to obtain an exclusive lock.
+		lock := syscall.Flock_t{Type: lockType}
+		err := syscall.FcntlFlock(fd, syscall.F_SETLK, &lock)
+		if err == nil {
+			return nil
+		} else if err != syscall.EAGAIN {
+			return err
+		}
+
+		// If we timed out then return an error.
+		if timeout != 0 && time.Since(t) > timeout-flockRetryTimeout {
+			return ErrTimeout
+		}
+
+		// Wait for a bit and try again.
+		time.Sleep(flockRetryTimeout)
+	}
+}
+
+// funlock releases an advisory lock on a file descriptor.
+func funlock(db *DB) error {
+	var lock syscall.Flock_t
+	lock.Start = 0
+	lock.Len = 0
+	lock.Type = syscall.F_UNLCK
+	lock.Whence = 0
+	return syscall.FcntlFlock(uintptr(db.file.Fd()), syscall.F_SETLK, &lock)
+}
+
+// mmap memory maps a DB's data file.
+func mmap(db *DB, sz int) error {
+	// Map the data file to memory.
+	b, err := unix.Mmap(int(db.file.Fd()), 0, sz, syscall.PROT_READ, syscall.MAP_SHARED|db.MmapFlags)
+	if err != nil {
+		return err
+	}
+
+	// Advise the kernel that the mmap is accessed randomly.
+	if err := unix.Madvise(b, syscall.MADV_RANDOM); err != nil {
+		return fmt.Errorf("madvise: %s", err)
+	}
+
+	// Save the original byte slice and convert to a byte array pointer.
+	db.dataref = b
+	db.data = (*[maxMapSize]byte)(unsafe.Pointer(&b[0]))
+	db.datasz = sz
+	return nil
+}
+
+// munmap unmaps a DB's data file from memory.
+func munmap(db *DB) error {
+	// Ignore the unmap if we have no mapped data.
+	if db.dataref == nil {
+		return nil
+	}
+
+	// Unmap using the original byte slice.
+	err := unix.Munmap(db.dataref)
+	db.dataref = nil
+	db.data = nil
+	db.datasz = 0
+	return err
+}
diff --git a/vendor/go.etcd.io/bbolt/bolt_windows.go b/vendor/go.etcd.io/bbolt/bolt_windows.go
new file mode 100644
index 0000000..fca178b
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/bolt_windows.go
@@ -0,0 +1,141 @@
+package bbolt
+
+import (
+	"fmt"
+	"os"
+	"syscall"
+	"time"
+	"unsafe"
+)
+
+// LockFileEx code derived from golang build filemutex_windows.go @ v1.5.1
+var (
+	modkernel32      = syscall.NewLazyDLL("kernel32.dll")
+	procLockFileEx   = modkernel32.NewProc("LockFileEx")
+	procUnlockFileEx = modkernel32.NewProc("UnlockFileEx")
+)
+
+const (
+	// see https://msdn.microsoft.com/en-us/library/windows/desktop/aa365203(v=vs.85).aspx
+	flagLockExclusive       = 2
+	flagLockFailImmediately = 1
+
+	// see https://msdn.microsoft.com/en-us/library/windows/desktop/ms681382(v=vs.85).aspx
+	errLockViolation syscall.Errno = 0x21
+)
+
+func lockFileEx(h syscall.Handle, flags, reserved, locklow, lockhigh uint32, ol *syscall.Overlapped) (err error) {
+	r, _, err := procLockFileEx.Call(uintptr(h), uintptr(flags), uintptr(reserved), uintptr(locklow), uintptr(lockhigh), uintptr(unsafe.Pointer(ol)))
+	if r == 0 {
+		return err
+	}
+	return nil
+}
+
+func unlockFileEx(h syscall.Handle, reserved, locklow, lockhigh uint32, ol *syscall.Overlapped) (err error) {
+	r, _, err := procUnlockFileEx.Call(uintptr(h), uintptr(reserved), uintptr(locklow), uintptr(lockhigh), uintptr(unsafe.Pointer(ol)), 0)
+	if r == 0 {
+		return err
+	}
+	return nil
+}
+
+// fdatasync flushes written data to a file descriptor.
+func fdatasync(db *DB) error {
+	return db.file.Sync()
+}
+
+// flock acquires an advisory lock on a file descriptor.
+func flock(db *DB, exclusive bool, timeout time.Duration) error {
+	var t time.Time
+	if timeout != 0 {
+		t = time.Now()
+	}
+	var flag uint32 = flagLockFailImmediately
+	if exclusive {
+		flag |= flagLockExclusive
+	}
+	for {
+		// Fix for https://github.com/etcd-io/bbolt/issues/121. Use byte-range
+		// -1..0 as the lock on the database file.
+		var m1 uint32 = (1 << 32) - 1 // -1 in a uint32
+		err := lockFileEx(syscall.Handle(db.file.Fd()), flag, 0, 1, 0, &syscall.Overlapped{
+			Offset:     m1,
+			OffsetHigh: m1,
+		})
+
+		if err == nil {
+			return nil
+		} else if err != errLockViolation {
+			return err
+		}
+
+		// If we timed oumercit then return an error.
+		if timeout != 0 && time.Since(t) > timeout-flockRetryTimeout {
+			return ErrTimeout
+		}
+
+		// Wait for a bit and try again.
+		time.Sleep(flockRetryTimeout)
+	}
+}
+
+// funlock releases an advisory lock on a file descriptor.
+func funlock(db *DB) error {
+	var m1 uint32 = (1 << 32) - 1 // -1 in a uint32
+	err := unlockFileEx(syscall.Handle(db.file.Fd()), 0, 1, 0, &syscall.Overlapped{
+		Offset:     m1,
+		OffsetHigh: m1,
+	})
+	return err
+}
+
+// mmap memory maps a DB's data file.
+// Based on: https://github.com/edsrzf/mmap-go
+func mmap(db *DB, sz int) error {
+	if !db.readOnly {
+		// Truncate the database to the size of the mmap.
+		if err := db.file.Truncate(int64(sz)); err != nil {
+			return fmt.Errorf("truncate: %s", err)
+		}
+	}
+
+	// Open a file mapping handle.
+	sizelo := uint32(sz >> 32)
+	sizehi := uint32(sz) & 0xffffffff
+	h, errno := syscall.CreateFileMapping(syscall.Handle(db.file.Fd()), nil, syscall.PAGE_READONLY, sizelo, sizehi, nil)
+	if h == 0 {
+		return os.NewSyscallError("CreateFileMapping", errno)
+	}
+
+	// Create the memory map.
+	addr, errno := syscall.MapViewOfFile(h, syscall.FILE_MAP_READ, 0, 0, uintptr(sz))
+	if addr == 0 {
+		return os.NewSyscallError("MapViewOfFile", errno)
+	}
+
+	// Close mapping handle.
+	if err := syscall.CloseHandle(syscall.Handle(h)); err != nil {
+		return os.NewSyscallError("CloseHandle", err)
+	}
+
+	// Convert to a byte array.
+	db.data = ((*[maxMapSize]byte)(unsafe.Pointer(addr)))
+	db.datasz = sz
+
+	return nil
+}
+
+// munmap unmaps a pointer from a file.
+// Based on: https://github.com/edsrzf/mmap-go
+func munmap(db *DB) error {
+	if db.data == nil {
+		return nil
+	}
+
+	addr := (uintptr)(unsafe.Pointer(&db.data[0]))
+	if err := syscall.UnmapViewOfFile(addr); err != nil {
+		return os.NewSyscallError("UnmapViewOfFile", err)
+	}
+	return nil
+}
diff --git a/vendor/go.etcd.io/bbolt/boltsync_unix.go b/vendor/go.etcd.io/bbolt/boltsync_unix.go
new file mode 100644
index 0000000..9587afe
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/boltsync_unix.go
@@ -0,0 +1,8 @@
+// +build !windows,!plan9,!linux,!openbsd
+
+package bbolt
+
+// fdatasync flushes written data to a file descriptor.
+func fdatasync(db *DB) error {
+	return db.file.Sync()
+}
diff --git a/vendor/go.etcd.io/bbolt/bucket.go b/vendor/go.etcd.io/bbolt/bucket.go
new file mode 100644
index 0000000..84bfd4d
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/bucket.go
@@ -0,0 +1,775 @@
+package bbolt
+
+import (
+	"bytes"
+	"fmt"
+	"unsafe"
+)
+
+const (
+	// MaxKeySize is the maximum length of a key, in bytes.
+	MaxKeySize = 32768
+
+	// MaxValueSize is the maximum length of a value, in bytes.
+	MaxValueSize = (1 << 31) - 2
+)
+
+const bucketHeaderSize = int(unsafe.Sizeof(bucket{}))
+
+const (
+	minFillPercent = 0.1
+	maxFillPercent = 1.0
+)
+
+// DefaultFillPercent is the percentage that split pages are filled.
+// This value can be changed by setting Bucket.FillPercent.
+const DefaultFillPercent = 0.5
+
+// Bucket represents a collection of key/value pairs inside the database.
+type Bucket struct {
+	*bucket
+	tx       *Tx                // the associated transaction
+	buckets  map[string]*Bucket // subbucket cache
+	page     *page              // inline page reference
+	rootNode *node              // materialized node for the root page.
+	nodes    map[pgid]*node     // node cache
+
+	// Sets the threshold for filling nodes when they split. By default,
+	// the bucket will fill to 50% but it can be useful to increase this
+	// amount if you know that your write workloads are mostly append-only.
+	//
+	// This is non-persisted across transactions so it must be set in every Tx.
+	FillPercent float64
+}
+
+// bucket represents the on-file representation of a bucket.
+// This is stored as the "value" of a bucket key. If the bucket is small enough,
+// then its root page can be stored inline in the "value", after the bucket
+// header. In the case of inline buckets, the "root" will be 0.
+type bucket struct {
+	root     pgid   // page id of the bucket's root-level page
+	sequence uint64 // monotonically incrementing, used by NextSequence()
+}
+
+// newBucket returns a new bucket associated with a transaction.
+func newBucket(tx *Tx) Bucket {
+	var b = Bucket{tx: tx, FillPercent: DefaultFillPercent}
+	if tx.writable {
+		b.buckets = make(map[string]*Bucket)
+		b.nodes = make(map[pgid]*node)
+	}
+	return b
+}
+
+// Tx returns the tx of the bucket.
+func (b *Bucket) Tx() *Tx {
+	return b.tx
+}
+
+// Root returns the root of the bucket.
+func (b *Bucket) Root() pgid {
+	return b.root
+}
+
+// Writable returns whether the bucket is writable.
+func (b *Bucket) Writable() bool {
+	return b.tx.writable
+}
+
+// Cursor creates a cursor associated with the bucket.
+// The cursor is only valid as long as the transaction is open.
+// Do not use a cursor after the transaction is closed.
+func (b *Bucket) Cursor() *Cursor {
+	// Update transaction statistics.
+	b.tx.stats.CursorCount++
+
+	// Allocate and return a cursor.
+	return &Cursor{
+		bucket: b,
+		stack:  make([]elemRef, 0),
+	}
+}
+
+// Bucket retrieves a nested bucket by name.
+// Returns nil if the bucket does not exist.
+// The bucket instance is only valid for the lifetime of the transaction.
+func (b *Bucket) Bucket(name []byte) *Bucket {
+	if b.buckets != nil {
+		if child := b.buckets[string(name)]; child != nil {
+			return child
+		}
+	}
+
+	// Move cursor to key.
+	c := b.Cursor()
+	k, v, flags := c.seek(name)
+
+	// Return nil if the key doesn't exist or it is not a bucket.
+	if !bytes.Equal(name, k) || (flags&bucketLeafFlag) == 0 {
+		return nil
+	}
+
+	// Otherwise create a bucket and cache it.
+	var child = b.openBucket(v)
+	if b.buckets != nil {
+		b.buckets[string(name)] = child
+	}
+
+	return child
+}
+
+// Helper method that re-interprets a sub-bucket value
+// from a parent into a Bucket
+func (b *Bucket) openBucket(value []byte) *Bucket {
+	var child = newBucket(b.tx)
+
+	// If unaligned load/stores are broken on this arch and value is
+	// unaligned simply clone to an aligned byte array.
+	unaligned := brokenUnaligned && uintptr(unsafe.Pointer(&value[0]))&3 != 0
+
+	if unaligned {
+		value = cloneBytes(value)
+	}
+
+	// If this is a writable transaction then we need to copy the bucket entry.
+	// Read-only transactions can point directly at the mmap entry.
+	if b.tx.writable && !unaligned {
+		child.bucket = &bucket{}
+		*child.bucket = *(*bucket)(unsafe.Pointer(&value[0]))
+	} else {
+		child.bucket = (*bucket)(unsafe.Pointer(&value[0]))
+	}
+
+	// Save a reference to the inline page if the bucket is inline.
+	if child.root == 0 {
+		child.page = (*page)(unsafe.Pointer(&value[bucketHeaderSize]))
+	}
+
+	return &child
+}
+
+// CreateBucket creates a new bucket at the given key and returns the new bucket.
+// Returns an error if the key already exists, if the bucket name is blank, or if the bucket name is too long.
+// The bucket instance is only valid for the lifetime of the transaction.
+func (b *Bucket) CreateBucket(key []byte) (*Bucket, error) {
+	if b.tx.db == nil {
+		return nil, ErrTxClosed
+	} else if !b.tx.writable {
+		return nil, ErrTxNotWritable
+	} else if len(key) == 0 {
+		return nil, ErrBucketNameRequired
+	}
+
+	// Move cursor to correct position.
+	c := b.Cursor()
+	k, _, flags := c.seek(key)
+
+	// Return an error if there is an existing key.
+	if bytes.Equal(key, k) {
+		if (flags & bucketLeafFlag) != 0 {
+			return nil, ErrBucketExists
+		}
+		return nil, ErrIncompatibleValue
+	}
+
+	// Create empty, inline bucket.
+	var bucket = Bucket{
+		bucket:      &bucket{},
+		rootNode:    &node{isLeaf: true},
+		FillPercent: DefaultFillPercent,
+	}
+	var value = bucket.write()
+
+	// Insert into node.
+	key = cloneBytes(key)
+	c.node().put(key, key, value, 0, bucketLeafFlag)
+
+	// Since subbuckets are not allowed on inline buckets, we need to
+	// dereference the inline page, if it exists. This will cause the bucket
+	// to be treated as a regular, non-inline bucket for the rest of the tx.
+	b.page = nil
+
+	return b.Bucket(key), nil
+}
+
+// CreateBucketIfNotExists creates a new bucket if it doesn't already exist and returns a reference to it.
+// Returns an error if the bucket name is blank, or if the bucket name is too long.
+// The bucket instance is only valid for the lifetime of the transaction.
+func (b *Bucket) CreateBucketIfNotExists(key []byte) (*Bucket, error) {
+	child, err := b.CreateBucket(key)
+	if err == ErrBucketExists {
+		return b.Bucket(key), nil
+	} else if err != nil {
+		return nil, err
+	}
+	return child, nil
+}
+
+// DeleteBucket deletes a bucket at the given key.
+// Returns an error if the bucket does not exists, or if the key represents a non-bucket value.
+func (b *Bucket) DeleteBucket(key []byte) error {
+	if b.tx.db == nil {
+		return ErrTxClosed
+	} else if !b.Writable() {
+		return ErrTxNotWritable
+	}
+
+	// Move cursor to correct position.
+	c := b.Cursor()
+	k, _, flags := c.seek(key)
+
+	// Return an error if bucket doesn't exist or is not a bucket.
+	if !bytes.Equal(key, k) {
+		return ErrBucketNotFound
+	} else if (flags & bucketLeafFlag) == 0 {
+		return ErrIncompatibleValue
+	}
+
+	// Recursively delete all child buckets.
+	child := b.Bucket(key)
+	err := child.ForEach(func(k, v []byte) error {
+		if v == nil {
+			if err := child.DeleteBucket(k); err != nil {
+				return fmt.Errorf("delete bucket: %s", err)
+			}
+		}
+		return nil
+	})
+	if err != nil {
+		return err
+	}
+
+	// Remove cached copy.
+	delete(b.buckets, string(key))
+
+	// Release all bucket pages to freelist.
+	child.nodes = nil
+	child.rootNode = nil
+	child.free()
+
+	// Delete the node if we have a matching key.
+	c.node().del(key)
+
+	return nil
+}
+
+// Get retrieves the value for a key in the bucket.
+// Returns a nil value if the key does not exist or if the key is a nested bucket.
+// The returned value is only valid for the life of the transaction.
+func (b *Bucket) Get(key []byte) []byte {
+	k, v, flags := b.Cursor().seek(key)
+
+	// Return nil if this is a bucket.
+	if (flags & bucketLeafFlag) != 0 {
+		return nil
+	}
+
+	// If our target node isn't the same key as what's passed in then return nil.
+	if !bytes.Equal(key, k) {
+		return nil
+	}
+	return v
+}
+
+// Put sets the value for a key in the bucket.
+// If the key exist then its previous value will be overwritten.
+// Supplied value must remain valid for the life of the transaction.
+// Returns an error if the bucket was created from a read-only transaction, if the key is blank, if the key is too large, or if the value is too large.
+func (b *Bucket) Put(key []byte, value []byte) error {
+	if b.tx.db == nil {
+		return ErrTxClosed
+	} else if !b.Writable() {
+		return ErrTxNotWritable
+	} else if len(key) == 0 {
+		return ErrKeyRequired
+	} else if len(key) > MaxKeySize {
+		return ErrKeyTooLarge
+	} else if int64(len(value)) > MaxValueSize {
+		return ErrValueTooLarge
+	}
+
+	// Move cursor to correct position.
+	c := b.Cursor()
+	k, _, flags := c.seek(key)
+
+	// Return an error if there is an existing key with a bucket value.
+	if bytes.Equal(key, k) && (flags&bucketLeafFlag) != 0 {
+		return ErrIncompatibleValue
+	}
+
+	// Insert into node.
+	key = cloneBytes(key)
+	c.node().put(key, key, value, 0, 0)
+
+	return nil
+}
+
+// Delete removes a key from the bucket.
+// If the key does not exist then nothing is done and a nil error is returned.
+// Returns an error if the bucket was created from a read-only transaction.
+func (b *Bucket) Delete(key []byte) error {
+	if b.tx.db == nil {
+		return ErrTxClosed
+	} else if !b.Writable() {
+		return ErrTxNotWritable
+	}
+
+	// Move cursor to correct position.
+	c := b.Cursor()
+	k, _, flags := c.seek(key)
+
+	// Return nil if the key doesn't exist.
+	if !bytes.Equal(key, k) {
+		return nil
+	}
+
+	// Return an error if there is already existing bucket value.
+	if (flags & bucketLeafFlag) != 0 {
+		return ErrIncompatibleValue
+	}
+
+	// Delete the node if we have a matching key.
+	c.node().del(key)
+
+	return nil
+}
+
+// Sequence returns the current integer for the bucket without incrementing it.
+func (b *Bucket) Sequence() uint64 { return b.bucket.sequence }
+
+// SetSequence updates the sequence number for the bucket.
+func (b *Bucket) SetSequence(v uint64) error {
+	if b.tx.db == nil {
+		return ErrTxClosed
+	} else if !b.Writable() {
+		return ErrTxNotWritable
+	}
+
+	// Materialize the root node if it hasn't been already so that the
+	// bucket will be saved during commit.
+	if b.rootNode == nil {
+		_ = b.node(b.root, nil)
+	}
+
+	// Increment and return the sequence.
+	b.bucket.sequence = v
+	return nil
+}
+
+// NextSequence returns an autoincrementing integer for the bucket.
+func (b *Bucket) NextSequence() (uint64, error) {
+	if b.tx.db == nil {
+		return 0, ErrTxClosed
+	} else if !b.Writable() {
+		return 0, ErrTxNotWritable
+	}
+
+	// Materialize the root node if it hasn't been already so that the
+	// bucket will be saved during commit.
+	if b.rootNode == nil {
+		_ = b.node(b.root, nil)
+	}
+
+	// Increment and return the sequence.
+	b.bucket.sequence++
+	return b.bucket.sequence, nil
+}
+
+// ForEach executes a function for each key/value pair in a bucket.
+// If the provided function returns an error then the iteration is stopped and
+// the error is returned to the caller. The provided function must not modify
+// the bucket; this will result in undefined behavior.
+func (b *Bucket) ForEach(fn func(k, v []byte) error) error {
+	if b.tx.db == nil {
+		return ErrTxClosed
+	}
+	c := b.Cursor()
+	for k, v := c.First(); k != nil; k, v = c.Next() {
+		if err := fn(k, v); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+// Stat returns stats on a bucket.
+func (b *Bucket) Stats() BucketStats {
+	var s, subStats BucketStats
+	pageSize := b.tx.db.pageSize
+	s.BucketN += 1
+	if b.root == 0 {
+		s.InlineBucketN += 1
+	}
+	b.forEachPage(func(p *page, depth int) {
+		if (p.flags & leafPageFlag) != 0 {
+			s.KeyN += int(p.count)
+
+			// used totals the used bytes for the page
+			used := pageHeaderSize
+
+			if p.count != 0 {
+				// If page has any elements, add all element headers.
+				used += leafPageElementSize * int(p.count-1)
+
+				// Add all element key, value sizes.
+				// The computation takes advantage of the fact that the position
+				// of the last element's key/value equals to the total of the sizes
+				// of all previous elements' keys and values.
+				// It also includes the last element's header.
+				lastElement := p.leafPageElement(p.count - 1)
+				used += int(lastElement.pos + lastElement.ksize + lastElement.vsize)
+			}
+
+			if b.root == 0 {
+				// For inlined bucket just update the inline stats
+				s.InlineBucketInuse += used
+			} else {
+				// For non-inlined bucket update all the leaf stats
+				s.LeafPageN++
+				s.LeafInuse += used
+				s.LeafOverflowN += int(p.overflow)
+
+				// Collect stats from sub-buckets.
+				// Do that by iterating over all element headers
+				// looking for the ones with the bucketLeafFlag.
+				for i := uint16(0); i < p.count; i++ {
+					e := p.leafPageElement(i)
+					if (e.flags & bucketLeafFlag) != 0 {
+						// For any bucket element, open the element value
+						// and recursively call Stats on the contained bucket.
+						subStats.Add(b.openBucket(e.value()).Stats())
+					}
+				}
+			}
+		} else if (p.flags & branchPageFlag) != 0 {
+			s.BranchPageN++
+			lastElement := p.branchPageElement(p.count - 1)
+
+			// used totals the used bytes for the page
+			// Add header and all element headers.
+			used := pageHeaderSize + (branchPageElementSize * int(p.count-1))
+
+			// Add size of all keys and values.
+			// Again, use the fact that last element's position equals to
+			// the total of key, value sizes of all previous elements.
+			used += int(lastElement.pos + lastElement.ksize)
+			s.BranchInuse += used
+			s.BranchOverflowN += int(p.overflow)
+		}
+
+		// Keep track of maximum page depth.
+		if depth+1 > s.Depth {
+			s.Depth = (depth + 1)
+		}
+	})
+
+	// Alloc stats can be computed from page counts and pageSize.
+	s.BranchAlloc = (s.BranchPageN + s.BranchOverflowN) * pageSize
+	s.LeafAlloc = (s.LeafPageN + s.LeafOverflowN) * pageSize
+
+	// Add the max depth of sub-buckets to get total nested depth.
+	s.Depth += subStats.Depth
+	// Add the stats for all sub-buckets
+	s.Add(subStats)
+	return s
+}
+
+// forEachPage iterates over every page in a bucket, including inline pages.
+func (b *Bucket) forEachPage(fn func(*page, int)) {
+	// If we have an inline page then just use that.
+	if b.page != nil {
+		fn(b.page, 0)
+		return
+	}
+
+	// Otherwise traverse the page hierarchy.
+	b.tx.forEachPage(b.root, 0, fn)
+}
+
+// forEachPageNode iterates over every page (or node) in a bucket.
+// This also includes inline pages.
+func (b *Bucket) forEachPageNode(fn func(*page, *node, int)) {
+	// If we have an inline page or root node then just use that.
+	if b.page != nil {
+		fn(b.page, nil, 0)
+		return
+	}
+	b._forEachPageNode(b.root, 0, fn)
+}
+
+func (b *Bucket) _forEachPageNode(pgid pgid, depth int, fn func(*page, *node, int)) {
+	var p, n = b.pageNode(pgid)
+
+	// Execute function.
+	fn(p, n, depth)
+
+	// Recursively loop over children.
+	if p != nil {
+		if (p.flags & branchPageFlag) != 0 {
+			for i := 0; i < int(p.count); i++ {
+				elem := p.branchPageElement(uint16(i))
+				b._forEachPageNode(elem.pgid, depth+1, fn)
+			}
+		}
+	} else {
+		if !n.isLeaf {
+			for _, inode := range n.inodes {
+				b._forEachPageNode(inode.pgid, depth+1, fn)
+			}
+		}
+	}
+}
+
+// spill writes all the nodes for this bucket to dirty pages.
+func (b *Bucket) spill() error {
+	// Spill all child buckets first.
+	for name, child := range b.buckets {
+		// If the child bucket is small enough and it has no child buckets then
+		// write it inline into the parent bucket's page. Otherwise spill it
+		// like a normal bucket and make the parent value a pointer to the page.
+		var value []byte
+		if child.inlineable() {
+			child.free()
+			value = child.write()
+		} else {
+			if err := child.spill(); err != nil {
+				return err
+			}
+
+			// Update the child bucket header in this bucket.
+			value = make([]byte, unsafe.Sizeof(bucket{}))
+			var bucket = (*bucket)(unsafe.Pointer(&value[0]))
+			*bucket = *child.bucket
+		}
+
+		// Skip writing the bucket if there are no materialized nodes.
+		if child.rootNode == nil {
+			continue
+		}
+
+		// Update parent node.
+		var c = b.Cursor()
+		k, _, flags := c.seek([]byte(name))
+		if !bytes.Equal([]byte(name), k) {
+			panic(fmt.Sprintf("misplaced bucket header: %x -> %x", []byte(name), k))
+		}
+		if flags&bucketLeafFlag == 0 {
+			panic(fmt.Sprintf("unexpected bucket header flag: %x", flags))
+		}
+		c.node().put([]byte(name), []byte(name), value, 0, bucketLeafFlag)
+	}
+
+	// Ignore if there's not a materialized root node.
+	if b.rootNode == nil {
+		return nil
+	}
+
+	// Spill nodes.
+	if err := b.rootNode.spill(); err != nil {
+		return err
+	}
+	b.rootNode = b.rootNode.root()
+
+	// Update the root node for this bucket.
+	if b.rootNode.pgid >= b.tx.meta.pgid {
+		panic(fmt.Sprintf("pgid (%d) above high water mark (%d)", b.rootNode.pgid, b.tx.meta.pgid))
+	}
+	b.root = b.rootNode.pgid
+
+	return nil
+}
+
+// inlineable returns true if a bucket is small enough to be written inline
+// and if it contains no subbuckets. Otherwise returns false.
+func (b *Bucket) inlineable() bool {
+	var n = b.rootNode
+
+	// Bucket must only contain a single leaf node.
+	if n == nil || !n.isLeaf {
+		return false
+	}
+
+	// Bucket is not inlineable if it contains subbuckets or if it goes beyond
+	// our threshold for inline bucket size.
+	var size = pageHeaderSize
+	for _, inode := range n.inodes {
+		size += leafPageElementSize + len(inode.key) + len(inode.value)
+
+		if inode.flags&bucketLeafFlag != 0 {
+			return false
+		} else if size > b.maxInlineBucketSize() {
+			return false
+		}
+	}
+
+	return true
+}
+
+// Returns the maximum total size of a bucket to make it a candidate for inlining.
+func (b *Bucket) maxInlineBucketSize() int {
+	return b.tx.db.pageSize / 4
+}
+
+// write allocates and writes a bucket to a byte slice.
+func (b *Bucket) write() []byte {
+	// Allocate the appropriate size.
+	var n = b.rootNode
+	var value = make([]byte, bucketHeaderSize+n.size())
+
+	// Write a bucket header.
+	var bucket = (*bucket)(unsafe.Pointer(&value[0]))
+	*bucket = *b.bucket
+
+	// Convert byte slice to a fake page and write the root node.
+	var p = (*page)(unsafe.Pointer(&value[bucketHeaderSize]))
+	n.write(p)
+
+	return value
+}
+
+// rebalance attempts to balance all nodes.
+func (b *Bucket) rebalance() {
+	for _, n := range b.nodes {
+		n.rebalance()
+	}
+	for _, child := range b.buckets {
+		child.rebalance()
+	}
+}
+
+// node creates a node from a page and associates it with a given parent.
+func (b *Bucket) node(pgid pgid, parent *node) *node {
+	_assert(b.nodes != nil, "nodes map expected")
+
+	// Retrieve node if it's already been created.
+	if n := b.nodes[pgid]; n != nil {
+		return n
+	}
+
+	// Otherwise create a node and cache it.
+	n := &node{bucket: b, parent: parent}
+	if parent == nil {
+		b.rootNode = n
+	} else {
+		parent.children = append(parent.children, n)
+	}
+
+	// Use the inline page if this is an inline bucket.
+	var p = b.page
+	if p == nil {
+		p = b.tx.page(pgid)
+	}
+
+	// Read the page into the node and cache it.
+	n.read(p)
+	b.nodes[pgid] = n
+
+	// Update statistics.
+	b.tx.stats.NodeCount++
+
+	return n
+}
+
+// free recursively frees all pages in the bucket.
+func (b *Bucket) free() {
+	if b.root == 0 {
+		return
+	}
+
+	var tx = b.tx
+	b.forEachPageNode(func(p *page, n *node, _ int) {
+		if p != nil {
+			tx.db.freelist.free(tx.meta.txid, p)
+		} else {
+			n.free()
+		}
+	})
+	b.root = 0
+}
+
+// dereference removes all references to the old mmap.
+func (b *Bucket) dereference() {
+	if b.rootNode != nil {
+		b.rootNode.root().dereference()
+	}
+
+	for _, child := range b.buckets {
+		child.dereference()
+	}
+}
+
+// pageNode returns the in-memory node, if it exists.
+// Otherwise returns the underlying page.
+func (b *Bucket) pageNode(id pgid) (*page, *node) {
+	// Inline buckets have a fake page embedded in their value so treat them
+	// differently. We'll return the rootNode (if available) or the fake page.
+	if b.root == 0 {
+		if id != 0 {
+			panic(fmt.Sprintf("inline bucket non-zero page access(2): %d != 0", id))
+		}
+		if b.rootNode != nil {
+			return nil, b.rootNode
+		}
+		return b.page, nil
+	}
+
+	// Check the node cache for non-inline buckets.
+	if b.nodes != nil {
+		if n := b.nodes[id]; n != nil {
+			return nil, n
+		}
+	}
+
+	// Finally lookup the page from the transaction if no node is materialized.
+	return b.tx.page(id), nil
+}
+
+// BucketStats records statistics about resources used by a bucket.
+type BucketStats struct {
+	// Page count statistics.
+	BranchPageN     int // number of logical branch pages
+	BranchOverflowN int // number of physical branch overflow pages
+	LeafPageN       int // number of logical leaf pages
+	LeafOverflowN   int // number of physical leaf overflow pages
+
+	// Tree statistics.
+	KeyN  int // number of keys/value pairs
+	Depth int // number of levels in B+tree
+
+	// Page size utilization.
+	BranchAlloc int // bytes allocated for physical branch pages
+	BranchInuse int // bytes actually used for branch data
+	LeafAlloc   int // bytes allocated for physical leaf pages
+	LeafInuse   int // bytes actually used for leaf data
+
+	// Bucket statistics
+	BucketN           int // total number of buckets including the top bucket
+	InlineBucketN     int // total number on inlined buckets
+	InlineBucketInuse int // bytes used for inlined buckets (also accounted for in LeafInuse)
+}
+
+func (s *BucketStats) Add(other BucketStats) {
+	s.BranchPageN += other.BranchPageN
+	s.BranchOverflowN += other.BranchOverflowN
+	s.LeafPageN += other.LeafPageN
+	s.LeafOverflowN += other.LeafOverflowN
+	s.KeyN += other.KeyN
+	if s.Depth < other.Depth {
+		s.Depth = other.Depth
+	}
+	s.BranchAlloc += other.BranchAlloc
+	s.BranchInuse += other.BranchInuse
+	s.LeafAlloc += other.LeafAlloc
+	s.LeafInuse += other.LeafInuse
+
+	s.BucketN += other.BucketN
+	s.InlineBucketN += other.InlineBucketN
+	s.InlineBucketInuse += other.InlineBucketInuse
+}
+
+// cloneBytes returns a copy of a given slice.
+func cloneBytes(v []byte) []byte {
+	var clone = make([]byte, len(v))
+	copy(clone, v)
+	return clone
+}
diff --git a/vendor/go.etcd.io/bbolt/cursor.go b/vendor/go.etcd.io/bbolt/cursor.go
new file mode 100644
index 0000000..3000ace
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/cursor.go
@@ -0,0 +1,396 @@
+package bbolt
+
+import (
+	"bytes"
+	"fmt"
+	"sort"
+)
+
+// Cursor represents an iterator that can traverse over all key/value pairs in a bucket in sorted order.
+// Cursors see nested buckets with value == nil.
+// Cursors can be obtained from a transaction and are valid as long as the transaction is open.
+//
+// Keys and values returned from the cursor are only valid for the life of the transaction.
+//
+// Changing data while traversing with a cursor may cause it to be invalidated
+// and return unexpected keys and/or values. You must reposition your cursor
+// after mutating data.
+type Cursor struct {
+	bucket *Bucket
+	stack  []elemRef
+}
+
+// Bucket returns the bucket that this cursor was created from.
+func (c *Cursor) Bucket() *Bucket {
+	return c.bucket
+}
+
+// First moves the cursor to the first item in the bucket and returns its key and value.
+// If the bucket is empty then a nil key and value are returned.
+// The returned key and value are only valid for the life of the transaction.
+func (c *Cursor) First() (key []byte, value []byte) {
+	_assert(c.bucket.tx.db != nil, "tx closed")
+	c.stack = c.stack[:0]
+	p, n := c.bucket.pageNode(c.bucket.root)
+	c.stack = append(c.stack, elemRef{page: p, node: n, index: 0})
+	c.first()
+
+	// If we land on an empty page then move to the next value.
+	// https://github.com/boltdb/bolt/issues/450
+	if c.stack[len(c.stack)-1].count() == 0 {
+		c.next()
+	}
+
+	k, v, flags := c.keyValue()
+	if (flags & uint32(bucketLeafFlag)) != 0 {
+		return k, nil
+	}
+	return k, v
+
+}
+
+// Last moves the cursor to the last item in the bucket and returns its key and value.
+// If the bucket is empty then a nil key and value are returned.
+// The returned key and value are only valid for the life of the transaction.
+func (c *Cursor) Last() (key []byte, value []byte) {
+	_assert(c.bucket.tx.db != nil, "tx closed")
+	c.stack = c.stack[:0]
+	p, n := c.bucket.pageNode(c.bucket.root)
+	ref := elemRef{page: p, node: n}
+	ref.index = ref.count() - 1
+	c.stack = append(c.stack, ref)
+	c.last()
+	k, v, flags := c.keyValue()
+	if (flags & uint32(bucketLeafFlag)) != 0 {
+		return k, nil
+	}
+	return k, v
+}
+
+// Next moves the cursor to the next item in the bucket and returns its key and value.
+// If the cursor is at the end of the bucket then a nil key and value are returned.
+// The returned key and value are only valid for the life of the transaction.
+func (c *Cursor) Next() (key []byte, value []byte) {
+	_assert(c.bucket.tx.db != nil, "tx closed")
+	k, v, flags := c.next()
+	if (flags & uint32(bucketLeafFlag)) != 0 {
+		return k, nil
+	}
+	return k, v
+}
+
+// Prev moves the cursor to the previous item in the bucket and returns its key and value.
+// If the cursor is at the beginning of the bucket then a nil key and value are returned.
+// The returned key and value are only valid for the life of the transaction.
+func (c *Cursor) Prev() (key []byte, value []byte) {
+	_assert(c.bucket.tx.db != nil, "tx closed")
+
+	// Attempt to move back one element until we're successful.
+	// Move up the stack as we hit the beginning of each page in our stack.
+	for i := len(c.stack) - 1; i >= 0; i-- {
+		elem := &c.stack[i]
+		if elem.index > 0 {
+			elem.index--
+			break
+		}
+		c.stack = c.stack[:i]
+	}
+
+	// If we've hit the end then return nil.
+	if len(c.stack) == 0 {
+		return nil, nil
+	}
+
+	// Move down the stack to find the last element of the last leaf under this branch.
+	c.last()
+	k, v, flags := c.keyValue()
+	if (flags & uint32(bucketLeafFlag)) != 0 {
+		return k, nil
+	}
+	return k, v
+}
+
+// Seek moves the cursor to a given key and returns it.
+// If the key does not exist then the next key is used. If no keys
+// follow, a nil key is returned.
+// The returned key and value are only valid for the life of the transaction.
+func (c *Cursor) Seek(seek []byte) (key []byte, value []byte) {
+	k, v, flags := c.seek(seek)
+
+	// If we ended up after the last element of a page then move to the next one.
+	if ref := &c.stack[len(c.stack)-1]; ref.index >= ref.count() {
+		k, v, flags = c.next()
+	}
+
+	if k == nil {
+		return nil, nil
+	} else if (flags & uint32(bucketLeafFlag)) != 0 {
+		return k, nil
+	}
+	return k, v
+}
+
+// Delete removes the current key/value under the cursor from the bucket.
+// Delete fails if current key/value is a bucket or if the transaction is not writable.
+func (c *Cursor) Delete() error {
+	if c.bucket.tx.db == nil {
+		return ErrTxClosed
+	} else if !c.bucket.Writable() {
+		return ErrTxNotWritable
+	}
+
+	key, _, flags := c.keyValue()
+	// Return an error if current value is a bucket.
+	if (flags & bucketLeafFlag) != 0 {
+		return ErrIncompatibleValue
+	}
+	c.node().del(key)
+
+	return nil
+}
+
+// seek moves the cursor to a given key and returns it.
+// If the key does not exist then the next key is used.
+func (c *Cursor) seek(seek []byte) (key []byte, value []byte, flags uint32) {
+	_assert(c.bucket.tx.db != nil, "tx closed")
+
+	// Start from root page/node and traverse to correct page.
+	c.stack = c.stack[:0]
+	c.search(seek, c.bucket.root)
+
+	// If this is a bucket then return a nil value.
+	return c.keyValue()
+}
+
+// first moves the cursor to the first leaf element under the last page in the stack.
+func (c *Cursor) first() {
+	for {
+		// Exit when we hit a leaf page.
+		var ref = &c.stack[len(c.stack)-1]
+		if ref.isLeaf() {
+			break
+		}
+
+		// Keep adding pages pointing to the first element to the stack.
+		var pgid pgid
+		if ref.node != nil {
+			pgid = ref.node.inodes[ref.index].pgid
+		} else {
+			pgid = ref.page.branchPageElement(uint16(ref.index)).pgid
+		}
+		p, n := c.bucket.pageNode(pgid)
+		c.stack = append(c.stack, elemRef{page: p, node: n, index: 0})
+	}
+}
+
+// last moves the cursor to the last leaf element under the last page in the stack.
+func (c *Cursor) last() {
+	for {
+		// Exit when we hit a leaf page.
+		ref := &c.stack[len(c.stack)-1]
+		if ref.isLeaf() {
+			break
+		}
+
+		// Keep adding pages pointing to the last element in the stack.
+		var pgid pgid
+		if ref.node != nil {
+			pgid = ref.node.inodes[ref.index].pgid
+		} else {
+			pgid = ref.page.branchPageElement(uint16(ref.index)).pgid
+		}
+		p, n := c.bucket.pageNode(pgid)
+
+		var nextRef = elemRef{page: p, node: n}
+		nextRef.index = nextRef.count() - 1
+		c.stack = append(c.stack, nextRef)
+	}
+}
+
+// next moves to the next leaf element and returns the key and value.
+// If the cursor is at the last leaf element then it stays there and returns nil.
+func (c *Cursor) next() (key []byte, value []byte, flags uint32) {
+	for {
+		// Attempt to move over one element until we're successful.
+		// Move up the stack as we hit the end of each page in our stack.
+		var i int
+		for i = len(c.stack) - 1; i >= 0; i-- {
+			elem := &c.stack[i]
+			if elem.index < elem.count()-1 {
+				elem.index++
+				break
+			}
+		}
+
+		// If we've hit the root page then stop and return. This will leave the
+		// cursor on the last element of the last page.
+		if i == -1 {
+			return nil, nil, 0
+		}
+
+		// Otherwise start from where we left off in the stack and find the
+		// first element of the first leaf page.
+		c.stack = c.stack[:i+1]
+		c.first()
+
+		// If this is an empty page then restart and move back up the stack.
+		// https://github.com/boltdb/bolt/issues/450
+		if c.stack[len(c.stack)-1].count() == 0 {
+			continue
+		}
+
+		return c.keyValue()
+	}
+}
+
+// search recursively performs a binary search against a given page/node until it finds a given key.
+func (c *Cursor) search(key []byte, pgid pgid) {
+	p, n := c.bucket.pageNode(pgid)
+	if p != nil && (p.flags&(branchPageFlag|leafPageFlag)) == 0 {
+		panic(fmt.Sprintf("invalid page type: %d: %x", p.id, p.flags))
+	}
+	e := elemRef{page: p, node: n}
+	c.stack = append(c.stack, e)
+
+	// If we're on a leaf page/node then find the specific node.
+	if e.isLeaf() {
+		c.nsearch(key)
+		return
+	}
+
+	if n != nil {
+		c.searchNode(key, n)
+		return
+	}
+	c.searchPage(key, p)
+}
+
+func (c *Cursor) searchNode(key []byte, n *node) {
+	var exact bool
+	index := sort.Search(len(n.inodes), func(i int) bool {
+		// TODO(benbjohnson): Optimize this range search. It's a bit hacky right now.
+		// sort.Search() finds the lowest index where f() != -1 but we need the highest index.
+		ret := bytes.Compare(n.inodes[i].key, key)
+		if ret == 0 {
+			exact = true
+		}
+		return ret != -1
+	})
+	if !exact && index > 0 {
+		index--
+	}
+	c.stack[len(c.stack)-1].index = index
+
+	// Recursively search to the next page.
+	c.search(key, n.inodes[index].pgid)
+}
+
+func (c *Cursor) searchPage(key []byte, p *page) {
+	// Binary search for the correct range.
+	inodes := p.branchPageElements()
+
+	var exact bool
+	index := sort.Search(int(p.count), func(i int) bool {
+		// TODO(benbjohnson): Optimize this range search. It's a bit hacky right now.
+		// sort.Search() finds the lowest index where f() != -1 but we need the highest index.
+		ret := bytes.Compare(inodes[i].key(), key)
+		if ret == 0 {
+			exact = true
+		}
+		return ret != -1
+	})
+	if !exact && index > 0 {
+		index--
+	}
+	c.stack[len(c.stack)-1].index = index
+
+	// Recursively search to the next page.
+	c.search(key, inodes[index].pgid)
+}
+
+// nsearch searches the leaf node on the top of the stack for a key.
+func (c *Cursor) nsearch(key []byte) {
+	e := &c.stack[len(c.stack)-1]
+	p, n := e.page, e.node
+
+	// If we have a node then search its inodes.
+	if n != nil {
+		index := sort.Search(len(n.inodes), func(i int) bool {
+			return bytes.Compare(n.inodes[i].key, key) != -1
+		})
+		e.index = index
+		return
+	}
+
+	// If we have a page then search its leaf elements.
+	inodes := p.leafPageElements()
+	index := sort.Search(int(p.count), func(i int) bool {
+		return bytes.Compare(inodes[i].key(), key) != -1
+	})
+	e.index = index
+}
+
+// keyValue returns the key and value of the current leaf element.
+func (c *Cursor) keyValue() ([]byte, []byte, uint32) {
+	ref := &c.stack[len(c.stack)-1]
+
+	// If the cursor is pointing to the end of page/node then return nil.
+	if ref.count() == 0 || ref.index >= ref.count() {
+		return nil, nil, 0
+	}
+
+	// Retrieve value from node.
+	if ref.node != nil {
+		inode := &ref.node.inodes[ref.index]
+		return inode.key, inode.value, inode.flags
+	}
+
+	// Or retrieve value from page.
+	elem := ref.page.leafPageElement(uint16(ref.index))
+	return elem.key(), elem.value(), elem.flags
+}
+
+// node returns the node that the cursor is currently positioned on.
+func (c *Cursor) node() *node {
+	_assert(len(c.stack) > 0, "accessing a node with a zero-length cursor stack")
+
+	// If the top of the stack is a leaf node then just return it.
+	if ref := &c.stack[len(c.stack)-1]; ref.node != nil && ref.isLeaf() {
+		return ref.node
+	}
+
+	// Start from root and traverse down the hierarchy.
+	var n = c.stack[0].node
+	if n == nil {
+		n = c.bucket.node(c.stack[0].page.id, nil)
+	}
+	for _, ref := range c.stack[:len(c.stack)-1] {
+		_assert(!n.isLeaf, "expected branch node")
+		n = n.childAt(int(ref.index))
+	}
+	_assert(n.isLeaf, "expected leaf node")
+	return n
+}
+
+// elemRef represents a reference to an element on a given page/node.
+type elemRef struct {
+	page  *page
+	node  *node
+	index int
+}
+
+// isLeaf returns whether the ref is pointing at a leaf page/node.
+func (r *elemRef) isLeaf() bool {
+	if r.node != nil {
+		return r.node.isLeaf
+	}
+	return (r.page.flags & leafPageFlag) != 0
+}
+
+// count returns the number of inodes or page elements.
+func (r *elemRef) count() int {
+	if r.node != nil {
+		return len(r.node.inodes)
+	}
+	return int(r.page.count)
+}
diff --git a/vendor/go.etcd.io/bbolt/db.go b/vendor/go.etcd.io/bbolt/db.go
new file mode 100644
index 0000000..870c8b1
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/db.go
@@ -0,0 +1,1174 @@
+package bbolt
+
+import (
+	"errors"
+	"fmt"
+	"hash/fnv"
+	"log"
+	"os"
+	"runtime"
+	"sort"
+	"sync"
+	"time"
+	"unsafe"
+)
+
+// The largest step that can be taken when remapping the mmap.
+const maxMmapStep = 1 << 30 // 1GB
+
+// The data file format version.
+const version = 2
+
+// Represents a marker value to indicate that a file is a Bolt DB.
+const magic uint32 = 0xED0CDAED
+
+const pgidNoFreelist pgid = 0xffffffffffffffff
+
+// IgnoreNoSync specifies whether the NoSync field of a DB is ignored when
+// syncing changes to a file.  This is required as some operating systems,
+// such as OpenBSD, do not have a unified buffer cache (UBC) and writes
+// must be synchronized using the msync(2) syscall.
+const IgnoreNoSync = runtime.GOOS == "openbsd"
+
+// Default values if not set in a DB instance.
+const (
+	DefaultMaxBatchSize  int = 1000
+	DefaultMaxBatchDelay     = 10 * time.Millisecond
+	DefaultAllocSize         = 16 * 1024 * 1024
+)
+
+// default page size for db is set to the OS page size.
+var defaultPageSize = os.Getpagesize()
+
+// The time elapsed between consecutive file locking attempts.
+const flockRetryTimeout = 50 * time.Millisecond
+
+// FreelistType is the type of the freelist backend
+type FreelistType string
+
+const (
+	// FreelistArrayType indicates backend freelist type is array
+	FreelistArrayType = FreelistType("array")
+	// FreelistMapType indicates backend freelist type is hashmap
+	FreelistMapType = FreelistType("hashmap")
+)
+
+// DB represents a collection of buckets persisted to a file on disk.
+// All data access is performed through transactions which can be obtained through the DB.
+// All the functions on DB will return a ErrDatabaseNotOpen if accessed before Open() is called.
+type DB struct {
+	// When enabled, the database will perform a Check() after every commit.
+	// A panic is issued if the database is in an inconsistent state. This
+	// flag has a large performance impact so it should only be used for
+	// debugging purposes.
+	StrictMode bool
+
+	// Setting the NoSync flag will cause the database to skip fsync()
+	// calls after each commit. This can be useful when bulk loading data
+	// into a database and you can restart the bulk load in the event of
+	// a system failure or database corruption. Do not set this flag for
+	// normal use.
+	//
+	// If the package global IgnoreNoSync constant is true, this value is
+	// ignored.  See the comment on that constant for more details.
+	//
+	// THIS IS UNSAFE. PLEASE USE WITH CAUTION.
+	NoSync bool
+
+	// When true, skips syncing freelist to disk. This improves the database
+	// write performance under normal operation, but requires a full database
+	// re-sync during recovery.
+	NoFreelistSync bool
+
+	// FreelistType sets the backend freelist type. There are two options. Array which is simple but endures
+	// dramatic performance degradation if database is large and framentation in freelist is common.
+	// The alternative one is using hashmap, it is faster in almost all circumstances
+	// but it doesn't guarantee that it offers the smallest page id available. In normal case it is safe.
+	// The default type is array
+	FreelistType FreelistType
+
+	// When true, skips the truncate call when growing the database.
+	// Setting this to true is only safe on non-ext3/ext4 systems.
+	// Skipping truncation avoids preallocation of hard drive space and
+	// bypasses a truncate() and fsync() syscall on remapping.
+	//
+	// https://github.com/boltdb/bolt/issues/284
+	NoGrowSync bool
+
+	// If you want to read the entire database fast, you can set MmapFlag to
+	// syscall.MAP_POPULATE on Linux 2.6.23+ for sequential read-ahead.
+	MmapFlags int
+
+	// MaxBatchSize is the maximum size of a batch. Default value is
+	// copied from DefaultMaxBatchSize in Open.
+	//
+	// If <=0, disables batching.
+	//
+	// Do not change concurrently with calls to Batch.
+	MaxBatchSize int
+
+	// MaxBatchDelay is the maximum delay before a batch starts.
+	// Default value is copied from DefaultMaxBatchDelay in Open.
+	//
+	// If <=0, effectively disables batching.
+	//
+	// Do not change concurrently with calls to Batch.
+	MaxBatchDelay time.Duration
+
+	// AllocSize is the amount of space allocated when the database
+	// needs to create new pages. This is done to amortize the cost
+	// of truncate() and fsync() when growing the data file.
+	AllocSize int
+
+	path     string
+	openFile func(string, int, os.FileMode) (*os.File, error)
+	file     *os.File
+	dataref  []byte // mmap'ed readonly, write throws SEGV
+	data     *[maxMapSize]byte
+	datasz   int
+	filesz   int // current on disk file size
+	meta0    *meta
+	meta1    *meta
+	pageSize int
+	opened   bool
+	rwtx     *Tx
+	txs      []*Tx
+	stats    Stats
+
+	freelist     *freelist
+	freelistLoad sync.Once
+
+	pagePool sync.Pool
+
+	batchMu sync.Mutex
+	batch   *batch
+
+	rwlock   sync.Mutex   // Allows only one writer at a time.
+	metalock sync.Mutex   // Protects meta page access.
+	mmaplock sync.RWMutex // Protects mmap access during remapping.
+	statlock sync.RWMutex // Protects stats access.
+
+	ops struct {
+		writeAt func(b []byte, off int64) (n int, err error)
+	}
+
+	// Read only mode.
+	// When true, Update() and Begin(true) return ErrDatabaseReadOnly immediately.
+	readOnly bool
+}
+
+// Path returns the path to currently open database file.
+func (db *DB) Path() string {
+	return db.path
+}
+
+// GoString returns the Go string representation of the database.
+func (db *DB) GoString() string {
+	return fmt.Sprintf("bolt.DB{path:%q}", db.path)
+}
+
+// String returns the string representation of the database.
+func (db *DB) String() string {
+	return fmt.Sprintf("DB<%q>", db.path)
+}
+
+// Open creates and opens a database at the given path.
+// If the file does not exist then it will be created automatically.
+// Passing in nil options will cause Bolt to open the database with the default options.
+func Open(path string, mode os.FileMode, options *Options) (*DB, error) {
+	db := &DB{
+		opened: true,
+	}
+	// Set default options if no options are provided.
+	if options == nil {
+		options = DefaultOptions
+	}
+	db.NoSync = options.NoSync
+	db.NoGrowSync = options.NoGrowSync
+	db.MmapFlags = options.MmapFlags
+	db.NoFreelistSync = options.NoFreelistSync
+	db.FreelistType = options.FreelistType
+
+	// Set default values for later DB operations.
+	db.MaxBatchSize = DefaultMaxBatchSize
+	db.MaxBatchDelay = DefaultMaxBatchDelay
+	db.AllocSize = DefaultAllocSize
+
+	flag := os.O_RDWR
+	if options.ReadOnly {
+		flag = os.O_RDONLY
+		db.readOnly = true
+	}
+
+	db.openFile = options.OpenFile
+	if db.openFile == nil {
+		db.openFile = os.OpenFile
+	}
+
+	// Open data file and separate sync handler for metadata writes.
+	db.path = path
+	var err error
+	if db.file, err = db.openFile(db.path, flag|os.O_CREATE, mode); err != nil {
+		_ = db.close()
+		return nil, err
+	}
+
+	// Lock file so that other processes using Bolt in read-write mode cannot
+	// use the database  at the same time. This would cause corruption since
+	// the two processes would write meta pages and free pages separately.
+	// The database file is locked exclusively (only one process can grab the lock)
+	// if !options.ReadOnly.
+	// The database file is locked using the shared lock (more than one process may
+	// hold a lock at the same time) otherwise (options.ReadOnly is set).
+	if err := flock(db, !db.readOnly, options.Timeout); err != nil {
+		_ = db.close()
+		return nil, err
+	}
+
+	// Default values for test hooks
+	db.ops.writeAt = db.file.WriteAt
+
+	if db.pageSize = options.PageSize; db.pageSize == 0 {
+		// Set the default page size to the OS page size.
+		db.pageSize = defaultPageSize
+	}
+
+	// Initialize the database if it doesn't exist.
+	if info, err := db.file.Stat(); err != nil {
+		_ = db.close()
+		return nil, err
+	} else if info.Size() == 0 {
+		// Initialize new files with meta pages.
+		if err := db.init(); err != nil {
+			// clean up file descriptor on initialization fail
+			_ = db.close()
+			return nil, err
+		}
+	} else {
+		// Read the first meta page to determine the page size.
+		var buf [0x1000]byte
+		// If we can't read the page size, but can read a page, assume
+		// it's the same as the OS or one given -- since that's how the
+		// page size was chosen in the first place.
+		//
+		// If the first page is invalid and this OS uses a different
+		// page size than what the database was created with then we
+		// are out of luck and cannot access the database.
+		//
+		// TODO: scan for next page
+		if bw, err := db.file.ReadAt(buf[:], 0); err == nil && bw == len(buf) {
+			if m := db.pageInBuffer(buf[:], 0).meta(); m.validate() == nil {
+				db.pageSize = int(m.pageSize)
+			}
+		} else {
+			_ = db.close()
+			return nil, ErrInvalid
+		}
+	}
+
+	// Initialize page pool.
+	db.pagePool = sync.Pool{
+		New: func() interface{} {
+			return make([]byte, db.pageSize)
+		},
+	}
+
+	// Memory map the data file.
+	if err := db.mmap(options.InitialMmapSize); err != nil {
+		_ = db.close()
+		return nil, err
+	}
+
+	if db.readOnly {
+		return db, nil
+	}
+
+	db.loadFreelist()
+
+	// Flush freelist when transitioning from no sync to sync so
+	// NoFreelistSync unaware boltdb can open the db later.
+	if !db.NoFreelistSync && !db.hasSyncedFreelist() {
+		tx, err := db.Begin(true)
+		if tx != nil {
+			err = tx.Commit()
+		}
+		if err != nil {
+			_ = db.close()
+			return nil, err
+		}
+	}
+
+	// Mark the database as opened and return.
+	return db, nil
+}
+
+// loadFreelist reads the freelist if it is synced, or reconstructs it
+// by scanning the DB if it is not synced. It assumes there are no
+// concurrent accesses being made to the freelist.
+func (db *DB) loadFreelist() {
+	db.freelistLoad.Do(func() {
+		db.freelist = newFreelist(db.FreelistType)
+		if !db.hasSyncedFreelist() {
+			// Reconstruct free list by scanning the DB.
+			db.freelist.readIDs(db.freepages())
+		} else {
+			// Read free list from freelist page.
+			db.freelist.read(db.page(db.meta().freelist))
+		}
+		db.stats.FreePageN = db.freelist.free_count()
+	})
+}
+
+func (db *DB) hasSyncedFreelist() bool {
+	return db.meta().freelist != pgidNoFreelist
+}
+
+// mmap opens the underlying memory-mapped file and initializes the meta references.
+// minsz is the minimum size that the new mmap can be.
+func (db *DB) mmap(minsz int) error {
+	db.mmaplock.Lock()
+	defer db.mmaplock.Unlock()
+
+	info, err := db.file.Stat()
+	if err != nil {
+		return fmt.Errorf("mmap stat error: %s", err)
+	} else if int(info.Size()) < db.pageSize*2 {
+		return fmt.Errorf("file size too small")
+	}
+
+	// Ensure the size is at least the minimum size.
+	var size = int(info.Size())
+	if size < minsz {
+		size = minsz
+	}
+	size, err = db.mmapSize(size)
+	if err != nil {
+		return err
+	}
+
+	// Dereference all mmap references before unmapping.
+	if db.rwtx != nil {
+		db.rwtx.root.dereference()
+	}
+
+	// Unmap existing data before continuing.
+	if err := db.munmap(); err != nil {
+		return err
+	}
+
+	// Memory-map the data file as a byte slice.
+	if err := mmap(db, size); err != nil {
+		return err
+	}
+
+	// Save references to the meta pages.
+	db.meta0 = db.page(0).meta()
+	db.meta1 = db.page(1).meta()
+
+	// Validate the meta pages. We only return an error if both meta pages fail
+	// validation, since meta0 failing validation means that it wasn't saved
+	// properly -- but we can recover using meta1. And vice-versa.
+	err0 := db.meta0.validate()
+	err1 := db.meta1.validate()
+	if err0 != nil && err1 != nil {
+		return err0
+	}
+
+	return nil
+}
+
+// munmap unmaps the data file from memory.
+func (db *DB) munmap() error {
+	if err := munmap(db); err != nil {
+		return fmt.Errorf("unmap error: " + err.Error())
+	}
+	return nil
+}
+
+// mmapSize determines the appropriate size for the mmap given the current size
+// of the database. The minimum size is 32KB and doubles until it reaches 1GB.
+// Returns an error if the new mmap size is greater than the max allowed.
+func (db *DB) mmapSize(size int) (int, error) {
+	// Double the size from 32KB until 1GB.
+	for i := uint(15); i <= 30; i++ {
+		if size <= 1<<i {
+			return 1 << i, nil
+		}
+	}
+
+	// Verify the requested size is not above the maximum allowed.
+	if size > maxMapSize {
+		return 0, fmt.Errorf("mmap too large")
+	}
+
+	// If larger than 1GB then grow by 1GB at a time.
+	sz := int64(size)
+	if remainder := sz % int64(maxMmapStep); remainder > 0 {
+		sz += int64(maxMmapStep) - remainder
+	}
+
+	// Ensure that the mmap size is a multiple of the page size.
+	// This should always be true since we're incrementing in MBs.
+	pageSize := int64(db.pageSize)
+	if (sz % pageSize) != 0 {
+		sz = ((sz / pageSize) + 1) * pageSize
+	}
+
+	// If we've exceeded the max size then only grow up to the max size.
+	if sz > maxMapSize {
+		sz = maxMapSize
+	}
+
+	return int(sz), nil
+}
+
+// init creates a new database file and initializes its meta pages.
+func (db *DB) init() error {
+	// Create two meta pages on a buffer.
+	buf := make([]byte, db.pageSize*4)
+	for i := 0; i < 2; i++ {
+		p := db.pageInBuffer(buf[:], pgid(i))
+		p.id = pgid(i)
+		p.flags = metaPageFlag
+
+		// Initialize the meta page.
+		m := p.meta()
+		m.magic = magic
+		m.version = version
+		m.pageSize = uint32(db.pageSize)
+		m.freelist = 2
+		m.root = bucket{root: 3}
+		m.pgid = 4
+		m.txid = txid(i)
+		m.checksum = m.sum64()
+	}
+
+	// Write an empty freelist at page 3.
+	p := db.pageInBuffer(buf[:], pgid(2))
+	p.id = pgid(2)
+	p.flags = freelistPageFlag
+	p.count = 0
+
+	// Write an empty leaf page at page 4.
+	p = db.pageInBuffer(buf[:], pgid(3))
+	p.id = pgid(3)
+	p.flags = leafPageFlag
+	p.count = 0
+
+	// Write the buffer to our data file.
+	if _, err := db.ops.writeAt(buf, 0); err != nil {
+		return err
+	}
+	if err := fdatasync(db); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+// Close releases all database resources.
+// It will block waiting for any open transactions to finish
+// before closing the database and returning.
+func (db *DB) Close() error {
+	db.rwlock.Lock()
+	defer db.rwlock.Unlock()
+
+	db.metalock.Lock()
+	defer db.metalock.Unlock()
+
+	db.mmaplock.Lock()
+	defer db.mmaplock.Unlock()
+
+	return db.close()
+}
+
+func (db *DB) close() error {
+	if !db.opened {
+		return nil
+	}
+
+	db.opened = false
+
+	db.freelist = nil
+
+	// Clear ops.
+	db.ops.writeAt = nil
+
+	// Close the mmap.
+	if err := db.munmap(); err != nil {
+		return err
+	}
+
+	// Close file handles.
+	if db.file != nil {
+		// No need to unlock read-only file.
+		if !db.readOnly {
+			// Unlock the file.
+			if err := funlock(db); err != nil {
+				log.Printf("bolt.Close(): funlock error: %s", err)
+			}
+		}
+
+		// Close the file descriptor.
+		if err := db.file.Close(); err != nil {
+			return fmt.Errorf("db file close: %s", err)
+		}
+		db.file = nil
+	}
+
+	db.path = ""
+	return nil
+}
+
+// Begin starts a new transaction.
+// Multiple read-only transactions can be used concurrently but only one
+// write transaction can be used at a time. Starting multiple write transactions
+// will cause the calls to block and be serialized until the current write
+// transaction finishes.
+//
+// Transactions should not be dependent on one another. Opening a read
+// transaction and a write transaction in the same goroutine can cause the
+// writer to deadlock because the database periodically needs to re-mmap itself
+// as it grows and it cannot do that while a read transaction is open.
+//
+// If a long running read transaction (for example, a snapshot transaction) is
+// needed, you might want to set DB.InitialMmapSize to a large enough value
+// to avoid potential blocking of write transaction.
+//
+// IMPORTANT: You must close read-only transactions after you are finished or
+// else the database will not reclaim old pages.
+func (db *DB) Begin(writable bool) (*Tx, error) {
+	if writable {
+		return db.beginRWTx()
+	}
+	return db.beginTx()
+}
+
+func (db *DB) beginTx() (*Tx, error) {
+	// Lock the meta pages while we initialize the transaction. We obtain
+	// the meta lock before the mmap lock because that's the order that the
+	// write transaction will obtain them.
+	db.metalock.Lock()
+
+	// Obtain a read-only lock on the mmap. When the mmap is remapped it will
+	// obtain a write lock so all transactions must finish before it can be
+	// remapped.
+	db.mmaplock.RLock()
+
+	// Exit if the database is not open yet.
+	if !db.opened {
+		db.mmaplock.RUnlock()
+		db.metalock.Unlock()
+		return nil, ErrDatabaseNotOpen
+	}
+
+	// Create a transaction associated with the database.
+	t := &Tx{}
+	t.init(db)
+
+	// Keep track of transaction until it closes.
+	db.txs = append(db.txs, t)
+	n := len(db.txs)
+
+	// Unlock the meta pages.
+	db.metalock.Unlock()
+
+	// Update the transaction stats.
+	db.statlock.Lock()
+	db.stats.TxN++
+	db.stats.OpenTxN = n
+	db.statlock.Unlock()
+
+	return t, nil
+}
+
+func (db *DB) beginRWTx() (*Tx, error) {
+	// If the database was opened with Options.ReadOnly, return an error.
+	if db.readOnly {
+		return nil, ErrDatabaseReadOnly
+	}
+
+	// Obtain writer lock. This is released by the transaction when it closes.
+	// This enforces only one writer transaction at a time.
+	db.rwlock.Lock()
+
+	// Once we have the writer lock then we can lock the meta pages so that
+	// we can set up the transaction.
+	db.metalock.Lock()
+	defer db.metalock.Unlock()
+
+	// Exit if the database is not open yet.
+	if !db.opened {
+		db.rwlock.Unlock()
+		return nil, ErrDatabaseNotOpen
+	}
+
+	// Create a transaction associated with the database.
+	t := &Tx{writable: true}
+	t.init(db)
+	db.rwtx = t
+	db.freePages()
+	return t, nil
+}
+
+// freePages releases any pages associated with closed read-only transactions.
+func (db *DB) freePages() {
+	// Free all pending pages prior to earliest open transaction.
+	sort.Sort(txsById(db.txs))
+	minid := txid(0xFFFFFFFFFFFFFFFF)
+	if len(db.txs) > 0 {
+		minid = db.txs[0].meta.txid
+	}
+	if minid > 0 {
+		db.freelist.release(minid - 1)
+	}
+	// Release unused txid extents.
+	for _, t := range db.txs {
+		db.freelist.releaseRange(minid, t.meta.txid-1)
+		minid = t.meta.txid + 1
+	}
+	db.freelist.releaseRange(minid, txid(0xFFFFFFFFFFFFFFFF))
+	// Any page both allocated and freed in an extent is safe to release.
+}
+
+type txsById []*Tx
+
+func (t txsById) Len() int           { return len(t) }
+func (t txsById) Swap(i, j int)      { t[i], t[j] = t[j], t[i] }
+func (t txsById) Less(i, j int) bool { return t[i].meta.txid < t[j].meta.txid }
+
+// removeTx removes a transaction from the database.
+func (db *DB) removeTx(tx *Tx) {
+	// Release the read lock on the mmap.
+	db.mmaplock.RUnlock()
+
+	// Use the meta lock to restrict access to the DB object.
+	db.metalock.Lock()
+
+	// Remove the transaction.
+	for i, t := range db.txs {
+		if t == tx {
+			last := len(db.txs) - 1
+			db.txs[i] = db.txs[last]
+			db.txs[last] = nil
+			db.txs = db.txs[:last]
+			break
+		}
+	}
+	n := len(db.txs)
+
+	// Unlock the meta pages.
+	db.metalock.Unlock()
+
+	// Merge statistics.
+	db.statlock.Lock()
+	db.stats.OpenTxN = n
+	db.stats.TxStats.add(&tx.stats)
+	db.statlock.Unlock()
+}
+
+// Update executes a function within the context of a read-write managed transaction.
+// If no error is returned from the function then the transaction is committed.
+// If an error is returned then the entire transaction is rolled back.
+// Any error that is returned from the function or returned from the commit is
+// returned from the Update() method.
+//
+// Attempting to manually commit or rollback within the function will cause a panic.
+func (db *DB) Update(fn func(*Tx) error) error {
+	t, err := db.Begin(true)
+	if err != nil {
+		return err
+	}
+
+	// Make sure the transaction rolls back in the event of a panic.
+	defer func() {
+		if t.db != nil {
+			t.rollback()
+		}
+	}()
+
+	// Mark as a managed tx so that the inner function cannot manually commit.
+	t.managed = true
+
+	// If an error is returned from the function then rollback and return error.
+	err = fn(t)
+	t.managed = false
+	if err != nil {
+		_ = t.Rollback()
+		return err
+	}
+
+	return t.Commit()
+}
+
+// View executes a function within the context of a managed read-only transaction.
+// Any error that is returned from the function is returned from the View() method.
+//
+// Attempting to manually rollback within the function will cause a panic.
+func (db *DB) View(fn func(*Tx) error) error {
+	t, err := db.Begin(false)
+	if err != nil {
+		return err
+	}
+
+	// Make sure the transaction rolls back in the event of a panic.
+	defer func() {
+		if t.db != nil {
+			t.rollback()
+		}
+	}()
+
+	// Mark as a managed tx so that the inner function cannot manually rollback.
+	t.managed = true
+
+	// If an error is returned from the function then pass it through.
+	err = fn(t)
+	t.managed = false
+	if err != nil {
+		_ = t.Rollback()
+		return err
+	}
+
+	return t.Rollback()
+}
+
+// Batch calls fn as part of a batch. It behaves similar to Update,
+// except:
+//
+// 1. concurrent Batch calls can be combined into a single Bolt
+// transaction.
+//
+// 2. the function passed to Batch may be called multiple times,
+// regardless of whether it returns error or not.
+//
+// This means that Batch function side effects must be idempotent and
+// take permanent effect only after a successful return is seen in
+// caller.
+//
+// The maximum batch size and delay can be adjusted with DB.MaxBatchSize
+// and DB.MaxBatchDelay, respectively.
+//
+// Batch is only useful when there are multiple goroutines calling it.
+func (db *DB) Batch(fn func(*Tx) error) error {
+	errCh := make(chan error, 1)
+
+	db.batchMu.Lock()
+	if (db.batch == nil) || (db.batch != nil && len(db.batch.calls) >= db.MaxBatchSize) {
+		// There is no existing batch, or the existing batch is full; start a new one.
+		db.batch = &batch{
+			db: db,
+		}
+		db.batch.timer = time.AfterFunc(db.MaxBatchDelay, db.batch.trigger)
+	}
+	db.batch.calls = append(db.batch.calls, call{fn: fn, err: errCh})
+	if len(db.batch.calls) >= db.MaxBatchSize {
+		// wake up batch, it's ready to run
+		go db.batch.trigger()
+	}
+	db.batchMu.Unlock()
+
+	err := <-errCh
+	if err == trySolo {
+		err = db.Update(fn)
+	}
+	return err
+}
+
+type call struct {
+	fn  func(*Tx) error
+	err chan<- error
+}
+
+type batch struct {
+	db    *DB
+	timer *time.Timer
+	start sync.Once
+	calls []call
+}
+
+// trigger runs the batch if it hasn't already been run.
+func (b *batch) trigger() {
+	b.start.Do(b.run)
+}
+
+// run performs the transactions in the batch and communicates results
+// back to DB.Batch.
+func (b *batch) run() {
+	b.db.batchMu.Lock()
+	b.timer.Stop()
+	// Make sure no new work is added to this batch, but don't break
+	// other batches.
+	if b.db.batch == b {
+		b.db.batch = nil
+	}
+	b.db.batchMu.Unlock()
+
+retry:
+	for len(b.calls) > 0 {
+		var failIdx = -1
+		err := b.db.Update(func(tx *Tx) error {
+			for i, c := range b.calls {
+				if err := safelyCall(c.fn, tx); err != nil {
+					failIdx = i
+					return err
+				}
+			}
+			return nil
+		})
+
+		if failIdx >= 0 {
+			// take the failing transaction out of the batch. it's
+			// safe to shorten b.calls here because db.batch no longer
+			// points to us, and we hold the mutex anyway.
+			c := b.calls[failIdx]
+			b.calls[failIdx], b.calls = b.calls[len(b.calls)-1], b.calls[:len(b.calls)-1]
+			// tell the submitter re-run it solo, continue with the rest of the batch
+			c.err <- trySolo
+			continue retry
+		}
+
+		// pass success, or bolt internal errors, to all callers
+		for _, c := range b.calls {
+			c.err <- err
+		}
+		break retry
+	}
+}
+
+// trySolo is a special sentinel error value used for signaling that a
+// transaction function should be re-run. It should never be seen by
+// callers.
+var trySolo = errors.New("batch function returned an error and should be re-run solo")
+
+type panicked struct {
+	reason interface{}
+}
+
+func (p panicked) Error() string {
+	if err, ok := p.reason.(error); ok {
+		return err.Error()
+	}
+	return fmt.Sprintf("panic: %v", p.reason)
+}
+
+func safelyCall(fn func(*Tx) error, tx *Tx) (err error) {
+	defer func() {
+		if p := recover(); p != nil {
+			err = panicked{p}
+		}
+	}()
+	return fn(tx)
+}
+
+// Sync executes fdatasync() against the database file handle.
+//
+// This is not necessary under normal operation, however, if you use NoSync
+// then it allows you to force the database file to sync against the disk.
+func (db *DB) Sync() error { return fdatasync(db) }
+
+// Stats retrieves ongoing performance stats for the database.
+// This is only updated when a transaction closes.
+func (db *DB) Stats() Stats {
+	db.statlock.RLock()
+	defer db.statlock.RUnlock()
+	return db.stats
+}
+
+// This is for internal access to the raw data bytes from the C cursor, use
+// carefully, or not at all.
+func (db *DB) Info() *Info {
+	return &Info{uintptr(unsafe.Pointer(&db.data[0])), db.pageSize}
+}
+
+// page retrieves a page reference from the mmap based on the current page size.
+func (db *DB) page(id pgid) *page {
+	pos := id * pgid(db.pageSize)
+	return (*page)(unsafe.Pointer(&db.data[pos]))
+}
+
+// pageInBuffer retrieves a page reference from a given byte array based on the current page size.
+func (db *DB) pageInBuffer(b []byte, id pgid) *page {
+	return (*page)(unsafe.Pointer(&b[id*pgid(db.pageSize)]))
+}
+
+// meta retrieves the current meta page reference.
+func (db *DB) meta() *meta {
+	// We have to return the meta with the highest txid which doesn't fail
+	// validation. Otherwise, we can cause errors when in fact the database is
+	// in a consistent state. metaA is the one with the higher txid.
+	metaA := db.meta0
+	metaB := db.meta1
+	if db.meta1.txid > db.meta0.txid {
+		metaA = db.meta1
+		metaB = db.meta0
+	}
+
+	// Use higher meta page if valid. Otherwise fallback to previous, if valid.
+	if err := metaA.validate(); err == nil {
+		return metaA
+	} else if err := metaB.validate(); err == nil {
+		return metaB
+	}
+
+	// This should never be reached, because both meta1 and meta0 were validated
+	// on mmap() and we do fsync() on every write.
+	panic("bolt.DB.meta(): invalid meta pages")
+}
+
+// allocate returns a contiguous block of memory starting at a given page.
+func (db *DB) allocate(txid txid, count int) (*page, error) {
+	// Allocate a temporary buffer for the page.
+	var buf []byte
+	if count == 1 {
+		buf = db.pagePool.Get().([]byte)
+	} else {
+		buf = make([]byte, count*db.pageSize)
+	}
+	p := (*page)(unsafe.Pointer(&buf[0]))
+	p.overflow = uint32(count - 1)
+
+	// Use pages from the freelist if they are available.
+	if p.id = db.freelist.allocate(txid, count); p.id != 0 {
+		return p, nil
+	}
+
+	// Resize mmap() if we're at the end.
+	p.id = db.rwtx.meta.pgid
+	var minsz = int((p.id+pgid(count))+1) * db.pageSize
+	if minsz >= db.datasz {
+		if err := db.mmap(minsz); err != nil {
+			return nil, fmt.Errorf("mmap allocate error: %s", err)
+		}
+	}
+
+	// Move the page id high water mark.
+	db.rwtx.meta.pgid += pgid(count)
+
+	return p, nil
+}
+
+// grow grows the size of the database to the given sz.
+func (db *DB) grow(sz int) error {
+	// Ignore if the new size is less than available file size.
+	if sz <= db.filesz {
+		return nil
+	}
+
+	// If the data is smaller than the alloc size then only allocate what's needed.
+	// Once it goes over the allocation size then allocate in chunks.
+	if db.datasz < db.AllocSize {
+		sz = db.datasz
+	} else {
+		sz += db.AllocSize
+	}
+
+	// Truncate and fsync to ensure file size metadata is flushed.
+	// https://github.com/boltdb/bolt/issues/284
+	if !db.NoGrowSync && !db.readOnly {
+		if runtime.GOOS != "windows" {
+			if err := db.file.Truncate(int64(sz)); err != nil {
+				return fmt.Errorf("file resize error: %s", err)
+			}
+		}
+		if err := db.file.Sync(); err != nil {
+			return fmt.Errorf("file sync error: %s", err)
+		}
+	}
+
+	db.filesz = sz
+	return nil
+}
+
+func (db *DB) IsReadOnly() bool {
+	return db.readOnly
+}
+
+func (db *DB) freepages() []pgid {
+	tx, err := db.beginTx()
+	defer func() {
+		err = tx.Rollback()
+		if err != nil {
+			panic("freepages: failed to rollback tx")
+		}
+	}()
+	if err != nil {
+		panic("freepages: failed to open read only tx")
+	}
+
+	reachable := make(map[pgid]*page)
+	nofreed := make(map[pgid]bool)
+	ech := make(chan error)
+	go func() {
+		for e := range ech {
+			panic(fmt.Sprintf("freepages: failed to get all reachable pages (%v)", e))
+		}
+	}()
+	tx.checkBucket(&tx.root, reachable, nofreed, ech)
+	close(ech)
+
+	var fids []pgid
+	for i := pgid(2); i < db.meta().pgid; i++ {
+		if _, ok := reachable[i]; !ok {
+			fids = append(fids, i)
+		}
+	}
+	return fids
+}
+
+// Options represents the options that can be set when opening a database.
+type Options struct {
+	// Timeout is the amount of time to wait to obtain a file lock.
+	// When set to zero it will wait indefinitely. This option is only
+	// available on Darwin and Linux.
+	Timeout time.Duration
+
+	// Sets the DB.NoGrowSync flag before memory mapping the file.
+	NoGrowSync bool
+
+	// Do not sync freelist to disk. This improves the database write performance
+	// under normal operation, but requires a full database re-sync during recovery.
+	NoFreelistSync bool
+
+	// FreelistType sets the backend freelist type. There are two options. Array which is simple but endures
+	// dramatic performance degradation if database is large and framentation in freelist is common.
+	// The alternative one is using hashmap, it is faster in almost all circumstances
+	// but it doesn't guarantee that it offers the smallest page id available. In normal case it is safe.
+	// The default type is array
+	FreelistType FreelistType
+
+	// Open database in read-only mode. Uses flock(..., LOCK_SH |LOCK_NB) to
+	// grab a shared lock (UNIX).
+	ReadOnly bool
+
+	// Sets the DB.MmapFlags flag before memory mapping the file.
+	MmapFlags int
+
+	// InitialMmapSize is the initial mmap size of the database
+	// in bytes. Read transactions won't block write transaction
+	// if the InitialMmapSize is large enough to hold database mmap
+	// size. (See DB.Begin for more information)
+	//
+	// If <=0, the initial map size is 0.
+	// If initialMmapSize is smaller than the previous database size,
+	// it takes no effect.
+	InitialMmapSize int
+
+	// PageSize overrides the default OS page size.
+	PageSize int
+
+	// NoSync sets the initial value of DB.NoSync. Normally this can just be
+	// set directly on the DB itself when returned from Open(), but this option
+	// is useful in APIs which expose Options but not the underlying DB.
+	NoSync bool
+
+	// OpenFile is used to open files. It defaults to os.OpenFile. This option
+	// is useful for writing hermetic tests.
+	OpenFile func(string, int, os.FileMode) (*os.File, error)
+}
+
+// DefaultOptions represent the options used if nil options are passed into Open().
+// No timeout is used which will cause Bolt to wait indefinitely for a lock.
+var DefaultOptions = &Options{
+	Timeout:      0,
+	NoGrowSync:   false,
+	FreelistType: FreelistArrayType,
+}
+
+// Stats represents statistics about the database.
+type Stats struct {
+	// Freelist stats
+	FreePageN     int // total number of free pages on the freelist
+	PendingPageN  int // total number of pending pages on the freelist
+	FreeAlloc     int // total bytes allocated in free pages
+	FreelistInuse int // total bytes used by the freelist
+
+	// Transaction stats
+	TxN     int // total number of started read transactions
+	OpenTxN int // number of currently open read transactions
+
+	TxStats TxStats // global, ongoing stats.
+}
+
+// Sub calculates and returns the difference between two sets of database stats.
+// This is useful when obtaining stats at two different points and time and
+// you need the performance counters that occurred within that time span.
+func (s *Stats) Sub(other *Stats) Stats {
+	if other == nil {
+		return *s
+	}
+	var diff Stats
+	diff.FreePageN = s.FreePageN
+	diff.PendingPageN = s.PendingPageN
+	diff.FreeAlloc = s.FreeAlloc
+	diff.FreelistInuse = s.FreelistInuse
+	diff.TxN = s.TxN - other.TxN
+	diff.TxStats = s.TxStats.Sub(&other.TxStats)
+	return diff
+}
+
+type Info struct {
+	Data     uintptr
+	PageSize int
+}
+
+type meta struct {
+	magic    uint32
+	version  uint32
+	pageSize uint32
+	flags    uint32
+	root     bucket
+	freelist pgid
+	pgid     pgid
+	txid     txid
+	checksum uint64
+}
+
+// validate checks the marker bytes and version of the meta page to ensure it matches this binary.
+func (m *meta) validate() error {
+	if m.magic != magic {
+		return ErrInvalid
+	} else if m.version != version {
+		return ErrVersionMismatch
+	} else if m.checksum != 0 && m.checksum != m.sum64() {
+		return ErrChecksum
+	}
+	return nil
+}
+
+// copy copies one meta object to another.
+func (m *meta) copy(dest *meta) {
+	*dest = *m
+}
+
+// write writes the meta onto a page.
+func (m *meta) write(p *page) {
+	if m.root.root >= m.pgid {
+		panic(fmt.Sprintf("root bucket pgid (%d) above high water mark (%d)", m.root.root, m.pgid))
+	} else if m.freelist >= m.pgid && m.freelist != pgidNoFreelist {
+		// TODO: reject pgidNoFreeList if !NoFreelistSync
+		panic(fmt.Sprintf("freelist pgid (%d) above high water mark (%d)", m.freelist, m.pgid))
+	}
+
+	// Page id is either going to be 0 or 1 which we can determine by the transaction ID.
+	p.id = pgid(m.txid % 2)
+	p.flags |= metaPageFlag
+
+	// Calculate the checksum.
+	m.checksum = m.sum64()
+
+	m.copy(p.meta())
+}
+
+// generates the checksum for the meta.
+func (m *meta) sum64() uint64 {
+	var h = fnv.New64a()
+	_, _ = h.Write((*[unsafe.Offsetof(meta{}.checksum)]byte)(unsafe.Pointer(m))[:])
+	return h.Sum64()
+}
+
+// _assert will panic with a given formatted message if the given condition is false.
+func _assert(condition bool, msg string, v ...interface{}) {
+	if !condition {
+		panic(fmt.Sprintf("assertion failed: "+msg, v...))
+	}
+}
diff --git a/vendor/go.etcd.io/bbolt/doc.go b/vendor/go.etcd.io/bbolt/doc.go
new file mode 100644
index 0000000..95f25f0
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/doc.go
@@ -0,0 +1,44 @@
+/*
+package bbolt implements a low-level key/value store in pure Go. It supports
+fully serializable transactions, ACID semantics, and lock-free MVCC with
+multiple readers and a single writer. Bolt can be used for projects that
+want a simple data store without the need to add large dependencies such as
+Postgres or MySQL.
+
+Bolt is a single-level, zero-copy, B+tree data store. This means that Bolt is
+optimized for fast read access and does not require recovery in the event of a
+system crash. Transactions which have not finished committing will simply be
+rolled back in the event of a crash.
+
+The design of Bolt is based on Howard Chu's LMDB database project.
+
+Bolt currently works on Windows, Mac OS X, and Linux.
+
+
+Basics
+
+There are only a few types in Bolt: DB, Bucket, Tx, and Cursor. The DB is
+a collection of buckets and is represented by a single file on disk. A bucket is
+a collection of unique keys that are associated with values.
+
+Transactions provide either read-only or read-write access to the database.
+Read-only transactions can retrieve key/value pairs and can use Cursors to
+iterate over the dataset sequentially. Read-write transactions can create and
+delete buckets and can insert and remove keys. Only one read-write transaction
+is allowed at a time.
+
+
+Caveats
+
+The database uses a read-only, memory-mapped data file to ensure that
+applications cannot corrupt the database, however, this means that keys and
+values returned from Bolt cannot be changed. Writing to a read-only byte slice
+will cause Go to panic.
+
+Keys and values retrieved from the database are only valid for the life of
+the transaction. When used outside the transaction, these byte slices can
+point to different data or can point to invalid memory which will cause a panic.
+
+
+*/
+package bbolt
diff --git a/vendor/go.etcd.io/bbolt/errors.go b/vendor/go.etcd.io/bbolt/errors.go
new file mode 100644
index 0000000..48758ca
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/errors.go
@@ -0,0 +1,71 @@
+package bbolt
+
+import "errors"
+
+// These errors can be returned when opening or calling methods on a DB.
+var (
+	// ErrDatabaseNotOpen is returned when a DB instance is accessed before it
+	// is opened or after it is closed.
+	ErrDatabaseNotOpen = errors.New("database not open")
+
+	// ErrDatabaseOpen is returned when opening a database that is
+	// already open.
+	ErrDatabaseOpen = errors.New("database already open")
+
+	// ErrInvalid is returned when both meta pages on a database are invalid.
+	// This typically occurs when a file is not a bolt database.
+	ErrInvalid = errors.New("invalid database")
+
+	// ErrVersionMismatch is returned when the data file was created with a
+	// different version of Bolt.
+	ErrVersionMismatch = errors.New("version mismatch")
+
+	// ErrChecksum is returned when either meta page checksum does not match.
+	ErrChecksum = errors.New("checksum error")
+
+	// ErrTimeout is returned when a database cannot obtain an exclusive lock
+	// on the data file after the timeout passed to Open().
+	ErrTimeout = errors.New("timeout")
+)
+
+// These errors can occur when beginning or committing a Tx.
+var (
+	// ErrTxNotWritable is returned when performing a write operation on a
+	// read-only transaction.
+	ErrTxNotWritable = errors.New("tx not writable")
+
+	// ErrTxClosed is returned when committing or rolling back a transaction
+	// that has already been committed or rolled back.
+	ErrTxClosed = errors.New("tx closed")
+
+	// ErrDatabaseReadOnly is returned when a mutating transaction is started on a
+	// read-only database.
+	ErrDatabaseReadOnly = errors.New("database is in read-only mode")
+)
+
+// These errors can occur when putting or deleting a value or a bucket.
+var (
+	// ErrBucketNotFound is returned when trying to access a bucket that has
+	// not been created yet.
+	ErrBucketNotFound = errors.New("bucket not found")
+
+	// ErrBucketExists is returned when creating a bucket that already exists.
+	ErrBucketExists = errors.New("bucket already exists")
+
+	// ErrBucketNameRequired is returned when creating a bucket with a blank name.
+	ErrBucketNameRequired = errors.New("bucket name required")
+
+	// ErrKeyRequired is returned when inserting a zero-length key.
+	ErrKeyRequired = errors.New("key required")
+
+	// ErrKeyTooLarge is returned when inserting a key that is larger than MaxKeySize.
+	ErrKeyTooLarge = errors.New("key too large")
+
+	// ErrValueTooLarge is returned when inserting a value that is larger than MaxValueSize.
+	ErrValueTooLarge = errors.New("value too large")
+
+	// ErrIncompatibleValue is returned when trying create or delete a bucket
+	// on an existing non-bucket key or when trying to create or delete a
+	// non-bucket key on an existing bucket key.
+	ErrIncompatibleValue = errors.New("incompatible value")
+)
diff --git a/vendor/go.etcd.io/bbolt/freelist.go b/vendor/go.etcd.io/bbolt/freelist.go
new file mode 100644
index 0000000..587b8cc
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/freelist.go
@@ -0,0 +1,392 @@
+package bbolt
+
+import (
+	"fmt"
+	"sort"
+	"unsafe"
+)
+
+// txPending holds a list of pgids and corresponding allocation txns
+// that are pending to be freed.
+type txPending struct {
+	ids              []pgid
+	alloctx          []txid // txids allocating the ids
+	lastReleaseBegin txid   // beginning txid of last matching releaseRange
+}
+
+// pidSet holds the set of starting pgids which have the same span size
+type pidSet map[pgid]struct{}
+
+// freelist represents a list of all pages that are available for allocation.
+// It also tracks pages that have been freed but are still in use by open transactions.
+type freelist struct {
+	freelistType   FreelistType                // freelist type
+	ids            []pgid                      // all free and available free page ids.
+	allocs         map[pgid]txid               // mapping of txid that allocated a pgid.
+	pending        map[txid]*txPending         // mapping of soon-to-be free page ids by tx.
+	cache          map[pgid]bool               // fast lookup of all free and pending page ids.
+	freemaps       map[uint64]pidSet           // key is the size of continuous pages(span), value is a set which contains the starting pgids of same size
+	forwardMap     map[pgid]uint64             // key is start pgid, value is its span size
+	backwardMap    map[pgid]uint64             // key is end pgid, value is its span size
+	allocate       func(txid txid, n int) pgid // the freelist allocate func
+	free_count     func() int                  // the function which gives you free page number
+	mergeSpans     func(ids pgids)             // the mergeSpan func
+	getFreePageIDs func() []pgid               // get free pgids func
+	readIDs        func(pgids []pgid)          // readIDs func reads list of pages and init the freelist
+}
+
+// newFreelist returns an empty, initialized freelist.
+func newFreelist(freelistType FreelistType) *freelist {
+	f := &freelist{
+		freelistType: freelistType,
+		allocs:       make(map[pgid]txid),
+		pending:      make(map[txid]*txPending),
+		cache:        make(map[pgid]bool),
+		freemaps:     make(map[uint64]pidSet),
+		forwardMap:   make(map[pgid]uint64),
+		backwardMap:  make(map[pgid]uint64),
+	}
+
+	if freelistType == FreelistMapType {
+		f.allocate = f.hashmapAllocate
+		f.free_count = f.hashmapFreeCount
+		f.mergeSpans = f.hashmapMergeSpans
+		f.getFreePageIDs = f.hashmapGetFreePageIDs
+		f.readIDs = f.hashmapReadIDs
+	} else {
+		f.allocate = f.arrayAllocate
+		f.free_count = f.arrayFreeCount
+		f.mergeSpans = f.arrayMergeSpans
+		f.getFreePageIDs = f.arrayGetFreePageIDs
+		f.readIDs = f.arrayReadIDs
+	}
+
+	return f
+}
+
+// size returns the size of the page after serialization.
+func (f *freelist) size() int {
+	n := f.count()
+	if n >= 0xFFFF {
+		// The first element will be used to store the count. See freelist.write.
+		n++
+	}
+	return pageHeaderSize + (int(unsafe.Sizeof(pgid(0))) * n)
+}
+
+// count returns count of pages on the freelist
+func (f *freelist) count() int {
+	return f.free_count() + f.pending_count()
+}
+
+// arrayFreeCount returns count of free pages(array version)
+func (f *freelist) arrayFreeCount() int {
+	return len(f.ids)
+}
+
+// pending_count returns count of pending pages
+func (f *freelist) pending_count() int {
+	var count int
+	for _, txp := range f.pending {
+		count += len(txp.ids)
+	}
+	return count
+}
+
+// copyall copies into dst a list of all free ids and all pending ids in one sorted list.
+// f.count returns the minimum length required for dst.
+func (f *freelist) copyall(dst []pgid) {
+	m := make(pgids, 0, f.pending_count())
+	for _, txp := range f.pending {
+		m = append(m, txp.ids...)
+	}
+	sort.Sort(m)
+	mergepgids(dst, f.getFreePageIDs(), m)
+}
+
+// arrayAllocate returns the starting page id of a contiguous list of pages of a given size.
+// If a contiguous block cannot be found then 0 is returned.
+func (f *freelist) arrayAllocate(txid txid, n int) pgid {
+	if len(f.ids) == 0 {
+		return 0
+	}
+
+	var initial, previd pgid
+	for i, id := range f.ids {
+		if id <= 1 {
+			panic(fmt.Sprintf("invalid page allocation: %d", id))
+		}
+
+		// Reset initial page if this is not contiguous.
+		if previd == 0 || id-previd != 1 {
+			initial = id
+		}
+
+		// If we found a contiguous block then remove it and return it.
+		if (id-initial)+1 == pgid(n) {
+			// If we're allocating off the beginning then take the fast path
+			// and just adjust the existing slice. This will use extra memory
+			// temporarily but the append() in free() will realloc the slice
+			// as is necessary.
+			if (i + 1) == n {
+				f.ids = f.ids[i+1:]
+			} else {
+				copy(f.ids[i-n+1:], f.ids[i+1:])
+				f.ids = f.ids[:len(f.ids)-n]
+			}
+
+			// Remove from the free cache.
+			for i := pgid(0); i < pgid(n); i++ {
+				delete(f.cache, initial+i)
+			}
+			f.allocs[initial] = txid
+			return initial
+		}
+
+		previd = id
+	}
+	return 0
+}
+
+// free releases a page and its overflow for a given transaction id.
+// If the page is already free then a panic will occur.
+func (f *freelist) free(txid txid, p *page) {
+	if p.id <= 1 {
+		panic(fmt.Sprintf("cannot free page 0 or 1: %d", p.id))
+	}
+
+	// Free page and all its overflow pages.
+	txp := f.pending[txid]
+	if txp == nil {
+		txp = &txPending{}
+		f.pending[txid] = txp
+	}
+	allocTxid, ok := f.allocs[p.id]
+	if ok {
+		delete(f.allocs, p.id)
+	} else if (p.flags & freelistPageFlag) != 0 {
+		// Freelist is always allocated by prior tx.
+		allocTxid = txid - 1
+	}
+
+	for id := p.id; id <= p.id+pgid(p.overflow); id++ {
+		// Verify that page is not already free.
+		if f.cache[id] {
+			panic(fmt.Sprintf("page %d already freed", id))
+		}
+		// Add to the freelist and cache.
+		txp.ids = append(txp.ids, id)
+		txp.alloctx = append(txp.alloctx, allocTxid)
+		f.cache[id] = true
+	}
+}
+
+// release moves all page ids for a transaction id (or older) to the freelist.
+func (f *freelist) release(txid txid) {
+	m := make(pgids, 0)
+	for tid, txp := range f.pending {
+		if tid <= txid {
+			// Move transaction's pending pages to the available freelist.
+			// Don't remove from the cache since the page is still free.
+			m = append(m, txp.ids...)
+			delete(f.pending, tid)
+		}
+	}
+	f.mergeSpans(m)
+}
+
+// releaseRange moves pending pages allocated within an extent [begin,end] to the free list.
+func (f *freelist) releaseRange(begin, end txid) {
+	if begin > end {
+		return
+	}
+	var m pgids
+	for tid, txp := range f.pending {
+		if tid < begin || tid > end {
+			continue
+		}
+		// Don't recompute freed pages if ranges haven't updated.
+		if txp.lastReleaseBegin == begin {
+			continue
+		}
+		for i := 0; i < len(txp.ids); i++ {
+			if atx := txp.alloctx[i]; atx < begin || atx > end {
+				continue
+			}
+			m = append(m, txp.ids[i])
+			txp.ids[i] = txp.ids[len(txp.ids)-1]
+			txp.ids = txp.ids[:len(txp.ids)-1]
+			txp.alloctx[i] = txp.alloctx[len(txp.alloctx)-1]
+			txp.alloctx = txp.alloctx[:len(txp.alloctx)-1]
+			i--
+		}
+		txp.lastReleaseBegin = begin
+		if len(txp.ids) == 0 {
+			delete(f.pending, tid)
+		}
+	}
+	f.mergeSpans(m)
+}
+
+// rollback removes the pages from a given pending tx.
+func (f *freelist) rollback(txid txid) {
+	// Remove page ids from cache.
+	txp := f.pending[txid]
+	if txp == nil {
+		return
+	}
+	var m pgids
+	for i, pgid := range txp.ids {
+		delete(f.cache, pgid)
+		tx := txp.alloctx[i]
+		if tx == 0 {
+			continue
+		}
+		if tx != txid {
+			// Pending free aborted; restore page back to alloc list.
+			f.allocs[pgid] = tx
+		} else {
+			// Freed page was allocated by this txn; OK to throw away.
+			m = append(m, pgid)
+		}
+	}
+	// Remove pages from pending list and mark as free if allocated by txid.
+	delete(f.pending, txid)
+	f.mergeSpans(m)
+}
+
+// freed returns whether a given page is in the free list.
+func (f *freelist) freed(pgid pgid) bool {
+	return f.cache[pgid]
+}
+
+// read initializes the freelist from a freelist page.
+func (f *freelist) read(p *page) {
+	if (p.flags & freelistPageFlag) == 0 {
+		panic(fmt.Sprintf("invalid freelist page: %d, page type is %s", p.id, p.typ()))
+	}
+	// If the page.count is at the max uint16 value (64k) then it's considered
+	// an overflow and the size of the freelist is stored as the first element.
+	idx, count := 0, int(p.count)
+	if count == 0xFFFF {
+		idx = 1
+		count = int(((*[maxAllocSize]pgid)(unsafe.Pointer(&p.ptr)))[0])
+	}
+
+	// Copy the list of page ids from the freelist.
+	if count == 0 {
+		f.ids = nil
+	} else {
+		ids := ((*[maxAllocSize]pgid)(unsafe.Pointer(&p.ptr)))[idx : idx+count]
+
+		// copy the ids, so we don't modify on the freelist page directly
+		idsCopy := make([]pgid, count)
+		copy(idsCopy, ids)
+		// Make sure they're sorted.
+		sort.Sort(pgids(idsCopy))
+
+		f.readIDs(idsCopy)
+	}
+}
+
+// arrayReadIDs initializes the freelist from a given list of ids.
+func (f *freelist) arrayReadIDs(ids []pgid) {
+	f.ids = ids
+	f.reindex()
+}
+
+func (f *freelist) arrayGetFreePageIDs() []pgid {
+	return f.ids
+}
+
+// write writes the page ids onto a freelist page. All free and pending ids are
+// saved to disk since in the event of a program crash, all pending ids will
+// become free.
+func (f *freelist) write(p *page) error {
+	// Combine the old free pgids and pgids waiting on an open transaction.
+
+	// Update the header flag.
+	p.flags |= freelistPageFlag
+
+	// The page.count can only hold up to 64k elements so if we overflow that
+	// number then we handle it by putting the size in the first element.
+	lenids := f.count()
+	if lenids == 0 {
+		p.count = uint16(lenids)
+	} else if lenids < 0xFFFF {
+		p.count = uint16(lenids)
+		f.copyall(((*[maxAllocSize]pgid)(unsafe.Pointer(&p.ptr)))[:])
+	} else {
+		p.count = 0xFFFF
+		((*[maxAllocSize]pgid)(unsafe.Pointer(&p.ptr)))[0] = pgid(lenids)
+		f.copyall(((*[maxAllocSize]pgid)(unsafe.Pointer(&p.ptr)))[1:])
+	}
+
+	return nil
+}
+
+// reload reads the freelist from a page and filters out pending items.
+func (f *freelist) reload(p *page) {
+	f.read(p)
+
+	// Build a cache of only pending pages.
+	pcache := make(map[pgid]bool)
+	for _, txp := range f.pending {
+		for _, pendingID := range txp.ids {
+			pcache[pendingID] = true
+		}
+	}
+
+	// Check each page in the freelist and build a new available freelist
+	// with any pages not in the pending lists.
+	var a []pgid
+	for _, id := range f.getFreePageIDs() {
+		if !pcache[id] {
+			a = append(a, id)
+		}
+	}
+
+	f.readIDs(a)
+}
+
+// noSyncReload reads the freelist from pgids and filters out pending items.
+func (f *freelist) noSyncReload(pgids []pgid) {
+	// Build a cache of only pending pages.
+	pcache := make(map[pgid]bool)
+	for _, txp := range f.pending {
+		for _, pendingID := range txp.ids {
+			pcache[pendingID] = true
+		}
+	}
+
+	// Check each page in the freelist and build a new available freelist
+	// with any pages not in the pending lists.
+	var a []pgid
+	for _, id := range pgids {
+		if !pcache[id] {
+			a = append(a, id)
+		}
+	}
+
+	f.readIDs(a)
+}
+
+// reindex rebuilds the free cache based on available and pending free lists.
+func (f *freelist) reindex() {
+	ids := f.getFreePageIDs()
+	f.cache = make(map[pgid]bool, len(ids))
+	for _, id := range ids {
+		f.cache[id] = true
+	}
+	for _, txp := range f.pending {
+		for _, pendingID := range txp.ids {
+			f.cache[pendingID] = true
+		}
+	}
+}
+
+// arrayMergeSpans try to merge list of pages(represented by pgids) with existing spans but using array
+func (f *freelist) arrayMergeSpans(ids pgids) {
+	sort.Sort(ids)
+	f.ids = pgids(f.ids).merge(ids)
+}
diff --git a/vendor/go.etcd.io/bbolt/freelist_hmap.go b/vendor/go.etcd.io/bbolt/freelist_hmap.go
new file mode 100644
index 0000000..6a03a6c
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/freelist_hmap.go
@@ -0,0 +1,178 @@
+package bbolt
+
+import "sort"
+
+// hashmapFreeCount returns count of free pages(hashmap version)
+func (f *freelist) hashmapFreeCount() int {
+	// use the forwardmap to get the total count
+	count := 0
+	for _, size := range f.forwardMap {
+		count += int(size)
+	}
+	return count
+}
+
+// hashmapAllocate serves the same purpose as arrayAllocate, but use hashmap as backend
+func (f *freelist) hashmapAllocate(txid txid, n int) pgid {
+	if n == 0 {
+		return 0
+	}
+
+	// if we have a exact size match just return short path
+	if bm, ok := f.freemaps[uint64(n)]; ok {
+		for pid := range bm {
+			// remove the span
+			f.delSpan(pid, uint64(n))
+
+			f.allocs[pid] = txid
+
+			for i := pgid(0); i < pgid(n); i++ {
+				delete(f.cache, pid+pgid(i))
+			}
+			return pid
+		}
+	}
+
+	// lookup the map to find larger span
+	for size, bm := range f.freemaps {
+		if size < uint64(n) {
+			continue
+		}
+
+		for pid := range bm {
+			// remove the initial
+			f.delSpan(pid, uint64(size))
+
+			f.allocs[pid] = txid
+
+			remain := size - uint64(n)
+
+			// add remain span
+			f.addSpan(pid+pgid(n), remain)
+
+			for i := pgid(0); i < pgid(n); i++ {
+				delete(f.cache, pid+pgid(i))
+			}
+			return pid
+		}
+	}
+
+	return 0
+}
+
+// hashmapReadIDs reads pgids as input an initial the freelist(hashmap version)
+func (f *freelist) hashmapReadIDs(pgids []pgid) {
+	f.init(pgids)
+
+	// Rebuild the page cache.
+	f.reindex()
+}
+
+// hashmapGetFreePageIDs returns the sorted free page ids
+func (f *freelist) hashmapGetFreePageIDs() []pgid {
+	count := f.free_count()
+	if count == 0 {
+		return nil
+	}
+
+	m := make([]pgid, 0, count)
+	for start, size := range f.forwardMap {
+		for i := 0; i < int(size); i++ {
+			m = append(m, start+pgid(i))
+		}
+	}
+	sort.Sort(pgids(m))
+
+	return m
+}
+
+// hashmapMergeSpans try to merge list of pages(represented by pgids) with existing spans
+func (f *freelist) hashmapMergeSpans(ids pgids) {
+	for _, id := range ids {
+		// try to see if we can merge and update
+		f.mergeWithExistingSpan(id)
+	}
+}
+
+// mergeWithExistingSpan merges pid to the existing free spans, try to merge it backward and forward
+func (f *freelist) mergeWithExistingSpan(pid pgid) {
+	prev := pid - 1
+	next := pid + 1
+
+	preSize, mergeWithPrev := f.backwardMap[prev]
+	nextSize, mergeWithNext := f.forwardMap[next]
+	newStart := pid
+	newSize := uint64(1)
+
+	if mergeWithPrev {
+		//merge with previous span
+		start := prev + 1 - pgid(preSize)
+		f.delSpan(start, preSize)
+
+		newStart -= pgid(preSize)
+		newSize += preSize
+	}
+
+	if mergeWithNext {
+		// merge with next span
+		f.delSpan(next, nextSize)
+		newSize += nextSize
+	}
+
+	f.addSpan(newStart, newSize)
+}
+
+func (f *freelist) addSpan(start pgid, size uint64) {
+	f.backwardMap[start-1+pgid(size)] = size
+	f.forwardMap[start] = size
+	if _, ok := f.freemaps[size]; !ok {
+		f.freemaps[size] = make(map[pgid]struct{})
+	}
+
+	f.freemaps[size][start] = struct{}{}
+}
+
+func (f *freelist) delSpan(start pgid, size uint64) {
+	delete(f.forwardMap, start)
+	delete(f.backwardMap, start+pgid(size-1))
+	delete(f.freemaps[size], start)
+	if len(f.freemaps[size]) == 0 {
+		delete(f.freemaps, size)
+	}
+}
+
+// initial from pgids using when use hashmap version
+// pgids must be sorted
+func (f *freelist) init(pgids []pgid) {
+	if len(pgids) == 0 {
+		return
+	}
+
+	size := uint64(1)
+	start := pgids[0]
+
+	if !sort.SliceIsSorted([]pgid(pgids), func(i, j int) bool { return pgids[i] < pgids[j] }) {
+		panic("pgids not sorted")
+	}
+
+	f.freemaps = make(map[uint64]pidSet)
+	f.forwardMap = make(map[pgid]uint64)
+	f.backwardMap = make(map[pgid]uint64)
+
+	for i := 1; i < len(pgids); i++ {
+		// continuous page
+		if pgids[i] == pgids[i-1]+1 {
+			size++
+		} else {
+			f.addSpan(start, size)
+
+			size = 1
+			start = pgids[i]
+		}
+	}
+
+	// init the tail
+	if size != 0 && start != 0 {
+		f.addSpan(start, size)
+	}
+}
diff --git a/vendor/go.etcd.io/bbolt/node.go b/vendor/go.etcd.io/bbolt/node.go
new file mode 100644
index 0000000..6c3fa55
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/node.go
@@ -0,0 +1,604 @@
+package bbolt
+
+import (
+	"bytes"
+	"fmt"
+	"sort"
+	"unsafe"
+)
+
+// node represents an in-memory, deserialized page.
+type node struct {
+	bucket     *Bucket
+	isLeaf     bool
+	unbalanced bool
+	spilled    bool
+	key        []byte
+	pgid       pgid
+	parent     *node
+	children   nodes
+	inodes     inodes
+}
+
+// root returns the top-level node this node is attached to.
+func (n *node) root() *node {
+	if n.parent == nil {
+		return n
+	}
+	return n.parent.root()
+}
+
+// minKeys returns the minimum number of inodes this node should have.
+func (n *node) minKeys() int {
+	if n.isLeaf {
+		return 1
+	}
+	return 2
+}
+
+// size returns the size of the node after serialization.
+func (n *node) size() int {
+	sz, elsz := pageHeaderSize, n.pageElementSize()
+	for i := 0; i < len(n.inodes); i++ {
+		item := &n.inodes[i]
+		sz += elsz + len(item.key) + len(item.value)
+	}
+	return sz
+}
+
+// sizeLessThan returns true if the node is less than a given size.
+// This is an optimization to avoid calculating a large node when we only need
+// to know if it fits inside a certain page size.
+func (n *node) sizeLessThan(v int) bool {
+	sz, elsz := pageHeaderSize, n.pageElementSize()
+	for i := 0; i < len(n.inodes); i++ {
+		item := &n.inodes[i]
+		sz += elsz + len(item.key) + len(item.value)
+		if sz >= v {
+			return false
+		}
+	}
+	return true
+}
+
+// pageElementSize returns the size of each page element based on the type of node.
+func (n *node) pageElementSize() int {
+	if n.isLeaf {
+		return leafPageElementSize
+	}
+	return branchPageElementSize
+}
+
+// childAt returns the child node at a given index.
+func (n *node) childAt(index int) *node {
+	if n.isLeaf {
+		panic(fmt.Sprintf("invalid childAt(%d) on a leaf node", index))
+	}
+	return n.bucket.node(n.inodes[index].pgid, n)
+}
+
+// childIndex returns the index of a given child node.
+func (n *node) childIndex(child *node) int {
+	index := sort.Search(len(n.inodes), func(i int) bool { return bytes.Compare(n.inodes[i].key, child.key) != -1 })
+	return index
+}
+
+// numChildren returns the number of children.
+func (n *node) numChildren() int {
+	return len(n.inodes)
+}
+
+// nextSibling returns the next node with the same parent.
+func (n *node) nextSibling() *node {
+	if n.parent == nil {
+		return nil
+	}
+	index := n.parent.childIndex(n)
+	if index >= n.parent.numChildren()-1 {
+		return nil
+	}
+	return n.parent.childAt(index + 1)
+}
+
+// prevSibling returns the previous node with the same parent.
+func (n *node) prevSibling() *node {
+	if n.parent == nil {
+		return nil
+	}
+	index := n.parent.childIndex(n)
+	if index == 0 {
+		return nil
+	}
+	return n.parent.childAt(index - 1)
+}
+
+// put inserts a key/value.
+func (n *node) put(oldKey, newKey, value []byte, pgid pgid, flags uint32) {
+	if pgid >= n.bucket.tx.meta.pgid {
+		panic(fmt.Sprintf("pgid (%d) above high water mark (%d)", pgid, n.bucket.tx.meta.pgid))
+	} else if len(oldKey) <= 0 {
+		panic("put: zero-length old key")
+	} else if len(newKey) <= 0 {
+		panic("put: zero-length new key")
+	}
+
+	// Find insertion index.
+	index := sort.Search(len(n.inodes), func(i int) bool { return bytes.Compare(n.inodes[i].key, oldKey) != -1 })
+
+	// Add capacity and shift nodes if we don't have an exact match and need to insert.
+	exact := (len(n.inodes) > 0 && index < len(n.inodes) && bytes.Equal(n.inodes[index].key, oldKey))
+	if !exact {
+		n.inodes = append(n.inodes, inode{})
+		copy(n.inodes[index+1:], n.inodes[index:])
+	}
+
+	inode := &n.inodes[index]
+	inode.flags = flags
+	inode.key = newKey
+	inode.value = value
+	inode.pgid = pgid
+	_assert(len(inode.key) > 0, "put: zero-length inode key")
+}
+
+// del removes a key from the node.
+func (n *node) del(key []byte) {
+	// Find index of key.
+	index := sort.Search(len(n.inodes), func(i int) bool { return bytes.Compare(n.inodes[i].key, key) != -1 })
+
+	// Exit if the key isn't found.
+	if index >= len(n.inodes) || !bytes.Equal(n.inodes[index].key, key) {
+		return
+	}
+
+	// Delete inode from the node.
+	n.inodes = append(n.inodes[:index], n.inodes[index+1:]...)
+
+	// Mark the node as needing rebalancing.
+	n.unbalanced = true
+}
+
+// read initializes the node from a page.
+func (n *node) read(p *page) {
+	n.pgid = p.id
+	n.isLeaf = ((p.flags & leafPageFlag) != 0)
+	n.inodes = make(inodes, int(p.count))
+
+	for i := 0; i < int(p.count); i++ {
+		inode := &n.inodes[i]
+		if n.isLeaf {
+			elem := p.leafPageElement(uint16(i))
+			inode.flags = elem.flags
+			inode.key = elem.key()
+			inode.value = elem.value()
+		} else {
+			elem := p.branchPageElement(uint16(i))
+			inode.pgid = elem.pgid
+			inode.key = elem.key()
+		}
+		_assert(len(inode.key) > 0, "read: zero-length inode key")
+	}
+
+	// Save first key so we can find the node in the parent when we spill.
+	if len(n.inodes) > 0 {
+		n.key = n.inodes[0].key
+		_assert(len(n.key) > 0, "read: zero-length node key")
+	} else {
+		n.key = nil
+	}
+}
+
+// write writes the items onto one or more pages.
+func (n *node) write(p *page) {
+	// Initialize page.
+	if n.isLeaf {
+		p.flags |= leafPageFlag
+	} else {
+		p.flags |= branchPageFlag
+	}
+
+	if len(n.inodes) >= 0xFFFF {
+		panic(fmt.Sprintf("inode overflow: %d (pgid=%d)", len(n.inodes), p.id))
+	}
+	p.count = uint16(len(n.inodes))
+
+	// Stop here if there are no items to write.
+	if p.count == 0 {
+		return
+	}
+
+	// Loop over each item and write it to the page.
+	b := (*[maxAllocSize]byte)(unsafe.Pointer(&p.ptr))[n.pageElementSize()*len(n.inodes):]
+	for i, item := range n.inodes {
+		_assert(len(item.key) > 0, "write: zero-length inode key")
+
+		// Write the page element.
+		if n.isLeaf {
+			elem := p.leafPageElement(uint16(i))
+			elem.pos = uint32(uintptr(unsafe.Pointer(&b[0])) - uintptr(unsafe.Pointer(elem)))
+			elem.flags = item.flags
+			elem.ksize = uint32(len(item.key))
+			elem.vsize = uint32(len(item.value))
+		} else {
+			elem := p.branchPageElement(uint16(i))
+			elem.pos = uint32(uintptr(unsafe.Pointer(&b[0])) - uintptr(unsafe.Pointer(elem)))
+			elem.ksize = uint32(len(item.key))
+			elem.pgid = item.pgid
+			_assert(elem.pgid != p.id, "write: circular dependency occurred")
+		}
+
+		// If the length of key+value is larger than the max allocation size
+		// then we need to reallocate the byte array pointer.
+		//
+		// See: https://github.com/boltdb/bolt/pull/335
+		klen, vlen := len(item.key), len(item.value)
+		if len(b) < klen+vlen {
+			b = (*[maxAllocSize]byte)(unsafe.Pointer(&b[0]))[:]
+		}
+
+		// Write data for the element to the end of the page.
+		copy(b[0:], item.key)
+		b = b[klen:]
+		copy(b[0:], item.value)
+		b = b[vlen:]
+	}
+
+	// DEBUG ONLY: n.dump()
+}
+
+// split breaks up a node into multiple smaller nodes, if appropriate.
+// This should only be called from the spill() function.
+func (n *node) split(pageSize int) []*node {
+	var nodes []*node
+
+	node := n
+	for {
+		// Split node into two.
+		a, b := node.splitTwo(pageSize)
+		nodes = append(nodes, a)
+
+		// If we can't split then exit the loop.
+		if b == nil {
+			break
+		}
+
+		// Set node to b so it gets split on the next iteration.
+		node = b
+	}
+
+	return nodes
+}
+
+// splitTwo breaks up a node into two smaller nodes, if appropriate.
+// This should only be called from the split() function.
+func (n *node) splitTwo(pageSize int) (*node, *node) {
+	// Ignore the split if the page doesn't have at least enough nodes for
+	// two pages or if the nodes can fit in a single page.
+	if len(n.inodes) <= (minKeysPerPage*2) || n.sizeLessThan(pageSize) {
+		return n, nil
+	}
+
+	// Determine the threshold before starting a new node.
+	var fillPercent = n.bucket.FillPercent
+	if fillPercent < minFillPercent {
+		fillPercent = minFillPercent
+	} else if fillPercent > maxFillPercent {
+		fillPercent = maxFillPercent
+	}
+	threshold := int(float64(pageSize) * fillPercent)
+
+	// Determine split position and sizes of the two pages.
+	splitIndex, _ := n.splitIndex(threshold)
+
+	// Split node into two separate nodes.
+	// If there's no parent then we'll need to create one.
+	if n.parent == nil {
+		n.parent = &node{bucket: n.bucket, children: []*node{n}}
+	}
+
+	// Create a new node and add it to the parent.
+	next := &node{bucket: n.bucket, isLeaf: n.isLeaf, parent: n.parent}
+	n.parent.children = append(n.parent.children, next)
+
+	// Split inodes across two nodes.
+	next.inodes = n.inodes[splitIndex:]
+	n.inodes = n.inodes[:splitIndex]
+
+	// Update the statistics.
+	n.bucket.tx.stats.Split++
+
+	return n, next
+}
+
+// splitIndex finds the position where a page will fill a given threshold.
+// It returns the index as well as the size of the first page.
+// This is only be called from split().
+func (n *node) splitIndex(threshold int) (index, sz int) {
+	sz = pageHeaderSize
+
+	// Loop until we only have the minimum number of keys required for the second page.
+	for i := 0; i < len(n.inodes)-minKeysPerPage; i++ {
+		index = i
+		inode := n.inodes[i]
+		elsize := n.pageElementSize() + len(inode.key) + len(inode.value)
+
+		// If we have at least the minimum number of keys and adding another
+		// node would put us over the threshold then exit and return.
+		if i >= minKeysPerPage && sz+elsize > threshold {
+			break
+		}
+
+		// Add the element size to the total size.
+		sz += elsize
+	}
+
+	return
+}
+
+// spill writes the nodes to dirty pages and splits nodes as it goes.
+// Returns an error if dirty pages cannot be allocated.
+func (n *node) spill() error {
+	var tx = n.bucket.tx
+	if n.spilled {
+		return nil
+	}
+
+	// Spill child nodes first. Child nodes can materialize sibling nodes in
+	// the case of split-merge so we cannot use a range loop. We have to check
+	// the children size on every loop iteration.
+	sort.Sort(n.children)
+	for i := 0; i < len(n.children); i++ {
+		if err := n.children[i].spill(); err != nil {
+			return err
+		}
+	}
+
+	// We no longer need the child list because it's only used for spill tracking.
+	n.children = nil
+
+	// Split nodes into appropriate sizes. The first node will always be n.
+	var nodes = n.split(tx.db.pageSize)
+	for _, node := range nodes {
+		// Add node's page to the freelist if it's not new.
+		if node.pgid > 0 {
+			tx.db.freelist.free(tx.meta.txid, tx.page(node.pgid))
+			node.pgid = 0
+		}
+
+		// Allocate contiguous space for the node.
+		p, err := tx.allocate((node.size() + tx.db.pageSize - 1) / tx.db.pageSize)
+		if err != nil {
+			return err
+		}
+
+		// Write the node.
+		if p.id >= tx.meta.pgid {
+			panic(fmt.Sprintf("pgid (%d) above high water mark (%d)", p.id, tx.meta.pgid))
+		}
+		node.pgid = p.id
+		node.write(p)
+		node.spilled = true
+
+		// Insert into parent inodes.
+		if node.parent != nil {
+			var key = node.key
+			if key == nil {
+				key = node.inodes[0].key
+			}
+
+			node.parent.put(key, node.inodes[0].key, nil, node.pgid, 0)
+			node.key = node.inodes[0].key
+			_assert(len(node.key) > 0, "spill: zero-length node key")
+		}
+
+		// Update the statistics.
+		tx.stats.Spill++
+	}
+
+	// If the root node split and created a new root then we need to spill that
+	// as well. We'll clear out the children to make sure it doesn't try to respill.
+	if n.parent != nil && n.parent.pgid == 0 {
+		n.children = nil
+		return n.parent.spill()
+	}
+
+	return nil
+}
+
+// rebalance attempts to combine the node with sibling nodes if the node fill
+// size is below a threshold or if there are not enough keys.
+func (n *node) rebalance() {
+	if !n.unbalanced {
+		return
+	}
+	n.unbalanced = false
+
+	// Update statistics.
+	n.bucket.tx.stats.Rebalance++
+
+	// Ignore if node is above threshold (25%) and has enough keys.
+	var threshold = n.bucket.tx.db.pageSize / 4
+	if n.size() > threshold && len(n.inodes) > n.minKeys() {
+		return
+	}
+
+	// Root node has special handling.
+	if n.parent == nil {
+		// If root node is a branch and only has one node then collapse it.
+		if !n.isLeaf && len(n.inodes) == 1 {
+			// Move root's child up.
+			child := n.bucket.node(n.inodes[0].pgid, n)
+			n.isLeaf = child.isLeaf
+			n.inodes = child.inodes[:]
+			n.children = child.children
+
+			// Reparent all child nodes being moved.
+			for _, inode := range n.inodes {
+				if child, ok := n.bucket.nodes[inode.pgid]; ok {
+					child.parent = n
+				}
+			}
+
+			// Remove old child.
+			child.parent = nil
+			delete(n.bucket.nodes, child.pgid)
+			child.free()
+		}
+
+		return
+	}
+
+	// If node has no keys then just remove it.
+	if n.numChildren() == 0 {
+		n.parent.del(n.key)
+		n.parent.removeChild(n)
+		delete(n.bucket.nodes, n.pgid)
+		n.free()
+		n.parent.rebalance()
+		return
+	}
+
+	_assert(n.parent.numChildren() > 1, "parent must have at least 2 children")
+
+	// Destination node is right sibling if idx == 0, otherwise left sibling.
+	var target *node
+	var useNextSibling = (n.parent.childIndex(n) == 0)
+	if useNextSibling {
+		target = n.nextSibling()
+	} else {
+		target = n.prevSibling()
+	}
+
+	// If both this node and the target node are too small then merge them.
+	if useNextSibling {
+		// Reparent all child nodes being moved.
+		for _, inode := range target.inodes {
+			if child, ok := n.bucket.nodes[inode.pgid]; ok {
+				child.parent.removeChild(child)
+				child.parent = n
+				child.parent.children = append(child.parent.children, child)
+			}
+		}
+
+		// Copy over inodes from target and remove target.
+		n.inodes = append(n.inodes, target.inodes...)
+		n.parent.del(target.key)
+		n.parent.removeChild(target)
+		delete(n.bucket.nodes, target.pgid)
+		target.free()
+	} else {
+		// Reparent all child nodes being moved.
+		for _, inode := range n.inodes {
+			if child, ok := n.bucket.nodes[inode.pgid]; ok {
+				child.parent.removeChild(child)
+				child.parent = target
+				child.parent.children = append(child.parent.children, child)
+			}
+		}
+
+		// Copy over inodes to target and remove node.
+		target.inodes = append(target.inodes, n.inodes...)
+		n.parent.del(n.key)
+		n.parent.removeChild(n)
+		delete(n.bucket.nodes, n.pgid)
+		n.free()
+	}
+
+	// Either this node or the target node was deleted from the parent so rebalance it.
+	n.parent.rebalance()
+}
+
+// removes a node from the list of in-memory children.
+// This does not affect the inodes.
+func (n *node) removeChild(target *node) {
+	for i, child := range n.children {
+		if child == target {
+			n.children = append(n.children[:i], n.children[i+1:]...)
+			return
+		}
+	}
+}
+
+// dereference causes the node to copy all its inode key/value references to heap memory.
+// This is required when the mmap is reallocated so inodes are not pointing to stale data.
+func (n *node) dereference() {
+	if n.key != nil {
+		key := make([]byte, len(n.key))
+		copy(key, n.key)
+		n.key = key
+		_assert(n.pgid == 0 || len(n.key) > 0, "dereference: zero-length node key on existing node")
+	}
+
+	for i := range n.inodes {
+		inode := &n.inodes[i]
+
+		key := make([]byte, len(inode.key))
+		copy(key, inode.key)
+		inode.key = key
+		_assert(len(inode.key) > 0, "dereference: zero-length inode key")
+
+		value := make([]byte, len(inode.value))
+		copy(value, inode.value)
+		inode.value = value
+	}
+
+	// Recursively dereference children.
+	for _, child := range n.children {
+		child.dereference()
+	}
+
+	// Update statistics.
+	n.bucket.tx.stats.NodeDeref++
+}
+
+// free adds the node's underlying page to the freelist.
+func (n *node) free() {
+	if n.pgid != 0 {
+		n.bucket.tx.db.freelist.free(n.bucket.tx.meta.txid, n.bucket.tx.page(n.pgid))
+		n.pgid = 0
+	}
+}
+
+// dump writes the contents of the node to STDERR for debugging purposes.
+/*
+func (n *node) dump() {
+	// Write node header.
+	var typ = "branch"
+	if n.isLeaf {
+		typ = "leaf"
+	}
+	warnf("[NODE %d {type=%s count=%d}]", n.pgid, typ, len(n.inodes))
+
+	// Write out abbreviated version of each item.
+	for _, item := range n.inodes {
+		if n.isLeaf {
+			if item.flags&bucketLeafFlag != 0 {
+				bucket := (*bucket)(unsafe.Pointer(&item.value[0]))
+				warnf("+L %08x -> (bucket root=%d)", trunc(item.key, 4), bucket.root)
+			} else {
+				warnf("+L %08x -> %08x", trunc(item.key, 4), trunc(item.value, 4))
+			}
+		} else {
+			warnf("+B %08x -> pgid=%d", trunc(item.key, 4), item.pgid)
+		}
+	}
+	warn("")
+}
+*/
+
+type nodes []*node
+
+func (s nodes) Len() int           { return len(s) }
+func (s nodes) Swap(i, j int)      { s[i], s[j] = s[j], s[i] }
+func (s nodes) Less(i, j int) bool { return bytes.Compare(s[i].inodes[0].key, s[j].inodes[0].key) == -1 }
+
+// inode represents an internal node inside of a node.
+// It can be used to point to elements in a page or point
+// to an element which hasn't been added to a page yet.
+type inode struct {
+	flags uint32
+	pgid  pgid
+	key   []byte
+	value []byte
+}
+
+type inodes []inode
diff --git a/vendor/go.etcd.io/bbolt/page.go b/vendor/go.etcd.io/bbolt/page.go
new file mode 100644
index 0000000..bca9615
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/page.go
@@ -0,0 +1,197 @@
+package bbolt
+
+import (
+	"fmt"
+	"os"
+	"sort"
+	"unsafe"
+)
+
+const pageHeaderSize = int(unsafe.Offsetof(((*page)(nil)).ptr))
+
+const minKeysPerPage = 2
+
+const branchPageElementSize = int(unsafe.Sizeof(branchPageElement{}))
+const leafPageElementSize = int(unsafe.Sizeof(leafPageElement{}))
+
+const (
+	branchPageFlag   = 0x01
+	leafPageFlag     = 0x02
+	metaPageFlag     = 0x04
+	freelistPageFlag = 0x10
+)
+
+const (
+	bucketLeafFlag = 0x01
+)
+
+type pgid uint64
+
+type page struct {
+	id       pgid
+	flags    uint16
+	count    uint16
+	overflow uint32
+	ptr      uintptr
+}
+
+// typ returns a human readable page type string used for debugging.
+func (p *page) typ() string {
+	if (p.flags & branchPageFlag) != 0 {
+		return "branch"
+	} else if (p.flags & leafPageFlag) != 0 {
+		return "leaf"
+	} else if (p.flags & metaPageFlag) != 0 {
+		return "meta"
+	} else if (p.flags & freelistPageFlag) != 0 {
+		return "freelist"
+	}
+	return fmt.Sprintf("unknown<%02x>", p.flags)
+}
+
+// meta returns a pointer to the metadata section of the page.
+func (p *page) meta() *meta {
+	return (*meta)(unsafe.Pointer(&p.ptr))
+}
+
+// leafPageElement retrieves the leaf node by index
+func (p *page) leafPageElement(index uint16) *leafPageElement {
+	n := &((*[0x7FFFFFF]leafPageElement)(unsafe.Pointer(&p.ptr)))[index]
+	return n
+}
+
+// leafPageElements retrieves a list of leaf nodes.
+func (p *page) leafPageElements() []leafPageElement {
+	if p.count == 0 {
+		return nil
+	}
+	return ((*[0x7FFFFFF]leafPageElement)(unsafe.Pointer(&p.ptr)))[:]
+}
+
+// branchPageElement retrieves the branch node by index
+func (p *page) branchPageElement(index uint16) *branchPageElement {
+	return &((*[0x7FFFFFF]branchPageElement)(unsafe.Pointer(&p.ptr)))[index]
+}
+
+// branchPageElements retrieves a list of branch nodes.
+func (p *page) branchPageElements() []branchPageElement {
+	if p.count == 0 {
+		return nil
+	}
+	return ((*[0x7FFFFFF]branchPageElement)(unsafe.Pointer(&p.ptr)))[:]
+}
+
+// dump writes n bytes of the page to STDERR as hex output.
+func (p *page) hexdump(n int) {
+	buf := (*[maxAllocSize]byte)(unsafe.Pointer(p))[:n]
+	fmt.Fprintf(os.Stderr, "%x\n", buf)
+}
+
+type pages []*page
+
+func (s pages) Len() int           { return len(s) }
+func (s pages) Swap(i, j int)      { s[i], s[j] = s[j], s[i] }
+func (s pages) Less(i, j int) bool { return s[i].id < s[j].id }
+
+// branchPageElement represents a node on a branch page.
+type branchPageElement struct {
+	pos   uint32
+	ksize uint32
+	pgid  pgid
+}
+
+// key returns a byte slice of the node key.
+func (n *branchPageElement) key() []byte {
+	buf := (*[maxAllocSize]byte)(unsafe.Pointer(n))
+	return (*[maxAllocSize]byte)(unsafe.Pointer(&buf[n.pos]))[:n.ksize]
+}
+
+// leafPageElement represents a node on a leaf page.
+type leafPageElement struct {
+	flags uint32
+	pos   uint32
+	ksize uint32
+	vsize uint32
+}
+
+// key returns a byte slice of the node key.
+func (n *leafPageElement) key() []byte {
+	buf := (*[maxAllocSize]byte)(unsafe.Pointer(n))
+	return (*[maxAllocSize]byte)(unsafe.Pointer(&buf[n.pos]))[:n.ksize:n.ksize]
+}
+
+// value returns a byte slice of the node value.
+func (n *leafPageElement) value() []byte {
+	buf := (*[maxAllocSize]byte)(unsafe.Pointer(n))
+	return (*[maxAllocSize]byte)(unsafe.Pointer(&buf[n.pos+n.ksize]))[:n.vsize:n.vsize]
+}
+
+// PageInfo represents human readable information about a page.
+type PageInfo struct {
+	ID            int
+	Type          string
+	Count         int
+	OverflowCount int
+}
+
+type pgids []pgid
+
+func (s pgids) Len() int           { return len(s) }
+func (s pgids) Swap(i, j int)      { s[i], s[j] = s[j], s[i] }
+func (s pgids) Less(i, j int) bool { return s[i] < s[j] }
+
+// merge returns the sorted union of a and b.
+func (a pgids) merge(b pgids) pgids {
+	// Return the opposite slice if one is nil.
+	if len(a) == 0 {
+		return b
+	}
+	if len(b) == 0 {
+		return a
+	}
+	merged := make(pgids, len(a)+len(b))
+	mergepgids(merged, a, b)
+	return merged
+}
+
+// mergepgids copies the sorted union of a and b into dst.
+// If dst is too small, it panics.
+func mergepgids(dst, a, b pgids) {
+	if len(dst) < len(a)+len(b) {
+		panic(fmt.Errorf("mergepgids bad len %d < %d + %d", len(dst), len(a), len(b)))
+	}
+	// Copy in the opposite slice if one is nil.
+	if len(a) == 0 {
+		copy(dst, b)
+		return
+	}
+	if len(b) == 0 {
+		copy(dst, a)
+		return
+	}
+
+	// Merged will hold all elements from both lists.
+	merged := dst[:0]
+
+	// Assign lead to the slice with a lower starting value, follow to the higher value.
+	lead, follow := a, b
+	if b[0] < a[0] {
+		lead, follow = b, a
+	}
+
+	// Continue while there are elements in the lead.
+	for len(lead) > 0 {
+		// Merge largest prefix of lead that is ahead of follow[0].
+		n := sort.Search(len(lead), func(i int) bool { return lead[i] > follow[0] })
+		merged = append(merged, lead[:n]...)
+		if n >= len(lead) {
+			break
+		}
+
+		// Swap lead and follow.
+		lead, follow = follow, lead[n:]
+	}
+
+	// Append what's left in follow.
+	_ = append(merged, follow...)
+}
diff --git a/vendor/go.etcd.io/bbolt/tx.go b/vendor/go.etcd.io/bbolt/tx.go
new file mode 100644
index 0000000..2df7688
--- /dev/null
+++ b/vendor/go.etcd.io/bbolt/tx.go
@@ -0,0 +1,726 @@
+package bbolt
+
+import (
+	"fmt"
+	"io"
+	"os"
+	"sort"
+	"strings"
+	"time"
+	"unsafe"
+)
+
+// txid represents the internal transaction identifier.
+type txid uint64
+
+// Tx represents a read-only or read/write transaction on the database.
+// Read-only transactions can be used for retrieving values for keys and creating cursors.
+// Read/write transactions can create and remove buckets and create and remove keys.
+//
+// IMPORTANT: You must commit or rollback transactions when you are done with
+// them. Pages can not be reclaimed by the writer until no more transactions
+// are using them. A long running read transaction can cause the database to
+// quickly grow.
+type Tx struct {
+	writable       bool
+	managed        bool
+	db             *DB
+	meta           *meta
+	root           Bucket
+	pages          map[pgid]*page
+	stats          TxStats
+	commitHandlers []func()
+
+	// WriteFlag specifies the flag for write-related methods like WriteTo().
+	// Tx opens the database file with the specified flag to copy the data.
+	//
+	// By default, the flag is unset, which works well for mostly in-memory
+	// workloads. For databases that are much larger than available RAM,
+	// set the flag to syscall.O_DIRECT to avoid trashing the page cache.
+	WriteFlag int
+}
+
+// init initializes the transaction.
+func (tx *Tx) init(db *DB) {
+	tx.db = db
+	tx.pages = nil
+
+	// Copy the meta page since it can be changed by the writer.
+	tx.meta = &meta{}
+	db.meta().copy(tx.meta)
+
+	// Copy over the root bucket.
+	tx.root = newBucket(tx)
+	tx.root.bucket = &bucket{}
+	*tx.root.bucket = tx.meta.root
+
+	// Increment the transaction id and add a page cache for writable transactions.
+	if tx.writable {
+		tx.pages = make(map[pgid]*page)
+		tx.meta.txid += txid(1)
+	}
+}
+
+// ID returns the transaction id.
+func (tx *Tx) ID() int {
+	return int(tx.meta.txid)
+}
+
+// DB returns a reference to the database that created the transaction.
+func (tx *Tx) DB() *DB {
+	return tx.db
+}
+
+// Size returns current database size in bytes as seen by this transaction.
+func (tx *Tx) Size() int64 {
+	return int64(tx.meta.pgid) * int64(tx.db.pageSize)
+}
+
+// Writable returns whether the transaction can perform write operations.
+func (tx *Tx) Writable() bool {
+	return tx.writable
+}
+
+// Cursor creates a cursor associated with the root bucket.
+// All items in the cursor will return a nil value because all root bucket keys point to buckets.
+// The cursor is only valid as long as the transaction is open.
+// Do not use a cursor after the transaction is closed.
+func (tx *Tx) Cursor() *Cursor {
+	return tx.root.Cursor()
+}
+
+// Stats retrieves a copy of the current transaction statistics.
+func (tx *Tx) Stats() TxStats {
+	return tx.stats
+}
+
+// Bucket retrieves a bucket by name.
+// Returns nil if the bucket does not exist.
+// The bucket instance is only valid for the lifetime of the transaction.
+func (tx *Tx) Bucket(name []byte) *Bucket {
+	return tx.root.Bucket(name)
+}
+
+// CreateBucket creates a new bucket.
+// Returns an error if the bucket already exists, if the bucket name is blank, or if the bucket name is too long.
+// The bucket instance is only valid for the lifetime of the transaction.
+func (tx *Tx) CreateBucket(name []byte) (*Bucket, error) {
+	return tx.root.CreateBucket(name)
+}
+
+// CreateBucketIfNotExists creates a new bucket if it doesn't already exist.
+// Returns an error if the bucket name is blank, or if the bucket name is too long.
+// The bucket instance is only valid for the lifetime of the transaction.
+func (tx *Tx) CreateBucketIfNotExists(name []byte) (*Bucket, error) {
+	return tx.root.CreateBucketIfNotExists(name)
+}
+
+// DeleteBucket deletes a bucket.
+// Returns an error if the bucket cannot be found or if the key represents a non-bucket value.
+func (tx *Tx) DeleteBucket(name []byte) error {
+	return tx.root.DeleteBucket(name)
+}
+
+// ForEach executes a function for each bucket in the root.
+// If the provided function returns an error then the iteration is stopped and
+// the error is returned to the caller.
+func (tx *Tx) ForEach(fn func(name []byte, b *Bucket) error) error {
+	return tx.root.ForEach(func(k, v []byte) error {
+		return fn(k, tx.root.Bucket(k))
+	})
+}
+
+// OnCommit adds a handler function to be executed after the transaction successfully commits.
+func (tx *Tx) OnCommit(fn func()) {
+	tx.commitHandlers = append(tx.commitHandlers, fn)
+}
+
+// Commit writes all changes to disk and updates the meta page.
+// Returns an error if a disk write error occurs, or if Commit is
+// called on a read-only transaction.
+func (tx *Tx) Commit() error {
+	_assert(!tx.managed, "managed tx commit not allowed")
+	if tx.db == nil {
+		return ErrTxClosed
+	} else if !tx.writable {
+		return ErrTxNotWritable
+	}
+
+	// TODO(benbjohnson): Use vectorized I/O to write out dirty pages.
+
+	// Rebalance nodes which have had deletions.
+	var startTime = time.Now()
+	tx.root.rebalance()
+	if tx.stats.Rebalance > 0 {
+		tx.stats.RebalanceTime += time.Since(startTime)
+	}
+
+	// spill data onto dirty pages.
+	startTime = time.Now()
+	if err := tx.root.spill(); err != nil {
+		tx.rollback()
+		return err
+	}
+	tx.stats.SpillTime += time.Since(startTime)
+
+	// Free the old root bucket.
+	tx.meta.root.root = tx.root.root
+
+	// Free the old freelist because commit writes out a fresh freelist.
+	if tx.meta.freelist != pgidNoFreelist {
+		tx.db.freelist.free(tx.meta.txid, tx.db.page(tx.meta.freelist))
+	}
+
+	if !tx.db.NoFreelistSync {
+		err := tx.commitFreelist()
+		if err != nil {
+			return err
+		}
+	} else {
+		tx.meta.freelist = pgidNoFreelist
+	}
+
+	// Write dirty pages to disk.
+	startTime = time.Now()
+	if err := tx.write(); err != nil {
+		tx.rollback()
+		return err
+	}
+
+	// If strict mode is enabled then perform a consistency check.
+	// Only the first consistency error is reported in the panic.
+	if tx.db.StrictMode {
+		ch := tx.Check()
+		var errs []string
+		for {
+			err, ok := <-ch
+			if !ok {
+				break
+			}
+			errs = append(errs, err.Error())
+		}
+		if len(errs) > 0 {
+			panic("check fail: " + strings.Join(errs, "\n"))
+		}
+	}
+
+	// Write meta to disk.
+	if err := tx.writeMeta(); err != nil {
+		tx.rollback()
+		return err
+	}
+	tx.stats.WriteTime += time.Since(startTime)
+
+	// Finalize the transaction.
+	tx.close()
+
+	// Execute commit handlers now that the locks have been removed.
+	for _, fn := range tx.commitHandlers {
+		fn()
+	}
+
+	return nil
+}
+
+func (tx *Tx) commitFreelist() error {
+	// Allocate new pages for the new free list. This will overestimate
+	// the size of the freelist but not underestimate the size (which would be bad).
+	opgid := tx.meta.pgid
+	p, err := tx.allocate((tx.db.freelist.size() / tx.db.pageSize) + 1)
+	if err != nil {
+		tx.rollback()
+		return err
+	}
+	if err := tx.db.freelist.write(p); err != nil {
+		tx.rollback()
+		return err
+	}
+	tx.meta.freelist = p.id
+	// If the high water mark has moved up then attempt to grow the database.
+	if tx.meta.pgid > opgid {
+		if err := tx.db.grow(int(tx.meta.pgid+1) * tx.db.pageSize); err != nil {
+			tx.rollback()
+			return err
+		}
+	}
+
+	return nil
+}
+
+// Rollback closes the transaction and ignores all previous updates. Read-only
+// transactions must be rolled back and not committed.
+func (tx *Tx) Rollback() error {
+	_assert(!tx.managed, "managed tx rollback not allowed")
+	if tx.db == nil {
+		return ErrTxClosed
+	}
+	tx.nonPhysicalRollback()
+	return nil
+}
+
+// nonPhysicalRollback is called when user calls Rollback directly, in this case we do not need to reload the free pages from disk.
+func (tx *Tx) nonPhysicalRollback() {
+	if tx.db == nil {
+		return
+	}
+	if tx.writable {
+		tx.db.freelist.rollback(tx.meta.txid)
+	}
+	tx.close()
+}
+
+// rollback needs to reload the free pages from disk in case some system error happens like fsync error.
+func (tx *Tx) rollback() {
+	if tx.db == nil {
+		return
+	}
+	if tx.writable {
+		tx.db.freelist.rollback(tx.meta.txid)
+		if !tx.db.hasSyncedFreelist() {
+			// Reconstruct free page list by scanning the DB to get the whole free page list.
+			// Note: scaning the whole db is heavy if your db size is large in NoSyncFreeList mode.
+			tx.db.freelist.noSyncReload(tx.db.freepages())
+		} else {
+			// Read free page list from freelist page.
+			tx.db.freelist.reload(tx.db.page(tx.db.meta().freelist))
+		}
+	}
+	tx.close()
+}
+
+func (tx *Tx) close() {
+	if tx.db == nil {
+		return
+	}
+	if tx.writable {
+		// Grab freelist stats.
+		var freelistFreeN = tx.db.freelist.free_count()
+		var freelistPendingN = tx.db.freelist.pending_count()
+		var freelistAlloc = tx.db.freelist.size()
+
+		// Remove transaction ref & writer lock.
+		tx.db.rwtx = nil
+		tx.db.rwlock.Unlock()
+
+		// Merge statistics.
+		tx.db.statlock.Lock()
+		tx.db.stats.FreePageN = freelistFreeN
+		tx.db.stats.PendingPageN = freelistPendingN
+		tx.db.stats.FreeAlloc = (freelistFreeN + freelistPendingN) * tx.db.pageSize
+		tx.db.stats.FreelistInuse = freelistAlloc
+		tx.db.stats.TxStats.add(&tx.stats)
+		tx.db.statlock.Unlock()
+	} else {
+		tx.db.removeTx(tx)
+	}
+
+	// Clear all references.
+	tx.db = nil
+	tx.meta = nil
+	tx.root = Bucket{tx: tx}
+	tx.pages = nil
+}
+
+// Copy writes the entire database to a writer.
+// This function exists for backwards compatibility.
+//
+// Deprecated; Use WriteTo() instead.
+func (tx *Tx) Copy(w io.Writer) error {
+	_, err := tx.WriteTo(w)
+	return err
+}
+
+// WriteTo writes the entire database to a writer.
+// If err == nil then exactly tx.Size() bytes will be written into the writer.
+func (tx *Tx) WriteTo(w io.Writer) (n int64, err error) {
+	// Attempt to open reader with WriteFlag
+	f, err := tx.db.openFile(tx.db.path, os.O_RDONLY|tx.WriteFlag, 0)
+	if err != nil {
+		return 0, err
+	}
+	defer func() {
+		if cerr := f.Close(); err == nil {
+			err = cerr
+		}
+	}()
+
+	// Generate a meta page. We use the same page data for both meta pages.
+	buf := make([]byte, tx.db.pageSize)
+	page := (*page)(unsafe.Pointer(&buf[0]))
+	page.flags = metaPageFlag
+	*page.meta() = *tx.meta
+
+	// Write meta 0.
+	page.id = 0
+	page.meta().checksum = page.meta().sum64()
+	nn, err := w.Write(buf)
+	n += int64(nn)
+	if err != nil {
+		return n, fmt.Errorf("meta 0 copy: %s", err)
+	}
+
+	// Write meta 1 with a lower transaction id.
+	page.id = 1
+	page.meta().txid -= 1
+	page.meta().checksum = page.meta().sum64()
+	nn, err = w.Write(buf)
+	n += int64(nn)
+	if err != nil {
+		return n, fmt.Errorf("meta 1 copy: %s", err)
+	}
+
+	// Move past the meta pages in the file.
+	if _, err := f.Seek(int64(tx.db.pageSize*2), io.SeekStart); err != nil {
+		return n, fmt.Errorf("seek: %s", err)
+	}
+
+	// Copy data pages.
+	wn, err := io.CopyN(w, f, tx.Size()-int64(tx.db.pageSize*2))
+	n += wn
+	if err != nil {
+		return n, err
+	}
+
+	return n, nil
+}
+
+// CopyFile copies the entire database to file at the given path.
+// A reader transaction is maintained during the copy so it is safe to continue
+// using the database while a copy is in progress.
+func (tx *Tx) CopyFile(path string, mode os.FileMode) error {
+	f, err := tx.db.openFile(path, os.O_RDWR|os.O_CREATE|os.O_TRUNC, mode)
+	if err != nil {
+		return err
+	}
+
+	err = tx.Copy(f)
+	if err != nil {
+		_ = f.Close()
+		return err
+	}
+	return f.Close()
+}
+
+// Check performs several consistency checks on the database for this transaction.
+// An error is returned if any inconsistency is found.
+//
+// It can be safely run concurrently on a writable transaction. However, this
+// incurs a high cost for large databases and databases with a lot of subbuckets
+// because of caching. This overhead can be removed if running on a read-only
+// transaction, however, it is not safe to execute other writer transactions at
+// the same time.
+func (tx *Tx) Check() <-chan error {
+	ch := make(chan error)
+	go tx.check(ch)
+	return ch
+}
+
+func (tx *Tx) check(ch chan error) {
+	// Force loading free list if opened in ReadOnly mode.
+	tx.db.loadFreelist()
+
+	// Check if any pages are double freed.
+	freed := make(map[pgid]bool)
+	all := make([]pgid, tx.db.freelist.count())
+	tx.db.freelist.copyall(all)
+	for _, id := range all {
+		if freed[id] {
+			ch <- fmt.Errorf("page %d: already freed", id)
+		}
+		freed[id] = true
+	}
+
+	// Track every reachable page.
+	reachable := make(map[pgid]*page)
+	reachable[0] = tx.page(0) // meta0
+	reachable[1] = tx.page(1) // meta1
+	if tx.meta.freelist != pgidNoFreelist {
+		for i := uint32(0); i <= tx.page(tx.meta.freelist).overflow; i++ {
+			reachable[tx.meta.freelist+pgid(i)] = tx.page(tx.meta.freelist)
+		}
+	}
+
+	// Recursively check buckets.
+	tx.checkBucket(&tx.root, reachable, freed, ch)
+
+	// Ensure all pages below high water mark are either reachable or freed.
+	for i := pgid(0); i < tx.meta.pgid; i++ {
+		_, isReachable := reachable[i]
+		if !isReachable && !freed[i] {
+			ch <- fmt.Errorf("page %d: unreachable unfreed", int(i))
+		}
+	}
+
+	// Close the channel to signal completion.
+	close(ch)
+}
+
+func (tx *Tx) checkBucket(b *Bucket, reachable map[pgid]*page, freed map[pgid]bool, ch chan error) {
+	// Ignore inline buckets.
+	if b.root == 0 {
+		return
+	}
+
+	// Check every page used by this bucket.
+	b.tx.forEachPage(b.root, 0, func(p *page, _ int) {
+		if p.id > tx.meta.pgid {
+			ch <- fmt.Errorf("page %d: out of bounds: %d", int(p.id), int(b.tx.meta.pgid))
+		}
+
+		// Ensure each page is only referenced once.
+		for i := pgid(0); i <= pgid(p.overflow); i++ {
+			var id = p.id + i
+			if _, ok := reachable[id]; ok {
+				ch <- fmt.Errorf("page %d: multiple references", int(id))
+			}
+			reachable[id] = p
+		}
+
+		// We should only encounter un-freed leaf and branch pages.
+		if freed[p.id] {
+			ch <- fmt.Errorf("page %d: reachable freed", int(p.id))
+		} else if (p.flags&branchPageFlag) == 0 && (p.flags&leafPageFlag) == 0 {
+			ch <- fmt.Errorf("page %d: invalid type: %s", int(p.id), p.typ())
+		}
+	})
+
+	// Check each bucket within this bucket.
+	_ = b.ForEach(func(k, v []byte) error {
+		if child := b.Bucket(k); child != nil {
+			tx.checkBucket(child, reachable, freed, ch)
+		}
+		return nil
+	})
+}
+
+// allocate returns a contiguous block of memory starting at a given page.
+func (tx *Tx) allocate(count int) (*page, error) {
+	p, err := tx.db.allocate(tx.meta.txid, count)
+	if err != nil {
+		return nil, err
+	}
+
+	// Save to our page cache.
+	tx.pages[p.id] = p
+
+	// Update statistics.
+	tx.stats.PageCount += count
+	tx.stats.PageAlloc += count * tx.db.pageSize
+
+	return p, nil
+}
+
+// write writes any dirty pages to disk.
+func (tx *Tx) write() error {
+	// Sort pages by id.
+	pages := make(pages, 0, len(tx.pages))
+	for _, p := range tx.pages {
+		pages = append(pages, p)
+	}
+	// Clear out page cache early.
+	tx.pages = make(map[pgid]*page)
+	sort.Sort(pages)
+
+	// Write pages to disk in order.
+	for _, p := range pages {
+		size := (int(p.overflow) + 1) * tx.db.pageSize
+		offset := int64(p.id) * int64(tx.db.pageSize)
+
+		// Write out page in "max allocation" sized chunks.
+		ptr := (*[maxAllocSize]byte)(unsafe.Pointer(p))
+		for {
+			// Limit our write to our max allocation size.
+			sz := size
+			if sz > maxAllocSize-1 {
+				sz = maxAllocSize - 1
+			}
+
+			// Write chunk to disk.
+			buf := ptr[:sz]
+			if _, err := tx.db.ops.writeAt(buf, offset); err != nil {
+				return err
+			}
+
+			// Update statistics.
+			tx.stats.Write++
+
+			// Exit inner for loop if we've written all the chunks.
+			size -= sz
+			if size == 0 {
+				break
+			}
+
+			// Otherwise move offset forward and move pointer to next chunk.
+			offset += int64(sz)
+			ptr = (*[maxAllocSize]byte)(unsafe.Pointer(&ptr[sz]))
+		}
+	}
+
+	// Ignore file sync if flag is set on DB.
+	if !tx.db.NoSync || IgnoreNoSync {
+		if err := fdatasync(tx.db); err != nil {
+			return err
+		}
+	}
+
+	// Put small pages back to page pool.
+	for _, p := range pages {
+		// Ignore page sizes over 1 page.
+		// These are allocated using make() instead of the page pool.
+		if int(p.overflow) != 0 {
+			continue
+		}
+
+		buf := (*[maxAllocSize]byte)(unsafe.Pointer(p))[:tx.db.pageSize]
+
+		// See https://go.googlesource.com/go/+/f03c9202c43e0abb130669852082117ca50aa9b1
+		for i := range buf {
+			buf[i] = 0
+		}
+		tx.db.pagePool.Put(buf)
+	}
+
+	return nil
+}
+
+// writeMeta writes the meta to the disk.
+func (tx *Tx) writeMeta() error {
+	// Create a temporary buffer for the meta page.
+	buf := make([]byte, tx.db.pageSize)
+	p := tx.db.pageInBuffer(buf, 0)
+	tx.meta.write(p)
+
+	// Write the meta page to file.
+	if _, err := tx.db.ops.writeAt(buf, int64(p.id)*int64(tx.db.pageSize)); err != nil {
+		return err
+	}
+	if !tx.db.NoSync || IgnoreNoSync {
+		if err := fdatasync(tx.db); err != nil {
+			return err
+		}
+	}
+
+	// Update statistics.
+	tx.stats.Write++
+
+	return nil
+}
+
+// page returns a reference to the page with a given id.
+// If page has been written to then a temporary buffered page is returned.
+func (tx *Tx) page(id pgid) *page {
+	// Check the dirty pages first.
+	if tx.pages != nil {
+		if p, ok := tx.pages[id]; ok {
+			return p
+		}
+	}
+
+	// Otherwise return directly from the mmap.
+	return tx.db.page(id)
+}
+
+// forEachPage iterates over every page within a given page and executes a function.
+func (tx *Tx) forEachPage(pgid pgid, depth int, fn func(*page, int)) {
+	p := tx.page(pgid)
+
+	// Execute function.
+	fn(p, depth)
+
+	// Recursively loop over children.
+	if (p.flags & branchPageFlag) != 0 {
+		for i := 0; i < int(p.count); i++ {
+			elem := p.branchPageElement(uint16(i))
+			tx.forEachPage(elem.pgid, depth+1, fn)
+		}
+	}
+}
+
+// Page returns page information for a given page number.
+// This is only safe for concurrent use when used by a writable transaction.
+func (tx *Tx) Page(id int) (*PageInfo, error) {
+	if tx.db == nil {
+		return nil, ErrTxClosed
+	} else if pgid(id) >= tx.meta.pgid {
+		return nil, nil
+	}
+
+	// Build the page info.
+	p := tx.db.page(pgid(id))
+	info := &PageInfo{
+		ID:            id,
+		Count:         int(p.count),
+		OverflowCount: int(p.overflow),
+	}
+
+	// Determine the type (or if it's free).
+	if tx.db.freelist.freed(pgid(id)) {
+		info.Type = "free"
+	} else {
+		info.Type = p.typ()
+	}
+
+	return info, nil
+}
+
+// TxStats represents statistics about the actions performed by the transaction.
+type TxStats struct {
+	// Page statistics.
+	PageCount int // number of page allocations
+	PageAlloc int // total bytes allocated
+
+	// Cursor statistics.
+	CursorCount int // number of cursors created
+
+	// Node statistics
+	NodeCount int // number of node allocations
+	NodeDeref int // number of node dereferences
+
+	// Rebalance statistics.
+	Rebalance     int           // number of node rebalances
+	RebalanceTime time.Duration // total time spent rebalancing
+
+	// Split/Spill statistics.
+	Split     int           // number of nodes split
+	Spill     int           // number of nodes spilled
+	SpillTime time.Duration // total time spent spilling
+
+	// Write statistics.
+	Write     int           // number of writes performed
+	WriteTime time.Duration // total time spent writing to disk
+}
+
+func (s *TxStats) add(other *TxStats) {
+	s.PageCount += other.PageCount
+	s.PageAlloc += other.PageAlloc
+	s.CursorCount += other.CursorCount
+	s.NodeCount += other.NodeCount
+	s.NodeDeref += other.NodeDeref
+	s.Rebalance += other.Rebalance
+	s.RebalanceTime += other.RebalanceTime
+	s.Split += other.Split
+	s.Spill += other.Spill
+	s.SpillTime += other.SpillTime
+	s.Write += other.Write
+	s.WriteTime += other.WriteTime
+}
+
+// Sub calculates and returns the difference between two sets of transaction stats.
+// This is useful when obtaining stats at two different points and time and
+// you need the performance counters that occurred within that time span.
+func (s *TxStats) Sub(other *TxStats) TxStats {
+	var diff TxStats
+	diff.PageCount = s.PageCount - other.PageCount
+	diff.PageAlloc = s.PageAlloc - other.PageAlloc
+	diff.CursorCount = s.CursorCount - other.CursorCount
+	diff.NodeCount = s.NodeCount - other.NodeCount
+	diff.NodeDeref = s.NodeDeref - other.NodeDeref
+	diff.Rebalance = s.Rebalance - other.Rebalance
+	diff.RebalanceTime = s.RebalanceTime - other.RebalanceTime
+	diff.Split = s.Split - other.Split
+	diff.Spill = s.Spill - other.Spill
+	diff.SpillTime = s.SpillTime - other.SpillTime
+	diff.Write = s.Write - other.Write
+	diff.WriteTime = s.WriteTime - other.WriteTime
+	return diff
+}
diff --git a/vendor/go.etcd.io/etcd/auth/doc.go b/vendor/go.etcd.io/etcd/auth/doc.go
new file mode 100644
index 0000000..72741a1
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/auth/doc.go
@@ -0,0 +1,16 @@
+// Copyright 2016 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 auth provides client role authentication for accessing keys in etcd.
+package auth
diff --git a/vendor/go.etcd.io/etcd/auth/jwt.go b/vendor/go.etcd.io/etcd/auth/jwt.go
new file mode 100644
index 0000000..c22ef89
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/auth/jwt.go
@@ -0,0 +1,184 @@
+// 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 auth
+
+import (
+	"context"
+	"crypto/ecdsa"
+	"crypto/rsa"
+	"errors"
+	"time"
+
+	jwt "github.com/dgrijalva/jwt-go"
+	"go.uber.org/zap"
+)
+
+type tokenJWT struct {
+	lg         *zap.Logger
+	signMethod jwt.SigningMethod
+	key        interface{}
+	ttl        time.Duration
+	verifyOnly bool
+}
+
+func (t *tokenJWT) enable()                         {}
+func (t *tokenJWT) disable()                        {}
+func (t *tokenJWT) invalidateUser(string)           {}
+func (t *tokenJWT) genTokenPrefix() (string, error) { return "", nil }
+
+func (t *tokenJWT) info(ctx context.Context, token string, rev uint64) (*AuthInfo, bool) {
+	// rev isn't used in JWT, it is only used in simple token
+	var (
+		username string
+		revision uint64
+	)
+
+	parsed, err := jwt.Parse(token, func(token *jwt.Token) (interface{}, error) {
+		if token.Method.Alg() != t.signMethod.Alg() {
+			return nil, errors.New("invalid signing method")
+		}
+		switch k := t.key.(type) {
+		case *rsa.PrivateKey:
+			return &k.PublicKey, nil
+		case *ecdsa.PrivateKey:
+			return &k.PublicKey, nil
+		default:
+			return t.key, nil
+		}
+	})
+
+	if err != nil {
+		if t.lg != nil {
+			t.lg.Warn(
+				"failed to parse a JWT token",
+				zap.String("token", token),
+				zap.Error(err),
+			)
+		} else {
+			plog.Warningf("failed to parse jwt token: %s", err)
+		}
+		return nil, false
+	}
+
+	claims, ok := parsed.Claims.(jwt.MapClaims)
+	if !parsed.Valid || !ok {
+		if t.lg != nil {
+			t.lg.Warn("invalid JWT token", zap.String("token", token))
+		} else {
+			plog.Warningf("invalid jwt token: %s", token)
+		}
+		return nil, false
+	}
+
+	username = claims["username"].(string)
+	revision = uint64(claims["revision"].(float64))
+
+	return &AuthInfo{Username: username, Revision: revision}, true
+}
+
+func (t *tokenJWT) assign(ctx context.Context, username string, revision uint64) (string, error) {
+	if t.verifyOnly {
+		return "", ErrVerifyOnly
+	}
+
+	// Future work: let a jwt token include permission information would be useful for
+	// permission checking in proxy side.
+	tk := jwt.NewWithClaims(t.signMethod,
+		jwt.MapClaims{
+			"username": username,
+			"revision": revision,
+			"exp":      time.Now().Add(t.ttl).Unix(),
+		})
+
+	token, err := tk.SignedString(t.key)
+	if err != nil {
+		if t.lg != nil {
+			t.lg.Warn(
+				"failed to sign a JWT token",
+				zap.String("user-name", username),
+				zap.Uint64("revision", revision),
+				zap.Error(err),
+			)
+		} else {
+			plog.Debugf("failed to sign jwt token: %s", err)
+		}
+		return "", err
+	}
+
+	if t.lg != nil {
+		t.lg.Info(
+			"created/assigned a new JWT token",
+			zap.String("user-name", username),
+			zap.Uint64("revision", revision),
+			zap.String("token", token),
+		)
+	} else {
+		plog.Debugf("jwt token: %s", token)
+	}
+	return token, err
+}
+
+func newTokenProviderJWT(lg *zap.Logger, optMap map[string]string) (*tokenJWT, error) {
+	var err error
+	var opts jwtOptions
+	err = opts.ParseWithDefaults(optMap)
+	if err != nil {
+		if lg != nil {
+			lg.Warn("problem loading JWT options", zap.Error(err))
+		} else {
+			plog.Errorf("problem loading JWT options: %s", err)
+		}
+		return nil, ErrInvalidAuthOpts
+	}
+
+	var keys = make([]string, 0, len(optMap))
+	for k := range optMap {
+		if !knownOptions[k] {
+			keys = append(keys, k)
+		}
+	}
+	if len(keys) > 0 {
+		if lg != nil {
+			lg.Warn("unknown JWT options", zap.Strings("keys", keys))
+		} else {
+			plog.Warningf("unknown JWT options: %v", keys)
+		}
+	}
+
+	key, err := opts.Key()
+	if err != nil {
+		return nil, err
+	}
+
+	t := &tokenJWT{
+		lg:         lg,
+		ttl:        opts.TTL,
+		signMethod: opts.SignMethod,
+		key:        key,
+	}
+
+	switch t.signMethod.(type) {
+	case *jwt.SigningMethodECDSA:
+		if _, ok := t.key.(*ecdsa.PublicKey); ok {
+			t.verifyOnly = true
+		}
+	case *jwt.SigningMethodRSA, *jwt.SigningMethodRSAPSS:
+		if _, ok := t.key.(*rsa.PublicKey); ok {
+			t.verifyOnly = true
+		}
+	}
+
+	return t, nil
+}
diff --git a/vendor/go.etcd.io/etcd/auth/nop.go b/vendor/go.etcd.io/etcd/auth/nop.go
new file mode 100644
index 0000000..d437874
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/auth/nop.go
@@ -0,0 +1,35 @@
+// Copyright 2018 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 auth
+
+import (
+	"context"
+)
+
+type tokenNop struct{}
+
+func (t *tokenNop) enable()                         {}
+func (t *tokenNop) disable()                        {}
+func (t *tokenNop) invalidateUser(string)           {}
+func (t *tokenNop) genTokenPrefix() (string, error) { return "", nil }
+func (t *tokenNop) info(ctx context.Context, token string, rev uint64) (*AuthInfo, bool) {
+	return nil, false
+}
+func (t *tokenNop) assign(ctx context.Context, username string, revision uint64) (string, error) {
+	return "", ErrAuthFailed
+}
+func newTokenProviderNop() (*tokenNop, error) {
+	return &tokenNop{}, nil
+}
diff --git a/vendor/go.etcd.io/etcd/auth/options.go b/vendor/go.etcd.io/etcd/auth/options.go
new file mode 100644
index 0000000..f40b92d
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/auth/options.go
@@ -0,0 +1,192 @@
+// Copyright 2018 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 auth
+
+import (
+	"crypto/ecdsa"
+	"crypto/rsa"
+	"fmt"
+	"io/ioutil"
+	"time"
+
+	jwt "github.com/dgrijalva/jwt-go"
+)
+
+const (
+	optSignMethod = "sign-method"
+	optPublicKey  = "pub-key"
+	optPrivateKey = "priv-key"
+	optTTL        = "ttl"
+)
+
+var knownOptions = map[string]bool{
+	optSignMethod: true,
+	optPublicKey:  true,
+	optPrivateKey: true,
+	optTTL:        true,
+}
+
+var (
+	// DefaultTTL will be used when a 'ttl' is not specified
+	DefaultTTL = 5 * time.Minute
+)
+
+type jwtOptions struct {
+	SignMethod jwt.SigningMethod
+	PublicKey  []byte
+	PrivateKey []byte
+	TTL        time.Duration
+}
+
+// ParseWithDefaults will load options from the specified map or set defaults where appropriate
+func (opts *jwtOptions) ParseWithDefaults(optMap map[string]string) error {
+	if opts.TTL == 0 && optMap[optTTL] == "" {
+		opts.TTL = DefaultTTL
+	}
+
+	return opts.Parse(optMap)
+}
+
+// Parse will load options from the specified map
+func (opts *jwtOptions) Parse(optMap map[string]string) error {
+	var err error
+	if ttl := optMap[optTTL]; ttl != "" {
+		opts.TTL, err = time.ParseDuration(ttl)
+		if err != nil {
+			return err
+		}
+	}
+
+	if file := optMap[optPublicKey]; file != "" {
+		opts.PublicKey, err = ioutil.ReadFile(file)
+		if err != nil {
+			return err
+		}
+	}
+
+	if file := optMap[optPrivateKey]; file != "" {
+		opts.PrivateKey, err = ioutil.ReadFile(file)
+		if err != nil {
+			return err
+		}
+	}
+
+	// signing method is a required field
+	method := optMap[optSignMethod]
+	opts.SignMethod = jwt.GetSigningMethod(method)
+	if opts.SignMethod == nil {
+		return ErrInvalidAuthMethod
+	}
+
+	return nil
+}
+
+// Key will parse and return the appropriately typed key for the selected signature method
+func (opts *jwtOptions) Key() (interface{}, error) {
+	switch opts.SignMethod.(type) {
+	case *jwt.SigningMethodRSA, *jwt.SigningMethodRSAPSS:
+		return opts.rsaKey()
+	case *jwt.SigningMethodECDSA:
+		return opts.ecKey()
+	case *jwt.SigningMethodHMAC:
+		return opts.hmacKey()
+	default:
+		return nil, fmt.Errorf("unsupported signing method: %T", opts.SignMethod)
+	}
+}
+
+func (opts *jwtOptions) hmacKey() (interface{}, error) {
+	if len(opts.PrivateKey) == 0 {
+		return nil, ErrMissingKey
+	}
+	return opts.PrivateKey, nil
+}
+
+func (opts *jwtOptions) rsaKey() (interface{}, error) {
+	var (
+		priv *rsa.PrivateKey
+		pub  *rsa.PublicKey
+		err  error
+	)
+
+	if len(opts.PrivateKey) > 0 {
+		priv, err = jwt.ParseRSAPrivateKeyFromPEM(opts.PrivateKey)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	if len(opts.PublicKey) > 0 {
+		pub, err = jwt.ParseRSAPublicKeyFromPEM(opts.PublicKey)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	if priv == nil {
+		if pub == nil {
+			// Neither key given
+			return nil, ErrMissingKey
+		}
+		// Public key only, can verify tokens
+		return pub, nil
+	}
+
+	// both keys provided, make sure they match
+	if pub != nil && pub.E != priv.E && pub.N.Cmp(priv.N) != 0 {
+		return nil, ErrKeyMismatch
+	}
+
+	return priv, nil
+}
+
+func (opts *jwtOptions) ecKey() (interface{}, error) {
+	var (
+		priv *ecdsa.PrivateKey
+		pub  *ecdsa.PublicKey
+		err  error
+	)
+
+	if len(opts.PrivateKey) > 0 {
+		priv, err = jwt.ParseECPrivateKeyFromPEM(opts.PrivateKey)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	if len(opts.PublicKey) > 0 {
+		pub, err = jwt.ParseECPublicKeyFromPEM(opts.PublicKey)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	if priv == nil {
+		if pub == nil {
+			// Neither key given
+			return nil, ErrMissingKey
+		}
+		// Public key only, can verify tokens
+		return pub, nil
+	}
+
+	// both keys provided, make sure they match
+	if pub != nil && pub.Curve != priv.Curve &&
+		pub.X.Cmp(priv.X) != 0 && pub.Y.Cmp(priv.Y) != 0 {
+		return nil, ErrKeyMismatch
+	}
+
+	return priv, nil
+}
diff --git a/vendor/go.etcd.io/etcd/auth/range_perm_cache.go b/vendor/go.etcd.io/etcd/auth/range_perm_cache.go
new file mode 100644
index 0000000..7b6c182
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/auth/range_perm_cache.go
@@ -0,0 +1,153 @@
+// Copyright 2016 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 auth
+
+import (
+	"go.etcd.io/etcd/auth/authpb"
+	"go.etcd.io/etcd/mvcc/backend"
+	"go.etcd.io/etcd/pkg/adt"
+
+	"go.uber.org/zap"
+)
+
+func getMergedPerms(lg *zap.Logger, tx backend.BatchTx, userName string) *unifiedRangePermissions {
+	user := getUser(lg, tx, userName)
+	if user == nil {
+		return nil
+	}
+
+	readPerms := adt.NewIntervalTree()
+	writePerms := adt.NewIntervalTree()
+
+	for _, roleName := range user.Roles {
+		role := getRole(tx, roleName)
+		if role == nil {
+			continue
+		}
+
+		for _, perm := range role.KeyPermission {
+			var ivl adt.Interval
+			var rangeEnd []byte
+
+			if len(perm.RangeEnd) != 1 || perm.RangeEnd[0] != 0 {
+				rangeEnd = perm.RangeEnd
+			}
+
+			if len(perm.RangeEnd) != 0 {
+				ivl = adt.NewBytesAffineInterval(perm.Key, rangeEnd)
+			} else {
+				ivl = adt.NewBytesAffinePoint(perm.Key)
+			}
+
+			switch perm.PermType {
+			case authpb.READWRITE:
+				readPerms.Insert(ivl, struct{}{})
+				writePerms.Insert(ivl, struct{}{})
+
+			case authpb.READ:
+				readPerms.Insert(ivl, struct{}{})
+
+			case authpb.WRITE:
+				writePerms.Insert(ivl, struct{}{})
+			}
+		}
+	}
+
+	return &unifiedRangePermissions{
+		readPerms:  readPerms,
+		writePerms: writePerms,
+	}
+}
+
+func checkKeyInterval(
+	lg *zap.Logger,
+	cachedPerms *unifiedRangePermissions,
+	key, rangeEnd []byte,
+	permtyp authpb.Permission_Type) bool {
+	if len(rangeEnd) == 1 && rangeEnd[0] == 0 {
+		rangeEnd = nil
+	}
+
+	ivl := adt.NewBytesAffineInterval(key, rangeEnd)
+	switch permtyp {
+	case authpb.READ:
+		return cachedPerms.readPerms.Contains(ivl)
+	case authpb.WRITE:
+		return cachedPerms.writePerms.Contains(ivl)
+	default:
+		if lg != nil {
+			lg.Panic("unknown auth type", zap.String("auth-type", permtyp.String()))
+		} else {
+			plog.Panicf("unknown auth type: %v", permtyp)
+		}
+	}
+	return false
+}
+
+func checkKeyPoint(lg *zap.Logger, cachedPerms *unifiedRangePermissions, key []byte, permtyp authpb.Permission_Type) bool {
+	pt := adt.NewBytesAffinePoint(key)
+	switch permtyp {
+	case authpb.READ:
+		return cachedPerms.readPerms.Intersects(pt)
+	case authpb.WRITE:
+		return cachedPerms.writePerms.Intersects(pt)
+	default:
+		if lg != nil {
+			lg.Panic("unknown auth type", zap.String("auth-type", permtyp.String()))
+		} else {
+			plog.Panicf("unknown auth type: %v", permtyp)
+		}
+	}
+	return false
+}
+
+func (as *authStore) isRangeOpPermitted(tx backend.BatchTx, userName string, key, rangeEnd []byte, permtyp authpb.Permission_Type) bool {
+	// assumption: tx is Lock()ed
+	_, ok := as.rangePermCache[userName]
+	if !ok {
+		perms := getMergedPerms(as.lg, tx, userName)
+		if perms == nil {
+			if as.lg != nil {
+				as.lg.Warn(
+					"failed to create a merged permission",
+					zap.String("user-name", userName),
+				)
+			} else {
+				plog.Errorf("failed to create a unified permission of user %s", userName)
+			}
+			return false
+		}
+		as.rangePermCache[userName] = perms
+	}
+
+	if len(rangeEnd) == 0 {
+		return checkKeyPoint(as.lg, as.rangePermCache[userName], key, permtyp)
+	}
+
+	return checkKeyInterval(as.lg, as.rangePermCache[userName], key, rangeEnd, permtyp)
+}
+
+func (as *authStore) clearCachedPerm() {
+	as.rangePermCache = make(map[string]*unifiedRangePermissions)
+}
+
+func (as *authStore) invalidateCachedPerm(userName string) {
+	delete(as.rangePermCache, userName)
+}
+
+type unifiedRangePermissions struct {
+	readPerms  adt.IntervalTree
+	writePerms adt.IntervalTree
+}
diff --git a/vendor/go.etcd.io/etcd/auth/simple_token.go b/vendor/go.etcd.io/etcd/auth/simple_token.go
new file mode 100644
index 0000000..934978c
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/auth/simple_token.go
@@ -0,0 +1,243 @@
+// Copyright 2016 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 auth
+
+// CAUTION: This random number based token mechanism is only for testing purpose.
+// JWT based mechanism will be added in the near future.
+
+import (
+	"context"
+	"crypto/rand"
+	"fmt"
+	"math/big"
+	"strconv"
+	"strings"
+	"sync"
+	"time"
+
+	"go.uber.org/zap"
+)
+
+const (
+	letters                  = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ"
+	defaultSimpleTokenLength = 16
+)
+
+// var for testing purposes
+var (
+	simpleTokenTTL           = 5 * time.Minute
+	simpleTokenTTLResolution = 1 * time.Second
+)
+
+type simpleTokenTTLKeeper struct {
+	tokens          map[string]time.Time
+	donec           chan struct{}
+	stopc           chan struct{}
+	deleteTokenFunc func(string)
+	mu              *sync.Mutex
+}
+
+func (tm *simpleTokenTTLKeeper) stop() {
+	select {
+	case tm.stopc <- struct{}{}:
+	case <-tm.donec:
+	}
+	<-tm.donec
+}
+
+func (tm *simpleTokenTTLKeeper) addSimpleToken(token string) {
+	tm.tokens[token] = time.Now().Add(simpleTokenTTL)
+}
+
+func (tm *simpleTokenTTLKeeper) resetSimpleToken(token string) {
+	if _, ok := tm.tokens[token]; ok {
+		tm.tokens[token] = time.Now().Add(simpleTokenTTL)
+	}
+}
+
+func (tm *simpleTokenTTLKeeper) deleteSimpleToken(token string) {
+	delete(tm.tokens, token)
+}
+
+func (tm *simpleTokenTTLKeeper) run() {
+	tokenTicker := time.NewTicker(simpleTokenTTLResolution)
+	defer func() {
+		tokenTicker.Stop()
+		close(tm.donec)
+	}()
+	for {
+		select {
+		case <-tokenTicker.C:
+			nowtime := time.Now()
+			tm.mu.Lock()
+			for t, tokenendtime := range tm.tokens {
+				if nowtime.After(tokenendtime) {
+					tm.deleteTokenFunc(t)
+					delete(tm.tokens, t)
+				}
+			}
+			tm.mu.Unlock()
+		case <-tm.stopc:
+			return
+		}
+	}
+}
+
+type tokenSimple struct {
+	lg                *zap.Logger
+	indexWaiter       func(uint64) <-chan struct{}
+	simpleTokenKeeper *simpleTokenTTLKeeper
+	simpleTokensMu    sync.Mutex
+	simpleTokens      map[string]string // token -> username
+}
+
+func (t *tokenSimple) genTokenPrefix() (string, error) {
+	ret := make([]byte, defaultSimpleTokenLength)
+
+	for i := 0; i < defaultSimpleTokenLength; i++ {
+		bInt, err := rand.Int(rand.Reader, big.NewInt(int64(len(letters))))
+		if err != nil {
+			return "", err
+		}
+
+		ret[i] = letters[bInt.Int64()]
+	}
+
+	return string(ret), nil
+}
+
+func (t *tokenSimple) assignSimpleTokenToUser(username, token string) {
+	t.simpleTokensMu.Lock()
+	defer t.simpleTokensMu.Unlock()
+	if t.simpleTokenKeeper == nil {
+		return
+	}
+
+	_, ok := t.simpleTokens[token]
+	if ok {
+		if t.lg != nil {
+			t.lg.Panic(
+				"failed to assign already-used simple token to a user",
+				zap.String("user-name", username),
+				zap.String("token", token),
+			)
+		} else {
+			plog.Panicf("token %s is already used", token)
+		}
+	}
+
+	t.simpleTokens[token] = username
+	t.simpleTokenKeeper.addSimpleToken(token)
+}
+
+func (t *tokenSimple) invalidateUser(username string) {
+	if t.simpleTokenKeeper == nil {
+		return
+	}
+	t.simpleTokensMu.Lock()
+	for token, name := range t.simpleTokens {
+		if name == username {
+			delete(t.simpleTokens, token)
+			t.simpleTokenKeeper.deleteSimpleToken(token)
+		}
+	}
+	t.simpleTokensMu.Unlock()
+}
+
+func (t *tokenSimple) enable() {
+	delf := func(tk string) {
+		if username, ok := t.simpleTokens[tk]; ok {
+			if t.lg != nil {
+				t.lg.Info(
+					"deleted a simple token",
+					zap.String("user-name", username),
+					zap.String("token", tk),
+				)
+			} else {
+				plog.Infof("deleting token %s for user %s", tk, username)
+			}
+			delete(t.simpleTokens, tk)
+		}
+	}
+	t.simpleTokenKeeper = &simpleTokenTTLKeeper{
+		tokens:          make(map[string]time.Time),
+		donec:           make(chan struct{}),
+		stopc:           make(chan struct{}),
+		deleteTokenFunc: delf,
+		mu:              &t.simpleTokensMu,
+	}
+	go t.simpleTokenKeeper.run()
+}
+
+func (t *tokenSimple) disable() {
+	t.simpleTokensMu.Lock()
+	tk := t.simpleTokenKeeper
+	t.simpleTokenKeeper = nil
+	t.simpleTokens = make(map[string]string) // invalidate all tokens
+	t.simpleTokensMu.Unlock()
+	if tk != nil {
+		tk.stop()
+	}
+}
+
+func (t *tokenSimple) info(ctx context.Context, token string, revision uint64) (*AuthInfo, bool) {
+	if !t.isValidSimpleToken(ctx, token) {
+		return nil, false
+	}
+	t.simpleTokensMu.Lock()
+	username, ok := t.simpleTokens[token]
+	if ok && t.simpleTokenKeeper != nil {
+		t.simpleTokenKeeper.resetSimpleToken(token)
+	}
+	t.simpleTokensMu.Unlock()
+	return &AuthInfo{Username: username, Revision: revision}, ok
+}
+
+func (t *tokenSimple) assign(ctx context.Context, username string, rev uint64) (string, error) {
+	// rev isn't used in simple token, it is only used in JWT
+	index := ctx.Value(AuthenticateParamIndex{}).(uint64)
+	simpleTokenPrefix := ctx.Value(AuthenticateParamSimpleTokenPrefix{}).(string)
+	token := fmt.Sprintf("%s.%d", simpleTokenPrefix, index)
+	t.assignSimpleTokenToUser(username, token)
+
+	return token, nil
+}
+
+func (t *tokenSimple) isValidSimpleToken(ctx context.Context, token string) bool {
+	splitted := strings.Split(token, ".")
+	if len(splitted) != 2 {
+		return false
+	}
+	index, err := strconv.Atoi(splitted[1])
+	if err != nil {
+		return false
+	}
+
+	select {
+	case <-t.indexWaiter(uint64(index)):
+		return true
+	case <-ctx.Done():
+	}
+
+	return false
+}
+
+func newTokenProviderSimple(lg *zap.Logger, indexWaiter func(uint64) <-chan struct{}) *tokenSimple {
+	return &tokenSimple{
+		lg:           lg,
+		simpleTokens: make(map[string]string),
+		indexWaiter:  indexWaiter,
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/auth/store.go b/vendor/go.etcd.io/etcd/auth/store.go
new file mode 100644
index 0000000..5212255
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/auth/store.go
@@ -0,0 +1,1420 @@
+// Copyright 2016 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 auth
+
+import (
+	"bytes"
+	"context"
+	"encoding/binary"
+	"errors"
+	"sort"
+	"strings"
+	"sync"
+	"sync/atomic"
+
+	"go.etcd.io/etcd/auth/authpb"
+	"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/mvcc/backend"
+
+	"github.com/coreos/pkg/capnslog"
+	"go.uber.org/zap"
+	"golang.org/x/crypto/bcrypt"
+	"google.golang.org/grpc/credentials"
+	"google.golang.org/grpc/metadata"
+	"google.golang.org/grpc/peer"
+)
+
+var (
+	enableFlagKey = []byte("authEnabled")
+	authEnabled   = []byte{1}
+	authDisabled  = []byte{0}
+
+	revisionKey = []byte("authRevision")
+
+	authBucketName      = []byte("auth")
+	authUsersBucketName = []byte("authUsers")
+	authRolesBucketName = []byte("authRoles")
+
+	plog = capnslog.NewPackageLogger("go.etcd.io/etcd", "auth")
+
+	ErrRootUserNotExist     = errors.New("auth: root user does not exist")
+	ErrRootRoleNotExist     = errors.New("auth: root user does not have root role")
+	ErrUserAlreadyExist     = errors.New("auth: user already exists")
+	ErrUserEmpty            = errors.New("auth: user name is empty")
+	ErrUserNotFound         = errors.New("auth: user not found")
+	ErrRoleAlreadyExist     = errors.New("auth: role already exists")
+	ErrRoleNotFound         = errors.New("auth: role not found")
+	ErrRoleEmpty            = errors.New("auth: role name is empty")
+	ErrAuthFailed           = errors.New("auth: authentication failed, invalid user ID or password")
+	ErrPermissionDenied     = errors.New("auth: permission denied")
+	ErrRoleNotGranted       = errors.New("auth: role is not granted to the user")
+	ErrPermissionNotGranted = errors.New("auth: permission is not granted to the role")
+	ErrAuthNotEnabled       = errors.New("auth: authentication is not enabled")
+	ErrAuthOldRevision      = errors.New("auth: revision in header is old")
+	ErrInvalidAuthToken     = errors.New("auth: invalid auth token")
+	ErrInvalidAuthOpts      = errors.New("auth: invalid auth options")
+	ErrInvalidAuthMgmt      = errors.New("auth: invalid auth management")
+	ErrInvalidAuthMethod    = errors.New("auth: invalid auth signature method")
+	ErrMissingKey           = errors.New("auth: missing key data")
+	ErrKeyMismatch          = errors.New("auth: public and private keys don't match")
+	ErrVerifyOnly           = errors.New("auth: token signing attempted with verify-only key")
+)
+
+const (
+	rootUser = "root"
+	rootRole = "root"
+
+	tokenTypeSimple = "simple"
+	tokenTypeJWT    = "jwt"
+
+	revBytesLen = 8
+)
+
+type AuthInfo struct {
+	Username string
+	Revision uint64
+}
+
+// AuthenticateParamIndex is used for a key of context in the parameters of Authenticate()
+type AuthenticateParamIndex struct{}
+
+// AuthenticateParamSimpleTokenPrefix is used for a key of context in the parameters of Authenticate()
+type AuthenticateParamSimpleTokenPrefix struct{}
+
+// AuthStore defines auth storage interface.
+type AuthStore interface {
+	// AuthEnable turns on the authentication feature
+	AuthEnable() error
+
+	// AuthDisable turns off the authentication feature
+	AuthDisable()
+
+	// IsAuthEnabled returns true if the authentication feature is enabled.
+	IsAuthEnabled() bool
+
+	// Authenticate does authentication based on given user name and password
+	Authenticate(ctx context.Context, username, password string) (*pb.AuthenticateResponse, error)
+
+	// Recover recovers the state of auth store from the given backend
+	Recover(b backend.Backend)
+
+	// UserAdd adds a new user
+	UserAdd(r *pb.AuthUserAddRequest) (*pb.AuthUserAddResponse, error)
+
+	// UserDelete deletes a user
+	UserDelete(r *pb.AuthUserDeleteRequest) (*pb.AuthUserDeleteResponse, error)
+
+	// UserChangePassword changes a password of a user
+	UserChangePassword(r *pb.AuthUserChangePasswordRequest) (*pb.AuthUserChangePasswordResponse, error)
+
+	// UserGrantRole grants a role to the user
+	UserGrantRole(r *pb.AuthUserGrantRoleRequest) (*pb.AuthUserGrantRoleResponse, error)
+
+	// UserGet gets the detailed information of a users
+	UserGet(r *pb.AuthUserGetRequest) (*pb.AuthUserGetResponse, error)
+
+	// UserRevokeRole revokes a role of a user
+	UserRevokeRole(r *pb.AuthUserRevokeRoleRequest) (*pb.AuthUserRevokeRoleResponse, error)
+
+	// RoleAdd adds a new role
+	RoleAdd(r *pb.AuthRoleAddRequest) (*pb.AuthRoleAddResponse, error)
+
+	// RoleGrantPermission grants a permission to a role
+	RoleGrantPermission(r *pb.AuthRoleGrantPermissionRequest) (*pb.AuthRoleGrantPermissionResponse, error)
+
+	// RoleGet gets the detailed information of a role
+	RoleGet(r *pb.AuthRoleGetRequest) (*pb.AuthRoleGetResponse, error)
+
+	// RoleRevokePermission gets the detailed information of a role
+	RoleRevokePermission(r *pb.AuthRoleRevokePermissionRequest) (*pb.AuthRoleRevokePermissionResponse, error)
+
+	// RoleDelete gets the detailed information of a role
+	RoleDelete(r *pb.AuthRoleDeleteRequest) (*pb.AuthRoleDeleteResponse, error)
+
+	// UserList gets a list of all users
+	UserList(r *pb.AuthUserListRequest) (*pb.AuthUserListResponse, error)
+
+	// RoleList gets a list of all roles
+	RoleList(r *pb.AuthRoleListRequest) (*pb.AuthRoleListResponse, error)
+
+	// IsPutPermitted checks put permission of the user
+	IsPutPermitted(authInfo *AuthInfo, key []byte) error
+
+	// IsRangePermitted checks range permission of the user
+	IsRangePermitted(authInfo *AuthInfo, key, rangeEnd []byte) error
+
+	// IsDeleteRangePermitted checks delete-range permission of the user
+	IsDeleteRangePermitted(authInfo *AuthInfo, key, rangeEnd []byte) error
+
+	// IsAdminPermitted checks admin permission of the user
+	IsAdminPermitted(authInfo *AuthInfo) error
+
+	// GenTokenPrefix produces a random string in a case of simple token
+	// in a case of JWT, it produces an empty string
+	GenTokenPrefix() (string, error)
+
+	// Revision gets current revision of authStore
+	Revision() uint64
+
+	// CheckPassword checks a given pair of username and password is correct
+	CheckPassword(username, password string) (uint64, error)
+
+	// Close does cleanup of AuthStore
+	Close() error
+
+	// AuthInfoFromCtx gets AuthInfo from gRPC's context
+	AuthInfoFromCtx(ctx context.Context) (*AuthInfo, error)
+
+	// AuthInfoFromTLS gets AuthInfo from TLS info of gRPC's context
+	AuthInfoFromTLS(ctx context.Context) *AuthInfo
+
+	// WithRoot generates and installs a token that can be used as a root credential
+	WithRoot(ctx context.Context) context.Context
+
+	// HasRole checks that user has role
+	HasRole(user, role string) bool
+}
+
+type TokenProvider interface {
+	info(ctx context.Context, token string, revision uint64) (*AuthInfo, bool)
+	assign(ctx context.Context, username string, revision uint64) (string, error)
+	enable()
+	disable()
+
+	invalidateUser(string)
+	genTokenPrefix() (string, error)
+}
+
+type authStore struct {
+	// atomic operations; need 64-bit align, or 32-bit tests will crash
+	revision uint64
+
+	lg        *zap.Logger
+	be        backend.Backend
+	enabled   bool
+	enabledMu sync.RWMutex
+
+	rangePermCache map[string]*unifiedRangePermissions // username -> unifiedRangePermissions
+
+	tokenProvider TokenProvider
+	bcryptCost    int // the algorithm cost / strength for hashing auth passwords
+}
+
+func (as *authStore) AuthEnable() error {
+	as.enabledMu.Lock()
+	defer as.enabledMu.Unlock()
+	if as.enabled {
+		if as.lg != nil {
+			as.lg.Info("authentication is already enabled; ignored auth enable request")
+		} else {
+			plog.Noticef("Authentication already enabled")
+		}
+		return nil
+	}
+	b := as.be
+	tx := b.BatchTx()
+	tx.Lock()
+	defer func() {
+		tx.Unlock()
+		b.ForceCommit()
+	}()
+
+	u := getUser(as.lg, tx, rootUser)
+	if u == nil {
+		return ErrRootUserNotExist
+	}
+
+	if !hasRootRole(u) {
+		return ErrRootRoleNotExist
+	}
+
+	tx.UnsafePut(authBucketName, enableFlagKey, authEnabled)
+
+	as.enabled = true
+	as.tokenProvider.enable()
+
+	as.rangePermCache = make(map[string]*unifiedRangePermissions)
+
+	as.setRevision(getRevision(tx))
+
+	if as.lg != nil {
+		as.lg.Info("enabled authentication")
+	} else {
+		plog.Noticef("Authentication enabled")
+	}
+	return nil
+}
+
+func (as *authStore) AuthDisable() {
+	as.enabledMu.Lock()
+	defer as.enabledMu.Unlock()
+	if !as.enabled {
+		return
+	}
+	b := as.be
+	tx := b.BatchTx()
+	tx.Lock()
+	tx.UnsafePut(authBucketName, enableFlagKey, authDisabled)
+	as.commitRevision(tx)
+	tx.Unlock()
+	b.ForceCommit()
+
+	as.enabled = false
+	as.tokenProvider.disable()
+
+	if as.lg != nil {
+		as.lg.Info("disabled authentication")
+	} else {
+		plog.Noticef("Authentication disabled")
+	}
+}
+
+func (as *authStore) Close() error {
+	as.enabledMu.Lock()
+	defer as.enabledMu.Unlock()
+	if !as.enabled {
+		return nil
+	}
+	as.tokenProvider.disable()
+	return nil
+}
+
+func (as *authStore) Authenticate(ctx context.Context, username, password string) (*pb.AuthenticateResponse, error) {
+	if !as.IsAuthEnabled() {
+		return nil, ErrAuthNotEnabled
+	}
+
+	tx := as.be.BatchTx()
+	tx.Lock()
+	defer tx.Unlock()
+
+	user := getUser(as.lg, tx, username)
+	if user == nil {
+		return nil, ErrAuthFailed
+	}
+
+	if user.Options.NoPassword {
+		return nil, ErrAuthFailed
+	}
+
+	// Password checking is already performed in the API layer, so we don't need to check for now.
+	// Staleness of password can be detected with OCC in the API layer, too.
+
+	token, err := as.tokenProvider.assign(ctx, username, as.Revision())
+	if err != nil {
+		return nil, err
+	}
+
+	if as.lg != nil {
+		as.lg.Debug(
+			"authenticated a user",
+			zap.String("user-name", username),
+			zap.String("token", token),
+		)
+	} else {
+		plog.Debugf("authorized %s, token is %s", username, token)
+	}
+	return &pb.AuthenticateResponse{Token: token}, nil
+}
+
+func (as *authStore) CheckPassword(username, password string) (uint64, error) {
+	if !as.IsAuthEnabled() {
+		return 0, ErrAuthNotEnabled
+	}
+
+	tx := as.be.BatchTx()
+	tx.Lock()
+	defer tx.Unlock()
+
+	user := getUser(as.lg, tx, username)
+	if user == nil {
+		return 0, ErrAuthFailed
+	}
+
+	if user.Options.NoPassword {
+		return 0, ErrAuthFailed
+	}
+
+	if bcrypt.CompareHashAndPassword(user.Password, []byte(password)) != nil {
+		if as.lg != nil {
+			as.lg.Info("invalid password", zap.String("user-name", username))
+		} else {
+			plog.Noticef("authentication failed, invalid password for user %s", username)
+		}
+		return 0, ErrAuthFailed
+	}
+	return getRevision(tx), nil
+}
+
+func (as *authStore) Recover(be backend.Backend) {
+	enabled := false
+	as.be = be
+	tx := be.BatchTx()
+	tx.Lock()
+	_, vs := tx.UnsafeRange(authBucketName, enableFlagKey, nil, 0)
+	if len(vs) == 1 {
+		if bytes.Equal(vs[0], authEnabled) {
+			enabled = true
+		}
+	}
+
+	as.setRevision(getRevision(tx))
+
+	tx.Unlock()
+
+	as.enabledMu.Lock()
+	as.enabled = enabled
+	as.enabledMu.Unlock()
+}
+
+func (as *authStore) UserAdd(r *pb.AuthUserAddRequest) (*pb.AuthUserAddResponse, error) {
+	if len(r.Name) == 0 {
+		return nil, ErrUserEmpty
+	}
+
+	var hashed []byte
+	var err error
+
+	if r.Options != nil && !r.Options.NoPassword {
+		hashed, err = bcrypt.GenerateFromPassword([]byte(r.Password), as.bcryptCost)
+		if err != nil {
+			if as.lg != nil {
+				as.lg.Warn(
+					"failed to bcrypt hash password",
+					zap.String("user-name", r.Name),
+					zap.Error(err),
+				)
+			} else {
+				plog.Errorf("failed to hash password: %s", err)
+			}
+			return nil, err
+		}
+	}
+
+	tx := as.be.BatchTx()
+	tx.Lock()
+	defer tx.Unlock()
+
+	user := getUser(as.lg, tx, r.Name)
+	if user != nil {
+		return nil, ErrUserAlreadyExist
+	}
+
+	options := r.Options
+	if options == nil {
+		options = &authpb.UserAddOptions{
+			NoPassword: false,
+		}
+	}
+
+	newUser := &authpb.User{
+		Name:     []byte(r.Name),
+		Password: hashed,
+		Options:  options,
+	}
+
+	putUser(as.lg, tx, newUser)
+
+	as.commitRevision(tx)
+
+	if as.lg != nil {
+		as.lg.Info("added a user", zap.String("user-name", r.Name))
+	} else {
+		plog.Noticef("added a new user: %s", r.Name)
+	}
+	return &pb.AuthUserAddResponse{}, nil
+}
+
+func (as *authStore) UserDelete(r *pb.AuthUserDeleteRequest) (*pb.AuthUserDeleteResponse, error) {
+	if as.enabled && r.Name == rootUser {
+		if as.lg != nil {
+			as.lg.Warn("cannot delete 'root' user", zap.String("user-name", r.Name))
+		} else {
+			plog.Errorf("the user root must not be deleted")
+		}
+		return nil, ErrInvalidAuthMgmt
+	}
+
+	tx := as.be.BatchTx()
+	tx.Lock()
+	defer tx.Unlock()
+
+	user := getUser(as.lg, tx, r.Name)
+	if user == nil {
+		return nil, ErrUserNotFound
+	}
+
+	delUser(tx, r.Name)
+
+	as.commitRevision(tx)
+
+	as.invalidateCachedPerm(r.Name)
+	as.tokenProvider.invalidateUser(r.Name)
+
+	if as.lg != nil {
+		as.lg.Info(
+			"deleted a user",
+			zap.String("user-name", r.Name),
+			zap.Strings("user-roles", user.Roles),
+		)
+	} else {
+		plog.Noticef("deleted a user: %s", r.Name)
+	}
+	return &pb.AuthUserDeleteResponse{}, nil
+}
+
+func (as *authStore) UserChangePassword(r *pb.AuthUserChangePasswordRequest) (*pb.AuthUserChangePasswordResponse, error) {
+	// TODO(mitake): measure the cost of bcrypt.GenerateFromPassword()
+	// If the cost is too high, we should move the encryption to outside of the raft
+	hashed, err := bcrypt.GenerateFromPassword([]byte(r.Password), as.bcryptCost)
+	if err != nil {
+		if as.lg != nil {
+			as.lg.Warn(
+				"failed to bcrypt hash password",
+				zap.String("user-name", r.Name),
+				zap.Error(err),
+			)
+		} else {
+			plog.Errorf("failed to hash password: %s", err)
+		}
+		return nil, err
+	}
+
+	tx := as.be.BatchTx()
+	tx.Lock()
+	defer tx.Unlock()
+
+	user := getUser(as.lg, tx, r.Name)
+	if user == nil {
+		return nil, ErrUserNotFound
+	}
+
+	updatedUser := &authpb.User{
+		Name:     []byte(r.Name),
+		Roles:    user.Roles,
+		Password: hashed,
+		Options:  user.Options,
+	}
+
+	putUser(as.lg, tx, updatedUser)
+
+	as.commitRevision(tx)
+
+	as.invalidateCachedPerm(r.Name)
+	as.tokenProvider.invalidateUser(r.Name)
+
+	if as.lg != nil {
+		as.lg.Info(
+			"changed a password of a user",
+			zap.String("user-name", r.Name),
+			zap.Strings("user-roles", user.Roles),
+		)
+	} else {
+		plog.Noticef("changed a password of a user: %s", r.Name)
+	}
+	return &pb.AuthUserChangePasswordResponse{}, nil
+}
+
+func (as *authStore) UserGrantRole(r *pb.AuthUserGrantRoleRequest) (*pb.AuthUserGrantRoleResponse, error) {
+	tx := as.be.BatchTx()
+	tx.Lock()
+	defer tx.Unlock()
+
+	user := getUser(as.lg, tx, r.User)
+	if user == nil {
+		return nil, ErrUserNotFound
+	}
+
+	if r.Role != rootRole {
+		role := getRole(tx, r.Role)
+		if role == nil {
+			return nil, ErrRoleNotFound
+		}
+	}
+
+	idx := sort.SearchStrings(user.Roles, r.Role)
+	if idx < len(user.Roles) && user.Roles[idx] == r.Role {
+		if as.lg != nil {
+			as.lg.Warn(
+				"ignored grant role request to a user",
+				zap.String("user-name", r.User),
+				zap.Strings("user-roles", user.Roles),
+				zap.String("duplicate-role-name", r.Role),
+			)
+		} else {
+			plog.Warningf("user %s is already granted role %s", r.User, r.Role)
+		}
+		return &pb.AuthUserGrantRoleResponse{}, nil
+	}
+
+	user.Roles = append(user.Roles, r.Role)
+	sort.Strings(user.Roles)
+
+	putUser(as.lg, tx, user)
+
+	as.invalidateCachedPerm(r.User)
+
+	as.commitRevision(tx)
+
+	if as.lg != nil {
+		as.lg.Info(
+			"granted a role to a user",
+			zap.String("user-name", r.User),
+			zap.Strings("user-roles", user.Roles),
+			zap.String("added-role-name", r.Role),
+		)
+	} else {
+		plog.Noticef("granted role %s to user %s", r.Role, r.User)
+	}
+	return &pb.AuthUserGrantRoleResponse{}, nil
+}
+
+func (as *authStore) UserGet(r *pb.AuthUserGetRequest) (*pb.AuthUserGetResponse, error) {
+	tx := as.be.BatchTx()
+	tx.Lock()
+	user := getUser(as.lg, tx, r.Name)
+	tx.Unlock()
+
+	if user == nil {
+		return nil, ErrUserNotFound
+	}
+
+	var resp pb.AuthUserGetResponse
+	resp.Roles = append(resp.Roles, user.Roles...)
+	return &resp, nil
+}
+
+func (as *authStore) UserList(r *pb.AuthUserListRequest) (*pb.AuthUserListResponse, error) {
+	tx := as.be.BatchTx()
+	tx.Lock()
+	users := getAllUsers(as.lg, tx)
+	tx.Unlock()
+
+	resp := &pb.AuthUserListResponse{Users: make([]string, len(users))}
+	for i := range users {
+		resp.Users[i] = string(users[i].Name)
+	}
+	return resp, nil
+}
+
+func (as *authStore) UserRevokeRole(r *pb.AuthUserRevokeRoleRequest) (*pb.AuthUserRevokeRoleResponse, error) {
+	if as.enabled && r.Name == rootUser && r.Role == rootRole {
+		if as.lg != nil {
+			as.lg.Warn(
+				"'root' user cannot revoke 'root' role",
+				zap.String("user-name", r.Name),
+				zap.String("role-name", r.Role),
+			)
+		} else {
+			plog.Errorf("the role root must not be revoked from the user root")
+		}
+		return nil, ErrInvalidAuthMgmt
+	}
+
+	tx := as.be.BatchTx()
+	tx.Lock()
+	defer tx.Unlock()
+
+	user := getUser(as.lg, tx, r.Name)
+	if user == nil {
+		return nil, ErrUserNotFound
+	}
+
+	updatedUser := &authpb.User{
+		Name:     user.Name,
+		Password: user.Password,
+		Options:  user.Options,
+	}
+
+	for _, role := range user.Roles {
+		if role != r.Role {
+			updatedUser.Roles = append(updatedUser.Roles, role)
+		}
+	}
+
+	if len(updatedUser.Roles) == len(user.Roles) {
+		return nil, ErrRoleNotGranted
+	}
+
+	putUser(as.lg, tx, updatedUser)
+
+	as.invalidateCachedPerm(r.Name)
+
+	as.commitRevision(tx)
+
+	if as.lg != nil {
+		as.lg.Info(
+			"revoked a role from a user",
+			zap.String("user-name", r.Name),
+			zap.Strings("old-user-roles", user.Roles),
+			zap.Strings("new-user-roles", updatedUser.Roles),
+			zap.String("revoked-role-name", r.Role),
+		)
+	} else {
+		plog.Noticef("revoked role %s from user %s", r.Role, r.Name)
+	}
+	return &pb.AuthUserRevokeRoleResponse{}, nil
+}
+
+func (as *authStore) RoleGet(r *pb.AuthRoleGetRequest) (*pb.AuthRoleGetResponse, error) {
+	tx := as.be.BatchTx()
+	tx.Lock()
+	defer tx.Unlock()
+
+	var resp pb.AuthRoleGetResponse
+
+	role := getRole(tx, r.Role)
+	if role == nil {
+		return nil, ErrRoleNotFound
+	}
+	resp.Perm = append(resp.Perm, role.KeyPermission...)
+	return &resp, nil
+}
+
+func (as *authStore) RoleList(r *pb.AuthRoleListRequest) (*pb.AuthRoleListResponse, error) {
+	tx := as.be.BatchTx()
+	tx.Lock()
+	roles := getAllRoles(as.lg, tx)
+	tx.Unlock()
+
+	resp := &pb.AuthRoleListResponse{Roles: make([]string, len(roles))}
+	for i := range roles {
+		resp.Roles[i] = string(roles[i].Name)
+	}
+	return resp, nil
+}
+
+func (as *authStore) RoleRevokePermission(r *pb.AuthRoleRevokePermissionRequest) (*pb.AuthRoleRevokePermissionResponse, error) {
+	tx := as.be.BatchTx()
+	tx.Lock()
+	defer tx.Unlock()
+
+	role := getRole(tx, r.Role)
+	if role == nil {
+		return nil, ErrRoleNotFound
+	}
+
+	updatedRole := &authpb.Role{
+		Name: role.Name,
+	}
+
+	for _, perm := range role.KeyPermission {
+		if !bytes.Equal(perm.Key, r.Key) || !bytes.Equal(perm.RangeEnd, r.RangeEnd) {
+			updatedRole.KeyPermission = append(updatedRole.KeyPermission, perm)
+		}
+	}
+
+	if len(role.KeyPermission) == len(updatedRole.KeyPermission) {
+		return nil, ErrPermissionNotGranted
+	}
+
+	putRole(as.lg, tx, updatedRole)
+
+	// TODO(mitake): currently single role update invalidates every cache
+	// It should be optimized.
+	as.clearCachedPerm()
+
+	as.commitRevision(tx)
+
+	if as.lg != nil {
+		as.lg.Info(
+			"revoked a permission on range",
+			zap.String("role-name", r.Role),
+			zap.String("key", string(r.Key)),
+			zap.String("range-end", string(r.RangeEnd)),
+		)
+	} else {
+		plog.Noticef("revoked key %s from role %s", r.Key, r.Role)
+	}
+	return &pb.AuthRoleRevokePermissionResponse{}, nil
+}
+
+func (as *authStore) RoleDelete(r *pb.AuthRoleDeleteRequest) (*pb.AuthRoleDeleteResponse, error) {
+	if as.enabled && r.Role == rootRole {
+		if as.lg != nil {
+			as.lg.Warn("cannot delete 'root' role", zap.String("role-name", r.Role))
+		} else {
+			plog.Errorf("the role root must not be deleted")
+		}
+		return nil, ErrInvalidAuthMgmt
+	}
+
+	tx := as.be.BatchTx()
+	tx.Lock()
+	defer tx.Unlock()
+
+	role := getRole(tx, r.Role)
+	if role == nil {
+		return nil, ErrRoleNotFound
+	}
+
+	delRole(tx, r.Role)
+
+	users := getAllUsers(as.lg, tx)
+	for _, user := range users {
+		updatedUser := &authpb.User{
+			Name:     user.Name,
+			Password: user.Password,
+			Options:  user.Options,
+		}
+
+		for _, role := range user.Roles {
+			if role != r.Role {
+				updatedUser.Roles = append(updatedUser.Roles, role)
+			}
+		}
+
+		if len(updatedUser.Roles) == len(user.Roles) {
+			continue
+		}
+
+		putUser(as.lg, tx, updatedUser)
+
+		as.invalidateCachedPerm(string(user.Name))
+	}
+
+	as.commitRevision(tx)
+
+	if as.lg != nil {
+		as.lg.Info("deleted a role", zap.String("role-name", r.Role))
+	} else {
+		plog.Noticef("deleted role %s", r.Role)
+	}
+	return &pb.AuthRoleDeleteResponse{}, nil
+}
+
+func (as *authStore) RoleAdd(r *pb.AuthRoleAddRequest) (*pb.AuthRoleAddResponse, error) {
+	if len(r.Name) == 0 {
+		return nil, ErrRoleEmpty
+	}
+
+	tx := as.be.BatchTx()
+	tx.Lock()
+	defer tx.Unlock()
+
+	role := getRole(tx, r.Name)
+	if role != nil {
+		return nil, ErrRoleAlreadyExist
+	}
+
+	newRole := &authpb.Role{
+		Name: []byte(r.Name),
+	}
+
+	putRole(as.lg, tx, newRole)
+
+	as.commitRevision(tx)
+
+	if as.lg != nil {
+		as.lg.Info("created a role", zap.String("role-name", r.Name))
+	} else {
+		plog.Noticef("Role %s is created", r.Name)
+	}
+	return &pb.AuthRoleAddResponse{}, nil
+}
+
+func (as *authStore) authInfoFromToken(ctx context.Context, token string) (*AuthInfo, bool) {
+	return as.tokenProvider.info(ctx, token, as.Revision())
+}
+
+type permSlice []*authpb.Permission
+
+func (perms permSlice) Len() int {
+	return len(perms)
+}
+
+func (perms permSlice) Less(i, j int) bool {
+	return bytes.Compare(perms[i].Key, perms[j].Key) < 0
+}
+
+func (perms permSlice) Swap(i, j int) {
+	perms[i], perms[j] = perms[j], perms[i]
+}
+
+func (as *authStore) RoleGrantPermission(r *pb.AuthRoleGrantPermissionRequest) (*pb.AuthRoleGrantPermissionResponse, error) {
+	tx := as.be.BatchTx()
+	tx.Lock()
+	defer tx.Unlock()
+
+	role := getRole(tx, r.Name)
+	if role == nil {
+		return nil, ErrRoleNotFound
+	}
+
+	idx := sort.Search(len(role.KeyPermission), func(i int) bool {
+		return bytes.Compare(role.KeyPermission[i].Key, r.Perm.Key) >= 0
+	})
+
+	if idx < len(role.KeyPermission) && bytes.Equal(role.KeyPermission[idx].Key, r.Perm.Key) && bytes.Equal(role.KeyPermission[idx].RangeEnd, r.Perm.RangeEnd) {
+		// update existing permission
+		role.KeyPermission[idx].PermType = r.Perm.PermType
+	} else {
+		// append new permission to the role
+		newPerm := &authpb.Permission{
+			Key:      r.Perm.Key,
+			RangeEnd: r.Perm.RangeEnd,
+			PermType: r.Perm.PermType,
+		}
+
+		role.KeyPermission = append(role.KeyPermission, newPerm)
+		sort.Sort(permSlice(role.KeyPermission))
+	}
+
+	putRole(as.lg, tx, role)
+
+	// TODO(mitake): currently single role update invalidates every cache
+	// It should be optimized.
+	as.clearCachedPerm()
+
+	as.commitRevision(tx)
+
+	if as.lg != nil {
+		as.lg.Info(
+			"granted/updated a permission to a user",
+			zap.String("user-name", r.Name),
+			zap.String("permission-name", authpb.Permission_Type_name[int32(r.Perm.PermType)]),
+		)
+	} else {
+		plog.Noticef("role %s's permission of key %s is updated as %s", r.Name, r.Perm.Key, authpb.Permission_Type_name[int32(r.Perm.PermType)])
+	}
+	return &pb.AuthRoleGrantPermissionResponse{}, nil
+}
+
+func (as *authStore) isOpPermitted(userName string, revision uint64, key, rangeEnd []byte, permTyp authpb.Permission_Type) error {
+	// TODO(mitake): this function would be costly so we need a caching mechanism
+	if !as.IsAuthEnabled() {
+		return nil
+	}
+
+	// only gets rev == 0 when passed AuthInfo{}; no user given
+	if revision == 0 {
+		return ErrUserEmpty
+	}
+
+	if revision < as.Revision() {
+		return ErrAuthOldRevision
+	}
+
+	tx := as.be.BatchTx()
+	tx.Lock()
+	defer tx.Unlock()
+
+	user := getUser(as.lg, tx, userName)
+	if user == nil {
+		if as.lg != nil {
+			as.lg.Warn("cannot find a user for permission check", zap.String("user-name", userName))
+		} else {
+			plog.Errorf("invalid user name %s for permission checking", userName)
+		}
+		return ErrPermissionDenied
+	}
+
+	// root role should have permission on all ranges
+	if hasRootRole(user) {
+		return nil
+	}
+
+	if as.isRangeOpPermitted(tx, userName, key, rangeEnd, permTyp) {
+		return nil
+	}
+
+	return ErrPermissionDenied
+}
+
+func (as *authStore) IsPutPermitted(authInfo *AuthInfo, key []byte) error {
+	return as.isOpPermitted(authInfo.Username, authInfo.Revision, key, nil, authpb.WRITE)
+}
+
+func (as *authStore) IsRangePermitted(authInfo *AuthInfo, key, rangeEnd []byte) error {
+	return as.isOpPermitted(authInfo.Username, authInfo.Revision, key, rangeEnd, authpb.READ)
+}
+
+func (as *authStore) IsDeleteRangePermitted(authInfo *AuthInfo, key, rangeEnd []byte) error {
+	return as.isOpPermitted(authInfo.Username, authInfo.Revision, key, rangeEnd, authpb.WRITE)
+}
+
+func (as *authStore) IsAdminPermitted(authInfo *AuthInfo) error {
+	if !as.IsAuthEnabled() {
+		return nil
+	}
+	if authInfo == nil {
+		return ErrUserEmpty
+	}
+
+	tx := as.be.BatchTx()
+	tx.Lock()
+	u := getUser(as.lg, tx, authInfo.Username)
+	tx.Unlock()
+
+	if u == nil {
+		return ErrUserNotFound
+	}
+
+	if !hasRootRole(u) {
+		return ErrPermissionDenied
+	}
+
+	return nil
+}
+
+func getUser(lg *zap.Logger, tx backend.BatchTx, username string) *authpb.User {
+	_, vs := tx.UnsafeRange(authUsersBucketName, []byte(username), nil, 0)
+	if len(vs) == 0 {
+		return nil
+	}
+
+	user := &authpb.User{}
+	err := user.Unmarshal(vs[0])
+	if err != nil {
+		if lg != nil {
+			lg.Panic(
+				"failed to unmarshal 'authpb.User'",
+				zap.String("user-name", username),
+				zap.Error(err),
+			)
+		} else {
+			plog.Panicf("failed to unmarshal user struct (name: %s): %s", username, err)
+		}
+	}
+	return user
+}
+
+func getAllUsers(lg *zap.Logger, tx backend.BatchTx) []*authpb.User {
+	_, vs := tx.UnsafeRange(authUsersBucketName, []byte{0}, []byte{0xff}, -1)
+	if len(vs) == 0 {
+		return nil
+	}
+
+	users := make([]*authpb.User, len(vs))
+	for i := range vs {
+		user := &authpb.User{}
+		err := user.Unmarshal(vs[i])
+		if err != nil {
+			if lg != nil {
+				lg.Panic("failed to unmarshal 'authpb.User'", zap.Error(err))
+			} else {
+				plog.Panicf("failed to unmarshal user struct: %s", err)
+			}
+		}
+		users[i] = user
+	}
+	return users
+}
+
+func putUser(lg *zap.Logger, tx backend.BatchTx, user *authpb.User) {
+	b, err := user.Marshal()
+	if err != nil {
+		if lg != nil {
+			lg.Panic("failed to unmarshal 'authpb.User'", zap.Error(err))
+		} else {
+			plog.Panicf("failed to marshal user struct (name: %s): %s", user.Name, err)
+		}
+	}
+	tx.UnsafePut(authUsersBucketName, user.Name, b)
+}
+
+func delUser(tx backend.BatchTx, username string) {
+	tx.UnsafeDelete(authUsersBucketName, []byte(username))
+}
+
+func getRole(tx backend.BatchTx, rolename string) *authpb.Role {
+	_, vs := tx.UnsafeRange(authRolesBucketName, []byte(rolename), nil, 0)
+	if len(vs) == 0 {
+		return nil
+	}
+
+	role := &authpb.Role{}
+	err := role.Unmarshal(vs[0])
+	if err != nil {
+		plog.Panicf("failed to unmarshal role struct (name: %s): %s", rolename, err)
+	}
+	return role
+}
+
+func getAllRoles(lg *zap.Logger, tx backend.BatchTx) []*authpb.Role {
+	_, vs := tx.UnsafeRange(authRolesBucketName, []byte{0}, []byte{0xff}, -1)
+	if len(vs) == 0 {
+		return nil
+	}
+
+	roles := make([]*authpb.Role, len(vs))
+	for i := range vs {
+		role := &authpb.Role{}
+		err := role.Unmarshal(vs[i])
+		if err != nil {
+			if lg != nil {
+				lg.Panic("failed to unmarshal 'authpb.Role'", zap.Error(err))
+			} else {
+				plog.Panicf("failed to unmarshal role struct: %s", err)
+			}
+		}
+		roles[i] = role
+	}
+	return roles
+}
+
+func putRole(lg *zap.Logger, tx backend.BatchTx, role *authpb.Role) {
+	b, err := role.Marshal()
+	if err != nil {
+		if lg != nil {
+			lg.Panic(
+				"failed to marshal 'authpb.Role'",
+				zap.String("role-name", string(role.Name)),
+				zap.Error(err),
+			)
+		} else {
+			plog.Panicf("failed to marshal role struct (name: %s): %s", role.Name, err)
+		}
+	}
+
+	tx.UnsafePut(authRolesBucketName, role.Name, b)
+}
+
+func delRole(tx backend.BatchTx, rolename string) {
+	tx.UnsafeDelete(authRolesBucketName, []byte(rolename))
+}
+
+func (as *authStore) IsAuthEnabled() bool {
+	as.enabledMu.RLock()
+	defer as.enabledMu.RUnlock()
+	return as.enabled
+}
+
+// NewAuthStore creates a new AuthStore.
+func NewAuthStore(lg *zap.Logger, be backend.Backend, tp TokenProvider, bcryptCost int) *authStore {
+	if bcryptCost < bcrypt.MinCost || bcryptCost > bcrypt.MaxCost {
+		if lg != nil {
+			lg.Warn(
+				"use default bcrypt cost instead of the invalid given cost",
+				zap.Int("min-cost", bcrypt.MinCost),
+				zap.Int("max-cost", bcrypt.MaxCost),
+				zap.Int("default-cost", bcrypt.DefaultCost),
+				zap.Int("given-cost", bcryptCost))
+		} else {
+			plog.Warningf("Use default bcrypt-cost %d instead of the invalid value %d",
+				bcrypt.DefaultCost, bcryptCost)
+		}
+
+		bcryptCost = bcrypt.DefaultCost
+	}
+
+	tx := be.BatchTx()
+	tx.Lock()
+
+	tx.UnsafeCreateBucket(authBucketName)
+	tx.UnsafeCreateBucket(authUsersBucketName)
+	tx.UnsafeCreateBucket(authRolesBucketName)
+
+	enabled := false
+	_, vs := tx.UnsafeRange(authBucketName, enableFlagKey, nil, 0)
+	if len(vs) == 1 {
+		if bytes.Equal(vs[0], authEnabled) {
+			enabled = true
+		}
+	}
+
+	as := &authStore{
+		revision:       getRevision(tx),
+		lg:             lg,
+		be:             be,
+		enabled:        enabled,
+		rangePermCache: make(map[string]*unifiedRangePermissions),
+		tokenProvider:  tp,
+		bcryptCost:     bcryptCost,
+	}
+
+	if enabled {
+		as.tokenProvider.enable()
+	}
+
+	if as.Revision() == 0 {
+		as.commitRevision(tx)
+	}
+
+	tx.Unlock()
+	be.ForceCommit()
+
+	return as
+}
+
+func hasRootRole(u *authpb.User) bool {
+	// u.Roles is sorted in UserGrantRole(), so we can use binary search.
+	idx := sort.SearchStrings(u.Roles, rootRole)
+	return idx != len(u.Roles) && u.Roles[idx] == rootRole
+}
+
+func (as *authStore) commitRevision(tx backend.BatchTx) {
+	atomic.AddUint64(&as.revision, 1)
+	revBytes := make([]byte, revBytesLen)
+	binary.BigEndian.PutUint64(revBytes, as.Revision())
+	tx.UnsafePut(authBucketName, revisionKey, revBytes)
+}
+
+func getRevision(tx backend.BatchTx) uint64 {
+	_, vs := tx.UnsafeRange(authBucketName, revisionKey, nil, 0)
+	if len(vs) != 1 {
+		// this can happen in the initialization phase
+		return 0
+	}
+	return binary.BigEndian.Uint64(vs[0])
+}
+
+func (as *authStore) setRevision(rev uint64) {
+	atomic.StoreUint64(&as.revision, rev)
+}
+
+func (as *authStore) Revision() uint64 {
+	return atomic.LoadUint64(&as.revision)
+}
+
+func (as *authStore) AuthInfoFromTLS(ctx context.Context) (ai *AuthInfo) {
+	peer, ok := peer.FromContext(ctx)
+	if !ok || peer == nil || peer.AuthInfo == nil {
+		return nil
+	}
+
+	tlsInfo := peer.AuthInfo.(credentials.TLSInfo)
+	for _, chains := range tlsInfo.State.VerifiedChains {
+		if len(chains) < 1 {
+			continue
+		}
+		ai = &AuthInfo{
+			Username: chains[0].Subject.CommonName,
+			Revision: as.Revision(),
+		}
+		md, ok := metadata.FromIncomingContext(ctx)
+		if !ok {
+			return nil
+		}
+
+		// gRPC-gateway proxy request to etcd server includes Grpcgateway-Accept
+		// header. The proxy uses etcd client server certificate. If the certificate
+		// has a CommonName we should never use this for authentication.
+		if gw := md["grpcgateway-accept"]; len(gw) > 0 {
+			if as.lg != nil {
+				as.lg.Warn(
+					"ignoring common name in gRPC-gateway proxy request",
+					zap.String("common-name", ai.Username),
+					zap.String("user-name", ai.Username),
+					zap.Uint64("revision", ai.Revision),
+				)
+			} else {
+				plog.Warningf("ignoring common name in gRPC-gateway proxy request %s", ai.Username)
+			}
+			return nil
+		}
+		if as.lg != nil {
+			as.lg.Debug(
+				"found command name",
+				zap.String("common-name", ai.Username),
+				zap.String("user-name", ai.Username),
+				zap.Uint64("revision", ai.Revision),
+			)
+		} else {
+			plog.Debugf("found common name %s", ai.Username)
+		}
+		break
+	}
+	return ai
+}
+
+func (as *authStore) AuthInfoFromCtx(ctx context.Context) (*AuthInfo, error) {
+	md, ok := metadata.FromIncomingContext(ctx)
+	if !ok {
+		return nil, nil
+	}
+
+	//TODO(mitake|hexfusion) review unifying key names
+	ts, ok := md[rpctypes.TokenFieldNameGRPC]
+	if !ok {
+		ts, ok = md[rpctypes.TokenFieldNameSwagger]
+	}
+	if !ok {
+		return nil, nil
+	}
+
+	token := ts[0]
+	authInfo, uok := as.authInfoFromToken(ctx, token)
+	if !uok {
+		if as.lg != nil {
+			as.lg.Warn("invalid auth token", zap.String("token", token))
+		} else {
+			plog.Warningf("invalid auth token: %s", token)
+		}
+		return nil, ErrInvalidAuthToken
+	}
+
+	return authInfo, nil
+}
+
+func (as *authStore) GenTokenPrefix() (string, error) {
+	return as.tokenProvider.genTokenPrefix()
+}
+
+func decomposeOpts(lg *zap.Logger, optstr string) (string, map[string]string, error) {
+	opts := strings.Split(optstr, ",")
+	tokenType := opts[0]
+
+	typeSpecificOpts := make(map[string]string)
+	for i := 1; i < len(opts); i++ {
+		pair := strings.Split(opts[i], "=")
+
+		if len(pair) != 2 {
+			if lg != nil {
+				lg.Warn("invalid token option", zap.String("option", optstr))
+			} else {
+				plog.Errorf("invalid token specific option: %s", optstr)
+			}
+			return "", nil, ErrInvalidAuthOpts
+		}
+
+		if _, ok := typeSpecificOpts[pair[0]]; ok {
+			if lg != nil {
+				lg.Warn(
+					"invalid token option",
+					zap.String("option", optstr),
+					zap.String("duplicate-parameter", pair[0]),
+				)
+			} else {
+				plog.Errorf("invalid token specific option, duplicated parameters (%s): %s", pair[0], optstr)
+			}
+			return "", nil, ErrInvalidAuthOpts
+		}
+
+		typeSpecificOpts[pair[0]] = pair[1]
+	}
+
+	return tokenType, typeSpecificOpts, nil
+
+}
+
+// NewTokenProvider creates a new token provider.
+func NewTokenProvider(
+	lg *zap.Logger,
+	tokenOpts string,
+	indexWaiter func(uint64) <-chan struct{}) (TokenProvider, error) {
+	tokenType, typeSpecificOpts, err := decomposeOpts(lg, tokenOpts)
+	if err != nil {
+		return nil, ErrInvalidAuthOpts
+	}
+
+	switch tokenType {
+	case tokenTypeSimple:
+		if lg != nil {
+			lg.Warn("simple token is not cryptographically signed")
+		} else {
+			plog.Warningf("simple token is not cryptographically signed")
+		}
+		return newTokenProviderSimple(lg, indexWaiter), nil
+
+	case tokenTypeJWT:
+		return newTokenProviderJWT(lg, typeSpecificOpts)
+
+	case "":
+		return newTokenProviderNop()
+
+	default:
+		if lg != nil {
+			lg.Warn(
+				"unknown token type",
+				zap.String("type", tokenType),
+				zap.Error(ErrInvalidAuthOpts),
+			)
+		} else {
+			plog.Errorf("unknown token type: %s", tokenType)
+		}
+		return nil, ErrInvalidAuthOpts
+	}
+}
+
+func (as *authStore) WithRoot(ctx context.Context) context.Context {
+	if !as.IsAuthEnabled() {
+		return ctx
+	}
+
+	var ctxForAssign context.Context
+	if ts, ok := as.tokenProvider.(*tokenSimple); ok && ts != nil {
+		ctx1 := context.WithValue(ctx, AuthenticateParamIndex{}, uint64(0))
+		prefix, err := ts.genTokenPrefix()
+		if err != nil {
+			if as.lg != nil {
+				as.lg.Warn(
+					"failed to generate prefix of internally used token",
+					zap.Error(err),
+				)
+			} else {
+				plog.Errorf("failed to generate prefix of internally used token")
+			}
+			return ctx
+		}
+		ctxForAssign = context.WithValue(ctx1, AuthenticateParamSimpleTokenPrefix{}, prefix)
+	} else {
+		ctxForAssign = ctx
+	}
+
+	token, err := as.tokenProvider.assign(ctxForAssign, "root", as.Revision())
+	if err != nil {
+		// this must not happen
+		if as.lg != nil {
+			as.lg.Warn(
+				"failed to assign token for lease revoking",
+				zap.Error(err),
+			)
+		} else {
+			plog.Errorf("failed to assign token for lease revoking: %s", err)
+		}
+		return ctx
+	}
+
+	mdMap := map[string]string{
+		rpctypes.TokenFieldNameGRPC: token,
+	}
+	tokenMD := metadata.New(mdMap)
+
+	// use "mdIncomingKey{}" since it's called from local etcdserver
+	return metadata.NewIncomingContext(ctx, tokenMD)
+}
+
+func (as *authStore) HasRole(user, role string) bool {
+	tx := as.be.BatchTx()
+	tx.Lock()
+	u := getUser(as.lg, tx, user)
+	tx.Unlock()
+
+	if u == nil {
+		if as.lg != nil {
+			as.lg.Warn(
+				"'has-role' requested for non-existing user",
+				zap.String("user-name", user),
+				zap.String("role-name", role),
+			)
+		} else {
+			plog.Warningf("tried to check user %s has role %s, but user %s doesn't exist", user, role, user)
+		}
+		return false
+	}
+
+	for _, r := range u.Roles {
+		if role == r {
+			return true
+		}
+	}
+	return false
+}
+
+func (as *authStore) BcryptCost() int {
+	return as.bcryptCost
+}
diff --git a/vendor/go.etcd.io/etcd/client/README.md b/vendor/go.etcd.io/etcd/client/README.md
new file mode 100644
index 0000000..521d6c0
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/client/README.md
@@ -0,0 +1,112 @@
+# etcd/client
+
+etcd/client is the Go client library for etcd.
+
+[![GoDoc](https://godoc.org/go.etcd.io/etcd/client?status.png)](https://godoc.org/go.etcd.io/etcd/client)
+
+For full compatibility, it is recommended to vendor builds using etcd's vendored packages, using tools like `golang/dep`, as in [vendor directories](https://golang.org/cmd/go/#hdr-Vendor_Directories).
+
+## Install
+
+```bash
+go get go.etcd.io/etcd/client
+```
+
+## Usage
+
+```go
+package main
+
+import (
+	"log"
+	"time"
+	"context"
+
+	"go.etcd.io/etcd/client"
+)
+
+func main() {
+	cfg := client.Config{
+		Endpoints:               []string{"http://127.0.0.1:2379"},
+		Transport:               client.DefaultTransport,
+		// set timeout per request to fail fast when the target endpoint is unavailable
+		HeaderTimeoutPerRequest: time.Second,
+	}
+	c, err := client.New(cfg)
+	if err != nil {
+		log.Fatal(err)
+	}
+	kapi := client.NewKeysAPI(c)
+	// set "/foo" key with "bar" value
+	log.Print("Setting '/foo' key with 'bar' value")
+	resp, err := kapi.Set(context.Background(), "/foo", "bar", nil)
+	if err != nil {
+		log.Fatal(err)
+	} else {
+		// print common key info
+		log.Printf("Set is done. Metadata is %q\n", resp)
+	}
+	// get "/foo" key's value
+	log.Print("Getting '/foo' key value")
+	resp, err = kapi.Get(context.Background(), "/foo", nil)
+	if err != nil {
+		log.Fatal(err)
+	} else {
+		// print common key info
+		log.Printf("Get is done. Metadata is %q\n", resp)
+		// print value
+		log.Printf("%q key has %q value\n", resp.Node.Key, resp.Node.Value)
+	}
+}
+```
+
+## Error Handling
+
+etcd client might return three types of errors.
+
+- context error
+
+Each API call has its first parameter as `context`. A context can be canceled or have an attached deadline. If the context is canceled or reaches its deadline, the responding context error will be returned no matter what internal errors the API call has already encountered.
+
+- cluster error
+
+Each API call tries to send request to the cluster endpoints one by one until it successfully gets a response. If a requests to an endpoint fails, due to exceeding per request timeout or connection issues, the error will be added into a list of errors. If all possible endpoints fail, a cluster error that includes all encountered errors will be returned.
+
+- response error
+
+If the response gets from the cluster is invalid, a plain string error will be returned. For example, it might be a invalid JSON error.
+
+Here is the example code to handle client errors:
+
+```go
+cfg := client.Config{Endpoints: []string{"http://etcd1:2379","http://etcd2:2379","http://etcd3:2379"}}
+c, err := client.New(cfg)
+if err != nil {
+	log.Fatal(err)
+}
+
+kapi := client.NewKeysAPI(c)
+resp, err := kapi.Set(ctx, "test", "bar", nil)
+if err != nil {
+	if err == context.Canceled {
+		// ctx is canceled by another routine
+	} else if err == context.DeadlineExceeded {
+		// ctx is attached with a deadline and it exceeded
+	} else if cerr, ok := err.(*client.ClusterError); ok {
+		// process (cerr.Errors)
+	} else {
+		// bad cluster endpoints, which are not etcd servers
+	}
+}
+```
+
+
+## Caveat
+
+1. etcd/client prefers to use the same endpoint as long as the endpoint continues to work well. This saves socket resources, and improves efficiency for both client and server side. This preference doesn't remove consistency from the data consumed by the client because data replicated to each etcd member has already passed through the consensus process.
+
+2. etcd/client does round-robin rotation on other available endpoints if the preferred endpoint isn't functioning properly. For example, if the member that etcd/client connects to is hard killed, etcd/client will fail on the first attempt with the killed member, and succeed on the second attempt with another member. If it fails to talk to all available endpoints, it will return all errors happened.
+
+3. Default etcd/client cannot handle the case that the remote server is SIGSTOPed now. TCP keepalive mechanism doesn't help in this scenario because operating system may still send TCP keep-alive packets. Over time we'd like to improve this functionality, but solving this issue isn't high priority because a real-life case in which a server is stopped, but the connection is kept alive, hasn't been brought to our attention.
+
+4. etcd/client cannot detect whether a member is healthy with watches and non-quorum read requests. If the member is isolated from the cluster, etcd/client may retrieve outdated data. Instead, users can either issue quorum read requests or monitor the /health endpoint for member health information.
diff --git a/vendor/go.etcd.io/etcd/client/auth_role.go b/vendor/go.etcd.io/etcd/client/auth_role.go
new file mode 100644
index 0000000..b6ba7e1
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/client/auth_role.go
@@ -0,0 +1,236 @@
+// Copyright 2015 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 client
+
+import (
+	"bytes"
+	"context"
+	"encoding/json"
+	"net/http"
+	"net/url"
+)
+
+type Role struct {
+	Role        string       `json:"role"`
+	Permissions Permissions  `json:"permissions"`
+	Grant       *Permissions `json:"grant,omitempty"`
+	Revoke      *Permissions `json:"revoke,omitempty"`
+}
+
+type Permissions struct {
+	KV rwPermission `json:"kv"`
+}
+
+type rwPermission struct {
+	Read  []string `json:"read"`
+	Write []string `json:"write"`
+}
+
+type PermissionType int
+
+const (
+	ReadPermission PermissionType = iota
+	WritePermission
+	ReadWritePermission
+)
+
+// NewAuthRoleAPI constructs a new AuthRoleAPI that uses HTTP to
+// interact with etcd's role creation and modification features.
+func NewAuthRoleAPI(c Client) AuthRoleAPI {
+	return &httpAuthRoleAPI{
+		client: c,
+	}
+}
+
+type AuthRoleAPI interface {
+	// AddRole adds a role.
+	AddRole(ctx context.Context, role string) error
+
+	// RemoveRole removes a role.
+	RemoveRole(ctx context.Context, role string) error
+
+	// GetRole retrieves role details.
+	GetRole(ctx context.Context, role string) (*Role, error)
+
+	// GrantRoleKV grants a role some permission prefixes for the KV store.
+	GrantRoleKV(ctx context.Context, role string, prefixes []string, permType PermissionType) (*Role, error)
+
+	// RevokeRoleKV revokes some permission prefixes for a role on the KV store.
+	RevokeRoleKV(ctx context.Context, role string, prefixes []string, permType PermissionType) (*Role, error)
+
+	// ListRoles lists roles.
+	ListRoles(ctx context.Context) ([]string, error)
+}
+
+type httpAuthRoleAPI struct {
+	client httpClient
+}
+
+type authRoleAPIAction struct {
+	verb string
+	name string
+	role *Role
+}
+
+type authRoleAPIList struct{}
+
+func (list *authRoleAPIList) HTTPRequest(ep url.URL) *http.Request {
+	u := v2AuthURL(ep, "roles", "")
+	req, _ := http.NewRequest("GET", u.String(), nil)
+	req.Header.Set("Content-Type", "application/json")
+	return req
+}
+
+func (l *authRoleAPIAction) HTTPRequest(ep url.URL) *http.Request {
+	u := v2AuthURL(ep, "roles", l.name)
+	if l.role == nil {
+		req, _ := http.NewRequest(l.verb, u.String(), nil)
+		return req
+	}
+	b, err := json.Marshal(l.role)
+	if err != nil {
+		panic(err)
+	}
+	body := bytes.NewReader(b)
+	req, _ := http.NewRequest(l.verb, u.String(), body)
+	req.Header.Set("Content-Type", "application/json")
+	return req
+}
+
+func (r *httpAuthRoleAPI) ListRoles(ctx context.Context) ([]string, error) {
+	resp, body, err := r.client.Do(ctx, &authRoleAPIList{})
+	if err != nil {
+		return nil, err
+	}
+	if err = assertStatusCode(resp.StatusCode, http.StatusOK); err != nil {
+		return nil, err
+	}
+	var roleList struct {
+		Roles []Role `json:"roles"`
+	}
+	if err = json.Unmarshal(body, &roleList); err != nil {
+		return nil, err
+	}
+	ret := make([]string, 0, len(roleList.Roles))
+	for _, r := range roleList.Roles {
+		ret = append(ret, r.Role)
+	}
+	return ret, nil
+}
+
+func (r *httpAuthRoleAPI) AddRole(ctx context.Context, rolename string) error {
+	role := &Role{
+		Role: rolename,
+	}
+	return r.addRemoveRole(ctx, &authRoleAPIAction{
+		verb: "PUT",
+		name: rolename,
+		role: role,
+	})
+}
+
+func (r *httpAuthRoleAPI) RemoveRole(ctx context.Context, rolename string) error {
+	return r.addRemoveRole(ctx, &authRoleAPIAction{
+		verb: "DELETE",
+		name: rolename,
+	})
+}
+
+func (r *httpAuthRoleAPI) addRemoveRole(ctx context.Context, req *authRoleAPIAction) error {
+	resp, body, err := r.client.Do(ctx, req)
+	if err != nil {
+		return err
+	}
+	if err := assertStatusCode(resp.StatusCode, http.StatusOK, http.StatusCreated); err != nil {
+		var sec authError
+		err := json.Unmarshal(body, &sec)
+		if err != nil {
+			return err
+		}
+		return sec
+	}
+	return nil
+}
+
+func (r *httpAuthRoleAPI) GetRole(ctx context.Context, rolename string) (*Role, error) {
+	return r.modRole(ctx, &authRoleAPIAction{
+		verb: "GET",
+		name: rolename,
+	})
+}
+
+func buildRWPermission(prefixes []string, permType PermissionType) rwPermission {
+	var out rwPermission
+	switch permType {
+	case ReadPermission:
+		out.Read = prefixes
+	case WritePermission:
+		out.Write = prefixes
+	case ReadWritePermission:
+		out.Read = prefixes
+		out.Write = prefixes
+	}
+	return out
+}
+
+func (r *httpAuthRoleAPI) GrantRoleKV(ctx context.Context, rolename string, prefixes []string, permType PermissionType) (*Role, error) {
+	rwp := buildRWPermission(prefixes, permType)
+	role := &Role{
+		Role: rolename,
+		Grant: &Permissions{
+			KV: rwp,
+		},
+	}
+	return r.modRole(ctx, &authRoleAPIAction{
+		verb: "PUT",
+		name: rolename,
+		role: role,
+	})
+}
+
+func (r *httpAuthRoleAPI) RevokeRoleKV(ctx context.Context, rolename string, prefixes []string, permType PermissionType) (*Role, error) {
+	rwp := buildRWPermission(prefixes, permType)
+	role := &Role{
+		Role: rolename,
+		Revoke: &Permissions{
+			KV: rwp,
+		},
+	}
+	return r.modRole(ctx, &authRoleAPIAction{
+		verb: "PUT",
+		name: rolename,
+		role: role,
+	})
+}
+
+func (r *httpAuthRoleAPI) modRole(ctx context.Context, req *authRoleAPIAction) (*Role, error) {
+	resp, body, err := r.client.Do(ctx, req)
+	if err != nil {
+		return nil, err
+	}
+	if err = assertStatusCode(resp.StatusCode, http.StatusOK); err != nil {
+		var sec authError
+		err = json.Unmarshal(body, &sec)
+		if err != nil {
+			return nil, err
+		}
+		return nil, sec
+	}
+	var role Role
+	if err = json.Unmarshal(body, &role); err != nil {
+		return nil, err
+	}
+	return &role, nil
+}
diff --git a/vendor/go.etcd.io/etcd/client/auth_user.go b/vendor/go.etcd.io/etcd/client/auth_user.go
new file mode 100644
index 0000000..8e7e2ef
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/client/auth_user.go
@@ -0,0 +1,319 @@
+// Copyright 2015 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 client
+
+import (
+	"bytes"
+	"context"
+	"encoding/json"
+	"net/http"
+	"net/url"
+	"path"
+)
+
+var (
+	defaultV2AuthPrefix = "/v2/auth"
+)
+
+type User struct {
+	User     string   `json:"user"`
+	Password string   `json:"password,omitempty"`
+	Roles    []string `json:"roles"`
+	Grant    []string `json:"grant,omitempty"`
+	Revoke   []string `json:"revoke,omitempty"`
+}
+
+// userListEntry is the user representation given by the server for ListUsers
+type userListEntry struct {
+	User  string `json:"user"`
+	Roles []Role `json:"roles"`
+}
+
+type UserRoles struct {
+	User  string `json:"user"`
+	Roles []Role `json:"roles"`
+}
+
+func v2AuthURL(ep url.URL, action string, name string) *url.URL {
+	if name != "" {
+		ep.Path = path.Join(ep.Path, defaultV2AuthPrefix, action, name)
+		return &ep
+	}
+	ep.Path = path.Join(ep.Path, defaultV2AuthPrefix, action)
+	return &ep
+}
+
+// NewAuthAPI constructs a new AuthAPI that uses HTTP to
+// interact with etcd's general auth features.
+func NewAuthAPI(c Client) AuthAPI {
+	return &httpAuthAPI{
+		client: c,
+	}
+}
+
+type AuthAPI interface {
+	// Enable auth.
+	Enable(ctx context.Context) error
+
+	// Disable auth.
+	Disable(ctx context.Context) error
+}
+
+type httpAuthAPI struct {
+	client httpClient
+}
+
+func (s *httpAuthAPI) Enable(ctx context.Context) error {
+	return s.enableDisable(ctx, &authAPIAction{"PUT"})
+}
+
+func (s *httpAuthAPI) Disable(ctx context.Context) error {
+	return s.enableDisable(ctx, &authAPIAction{"DELETE"})
+}
+
+func (s *httpAuthAPI) enableDisable(ctx context.Context, req httpAction) error {
+	resp, body, err := s.client.Do(ctx, req)
+	if err != nil {
+		return err
+	}
+	if err = assertStatusCode(resp.StatusCode, http.StatusOK, http.StatusCreated); err != nil {
+		var sec authError
+		err = json.Unmarshal(body, &sec)
+		if err != nil {
+			return err
+		}
+		return sec
+	}
+	return nil
+}
+
+type authAPIAction struct {
+	verb string
+}
+
+func (l *authAPIAction) HTTPRequest(ep url.URL) *http.Request {
+	u := v2AuthURL(ep, "enable", "")
+	req, _ := http.NewRequest(l.verb, u.String(), nil)
+	return req
+}
+
+type authError struct {
+	Message string `json:"message"`
+	Code    int    `json:"-"`
+}
+
+func (e authError) Error() string {
+	return e.Message
+}
+
+// NewAuthUserAPI constructs a new AuthUserAPI that uses HTTP to
+// interact with etcd's user creation and modification features.
+func NewAuthUserAPI(c Client) AuthUserAPI {
+	return &httpAuthUserAPI{
+		client: c,
+	}
+}
+
+type AuthUserAPI interface {
+	// AddUser adds a user.
+	AddUser(ctx context.Context, username string, password string) error
+
+	// RemoveUser removes a user.
+	RemoveUser(ctx context.Context, username string) error
+
+	// GetUser retrieves user details.
+	GetUser(ctx context.Context, username string) (*User, error)
+
+	// GrantUser grants a user some permission roles.
+	GrantUser(ctx context.Context, username string, roles []string) (*User, error)
+
+	// RevokeUser revokes some permission roles from a user.
+	RevokeUser(ctx context.Context, username string, roles []string) (*User, error)
+
+	// ChangePassword changes the user's password.
+	ChangePassword(ctx context.Context, username string, password string) (*User, error)
+
+	// ListUsers lists the users.
+	ListUsers(ctx context.Context) ([]string, error)
+}
+
+type httpAuthUserAPI struct {
+	client httpClient
+}
+
+type authUserAPIAction struct {
+	verb     string
+	username string
+	user     *User
+}
+
+type authUserAPIList struct{}
+
+func (list *authUserAPIList) HTTPRequest(ep url.URL) *http.Request {
+	u := v2AuthURL(ep, "users", "")
+	req, _ := http.NewRequest("GET", u.String(), nil)
+	req.Header.Set("Content-Type", "application/json")
+	return req
+}
+
+func (l *authUserAPIAction) HTTPRequest(ep url.URL) *http.Request {
+	u := v2AuthURL(ep, "users", l.username)
+	if l.user == nil {
+		req, _ := http.NewRequest(l.verb, u.String(), nil)
+		return req
+	}
+	b, err := json.Marshal(l.user)
+	if err != nil {
+		panic(err)
+	}
+	body := bytes.NewReader(b)
+	req, _ := http.NewRequest(l.verb, u.String(), body)
+	req.Header.Set("Content-Type", "application/json")
+	return req
+}
+
+func (u *httpAuthUserAPI) ListUsers(ctx context.Context) ([]string, error) {
+	resp, body, err := u.client.Do(ctx, &authUserAPIList{})
+	if err != nil {
+		return nil, err
+	}
+	if err = assertStatusCode(resp.StatusCode, http.StatusOK); err != nil {
+		var sec authError
+		err = json.Unmarshal(body, &sec)
+		if err != nil {
+			return nil, err
+		}
+		return nil, sec
+	}
+
+	var userList struct {
+		Users []userListEntry `json:"users"`
+	}
+
+	if err = json.Unmarshal(body, &userList); err != nil {
+		return nil, err
+	}
+
+	ret := make([]string, 0, len(userList.Users))
+	for _, u := range userList.Users {
+		ret = append(ret, u.User)
+	}
+	return ret, nil
+}
+
+func (u *httpAuthUserAPI) AddUser(ctx context.Context, username string, password string) error {
+	user := &User{
+		User:     username,
+		Password: password,
+	}
+	return u.addRemoveUser(ctx, &authUserAPIAction{
+		verb:     "PUT",
+		username: username,
+		user:     user,
+	})
+}
+
+func (u *httpAuthUserAPI) RemoveUser(ctx context.Context, username string) error {
+	return u.addRemoveUser(ctx, &authUserAPIAction{
+		verb:     "DELETE",
+		username: username,
+	})
+}
+
+func (u *httpAuthUserAPI) addRemoveUser(ctx context.Context, req *authUserAPIAction) error {
+	resp, body, err := u.client.Do(ctx, req)
+	if err != nil {
+		return err
+	}
+	if err = assertStatusCode(resp.StatusCode, http.StatusOK, http.StatusCreated); err != nil {
+		var sec authError
+		err = json.Unmarshal(body, &sec)
+		if err != nil {
+			return err
+		}
+		return sec
+	}
+	return nil
+}
+
+func (u *httpAuthUserAPI) GetUser(ctx context.Context, username string) (*User, error) {
+	return u.modUser(ctx, &authUserAPIAction{
+		verb:     "GET",
+		username: username,
+	})
+}
+
+func (u *httpAuthUserAPI) GrantUser(ctx context.Context, username string, roles []string) (*User, error) {
+	user := &User{
+		User:  username,
+		Grant: roles,
+	}
+	return u.modUser(ctx, &authUserAPIAction{
+		verb:     "PUT",
+		username: username,
+		user:     user,
+	})
+}
+
+func (u *httpAuthUserAPI) RevokeUser(ctx context.Context, username string, roles []string) (*User, error) {
+	user := &User{
+		User:   username,
+		Revoke: roles,
+	}
+	return u.modUser(ctx, &authUserAPIAction{
+		verb:     "PUT",
+		username: username,
+		user:     user,
+	})
+}
+
+func (u *httpAuthUserAPI) ChangePassword(ctx context.Context, username string, password string) (*User, error) {
+	user := &User{
+		User:     username,
+		Password: password,
+	}
+	return u.modUser(ctx, &authUserAPIAction{
+		verb:     "PUT",
+		username: username,
+		user:     user,
+	})
+}
+
+func (u *httpAuthUserAPI) modUser(ctx context.Context, req *authUserAPIAction) (*User, error) {
+	resp, body, err := u.client.Do(ctx, req)
+	if err != nil {
+		return nil, err
+	}
+	if err = assertStatusCode(resp.StatusCode, http.StatusOK); err != nil {
+		var sec authError
+		err = json.Unmarshal(body, &sec)
+		if err != nil {
+			return nil, err
+		}
+		return nil, sec
+	}
+	var user User
+	if err = json.Unmarshal(body, &user); err != nil {
+		var userR UserRoles
+		if urerr := json.Unmarshal(body, &userR); urerr != nil {
+			return nil, err
+		}
+		user.User = userR.User
+		for _, r := range userR.Roles {
+			user.Roles = append(user.Roles, r.Role)
+		}
+	}
+	return &user, nil
+}
diff --git a/vendor/go.etcd.io/etcd/client/cancelreq.go b/vendor/go.etcd.io/etcd/client/cancelreq.go
new file mode 100644
index 0000000..76d1f04
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/client/cancelreq.go
@@ -0,0 +1,18 @@
+// Copyright 2015 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// borrowed from golang/net/context/ctxhttp/cancelreq.go
+
+package client
+
+import "net/http"
+
+func requestCanceler(tr CancelableTransport, req *http.Request) func() {
+	ch := make(chan struct{})
+	req.Cancel = ch
+
+	return func() {
+		close(ch)
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/client/client.go b/vendor/go.etcd.io/etcd/client/client.go
new file mode 100644
index 0000000..de9ab79
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/client/client.go
@@ -0,0 +1,710 @@
+// Copyright 2015 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 client
+
+import (
+	"context"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"io/ioutil"
+	"math/rand"
+	"net"
+	"net/http"
+	"net/url"
+	"sort"
+	"strconv"
+	"sync"
+	"time"
+
+	"go.etcd.io/etcd/version"
+)
+
+var (
+	ErrNoEndpoints           = errors.New("client: no endpoints available")
+	ErrTooManyRedirects      = errors.New("client: too many redirects")
+	ErrClusterUnavailable    = errors.New("client: etcd cluster is unavailable or misconfigured")
+	ErrNoLeaderEndpoint      = errors.New("client: no leader endpoint available")
+	errTooManyRedirectChecks = errors.New("client: too many redirect checks")
+
+	// oneShotCtxValue is set on a context using WithValue(&oneShotValue) so
+	// that Do() will not retry a request
+	oneShotCtxValue interface{}
+)
+
+var DefaultRequestTimeout = 5 * time.Second
+
+var DefaultTransport CancelableTransport = &http.Transport{
+	Proxy: http.ProxyFromEnvironment,
+	Dial: (&net.Dialer{
+		Timeout:   30 * time.Second,
+		KeepAlive: 30 * time.Second,
+	}).Dial,
+	TLSHandshakeTimeout: 10 * time.Second,
+}
+
+type EndpointSelectionMode int
+
+const (
+	// EndpointSelectionRandom is the default value of the 'SelectionMode'.
+	// As the name implies, the client object will pick a node from the members
+	// of the cluster in a random fashion. If the cluster has three members, A, B,
+	// and C, the client picks any node from its three members as its request
+	// destination.
+	EndpointSelectionRandom EndpointSelectionMode = iota
+
+	// If 'SelectionMode' is set to 'EndpointSelectionPrioritizeLeader',
+	// requests are sent directly to the cluster leader. This reduces
+	// forwarding roundtrips compared to making requests to etcd followers
+	// who then forward them to the cluster leader. In the event of a leader
+	// failure, however, clients configured this way cannot prioritize among
+	// the remaining etcd followers. Therefore, when a client sets 'SelectionMode'
+	// to 'EndpointSelectionPrioritizeLeader', it must use 'client.AutoSync()' to
+	// maintain its knowledge of current cluster state.
+	//
+	// This mode should be used with Client.AutoSync().
+	EndpointSelectionPrioritizeLeader
+)
+
+type Config struct {
+	// Endpoints defines a set of URLs (schemes, hosts and ports only)
+	// that can be used to communicate with a logical etcd cluster. For
+	// example, a three-node cluster could be provided like so:
+	//
+	// 	Endpoints: []string{
+	//		"http://node1.example.com:2379",
+	//		"http://node2.example.com:2379",
+	//		"http://node3.example.com:2379",
+	//	}
+	//
+	// If multiple endpoints are provided, the Client will attempt to
+	// use them all in the event that one or more of them are unusable.
+	//
+	// If Client.Sync is ever called, the Client may cache an alternate
+	// set of endpoints to continue operation.
+	Endpoints []string
+
+	// Transport is used by the Client to drive HTTP requests. If not
+	// provided, DefaultTransport will be used.
+	Transport CancelableTransport
+
+	// CheckRedirect specifies the policy for handling HTTP redirects.
+	// If CheckRedirect is not nil, the Client calls it before
+	// following an HTTP redirect. The sole argument is the number of
+	// requests that have already been made. If CheckRedirect returns
+	// an error, Client.Do will not make any further requests and return
+	// the error back it to the caller.
+	//
+	// If CheckRedirect is nil, the Client uses its default policy,
+	// which is to stop after 10 consecutive requests.
+	CheckRedirect CheckRedirectFunc
+
+	// Username specifies the user credential to add as an authorization header
+	Username string
+
+	// Password is the password for the specified user to add as an authorization header
+	// to the request.
+	Password string
+
+	// HeaderTimeoutPerRequest specifies the time limit to wait for response
+	// header in a single request made by the Client. The timeout includes
+	// connection time, any redirects, and header wait time.
+	//
+	// For non-watch GET request, server returns the response body immediately.
+	// For PUT/POST/DELETE request, server will attempt to commit request
+	// before responding, which is expected to take `100ms + 2 * RTT`.
+	// For watch request, server returns the header immediately to notify Client
+	// watch start. But if server is behind some kind of proxy, the response
+	// header may be cached at proxy, and Client cannot rely on this behavior.
+	//
+	// Especially, wait request will ignore this timeout.
+	//
+	// One API call may send multiple requests to different etcd servers until it
+	// succeeds. Use context of the API to specify the overall timeout.
+	//
+	// A HeaderTimeoutPerRequest of zero means no timeout.
+	HeaderTimeoutPerRequest time.Duration
+
+	// SelectionMode is an EndpointSelectionMode enum that specifies the
+	// policy for choosing the etcd cluster node to which requests are sent.
+	SelectionMode EndpointSelectionMode
+}
+
+func (cfg *Config) transport() CancelableTransport {
+	if cfg.Transport == nil {
+		return DefaultTransport
+	}
+	return cfg.Transport
+}
+
+func (cfg *Config) checkRedirect() CheckRedirectFunc {
+	if cfg.CheckRedirect == nil {
+		return DefaultCheckRedirect
+	}
+	return cfg.CheckRedirect
+}
+
+// CancelableTransport mimics net/http.Transport, but requires that
+// the object also support request cancellation.
+type CancelableTransport interface {
+	http.RoundTripper
+	CancelRequest(req *http.Request)
+}
+
+type CheckRedirectFunc func(via int) error
+
+// DefaultCheckRedirect follows up to 10 redirects, but no more.
+var DefaultCheckRedirect CheckRedirectFunc = func(via int) error {
+	if via > 10 {
+		return ErrTooManyRedirects
+	}
+	return nil
+}
+
+type Client interface {
+	// Sync updates the internal cache of the etcd cluster's membership.
+	Sync(context.Context) error
+
+	// AutoSync periodically calls Sync() every given interval.
+	// The recommended sync interval is 10 seconds to 1 minute, which does
+	// not bring too much overhead to server and makes client catch up the
+	// cluster change in time.
+	//
+	// The example to use it:
+	//
+	//  for {
+	//      err := client.AutoSync(ctx, 10*time.Second)
+	//      if err == context.DeadlineExceeded || err == context.Canceled {
+	//          break
+	//      }
+	//      log.Print(err)
+	//  }
+	AutoSync(context.Context, time.Duration) error
+
+	// Endpoints returns a copy of the current set of API endpoints used
+	// by Client to resolve HTTP requests. If Sync has ever been called,
+	// this may differ from the initial Endpoints provided in the Config.
+	Endpoints() []string
+
+	// SetEndpoints sets the set of API endpoints used by Client to resolve
+	// HTTP requests. If the given endpoints are not valid, an error will be
+	// returned
+	SetEndpoints(eps []string) error
+
+	// GetVersion retrieves the current etcd server and cluster version
+	GetVersion(ctx context.Context) (*version.Versions, error)
+
+	httpClient
+}
+
+func New(cfg Config) (Client, error) {
+	c := &httpClusterClient{
+		clientFactory: newHTTPClientFactory(cfg.transport(), cfg.checkRedirect(), cfg.HeaderTimeoutPerRequest),
+		rand:          rand.New(rand.NewSource(int64(time.Now().Nanosecond()))),
+		selectionMode: cfg.SelectionMode,
+	}
+	if cfg.Username != "" {
+		c.credentials = &credentials{
+			username: cfg.Username,
+			password: cfg.Password,
+		}
+	}
+	if err := c.SetEndpoints(cfg.Endpoints); err != nil {
+		return nil, err
+	}
+	return c, nil
+}
+
+type httpClient interface {
+	Do(context.Context, httpAction) (*http.Response, []byte, error)
+}
+
+func newHTTPClientFactory(tr CancelableTransport, cr CheckRedirectFunc, headerTimeout time.Duration) httpClientFactory {
+	return func(ep url.URL) httpClient {
+		return &redirectFollowingHTTPClient{
+			checkRedirect: cr,
+			client: &simpleHTTPClient{
+				transport:     tr,
+				endpoint:      ep,
+				headerTimeout: headerTimeout,
+			},
+		}
+	}
+}
+
+type credentials struct {
+	username string
+	password string
+}
+
+type httpClientFactory func(url.URL) httpClient
+
+type httpAction interface {
+	HTTPRequest(url.URL) *http.Request
+}
+
+type httpClusterClient struct {
+	clientFactory httpClientFactory
+	endpoints     []url.URL
+	pinned        int
+	credentials   *credentials
+	sync.RWMutex
+	rand          *rand.Rand
+	selectionMode EndpointSelectionMode
+}
+
+func (c *httpClusterClient) getLeaderEndpoint(ctx context.Context, eps []url.URL) (string, error) {
+	ceps := make([]url.URL, len(eps))
+	copy(ceps, eps)
+
+	// To perform a lookup on the new endpoint list without using the current
+	// client, we'll copy it
+	clientCopy := &httpClusterClient{
+		clientFactory: c.clientFactory,
+		credentials:   c.credentials,
+		rand:          c.rand,
+
+		pinned:    0,
+		endpoints: ceps,
+	}
+
+	mAPI := NewMembersAPI(clientCopy)
+	leader, err := mAPI.Leader(ctx)
+	if err != nil {
+		return "", err
+	}
+	if len(leader.ClientURLs) == 0 {
+		return "", ErrNoLeaderEndpoint
+	}
+
+	return leader.ClientURLs[0], nil // TODO: how to handle multiple client URLs?
+}
+
+func (c *httpClusterClient) parseEndpoints(eps []string) ([]url.URL, error) {
+	if len(eps) == 0 {
+		return []url.URL{}, ErrNoEndpoints
+	}
+
+	neps := make([]url.URL, len(eps))
+	for i, ep := range eps {
+		u, err := url.Parse(ep)
+		if err != nil {
+			return []url.URL{}, err
+		}
+		neps[i] = *u
+	}
+	return neps, nil
+}
+
+func (c *httpClusterClient) SetEndpoints(eps []string) error {
+	neps, err := c.parseEndpoints(eps)
+	if err != nil {
+		return err
+	}
+
+	c.Lock()
+	defer c.Unlock()
+
+	c.endpoints = shuffleEndpoints(c.rand, neps)
+	// We're not doing anything for PrioritizeLeader here. This is
+	// due to not having a context meaning we can't call getLeaderEndpoint
+	// However, if you're using PrioritizeLeader, you've already been told
+	// to regularly call sync, where we do have a ctx, and can figure the
+	// leader. PrioritizeLeader is also quite a loose guarantee, so deal
+	// with it
+	c.pinned = 0
+
+	return nil
+}
+
+func (c *httpClusterClient) Do(ctx context.Context, act httpAction) (*http.Response, []byte, error) {
+	action := act
+	c.RLock()
+	leps := len(c.endpoints)
+	eps := make([]url.URL, leps)
+	n := copy(eps, c.endpoints)
+	pinned := c.pinned
+
+	if c.credentials != nil {
+		action = &authedAction{
+			act:         act,
+			credentials: *c.credentials,
+		}
+	}
+	c.RUnlock()
+
+	if leps == 0 {
+		return nil, nil, ErrNoEndpoints
+	}
+
+	if leps != n {
+		return nil, nil, errors.New("unable to pick endpoint: copy failed")
+	}
+
+	var resp *http.Response
+	var body []byte
+	var err error
+	cerr := &ClusterError{}
+	isOneShot := ctx.Value(&oneShotCtxValue) != nil
+
+	for i := pinned; i < leps+pinned; i++ {
+		k := i % leps
+		hc := c.clientFactory(eps[k])
+		resp, body, err = hc.Do(ctx, action)
+		if err != nil {
+			cerr.Errors = append(cerr.Errors, err)
+			if err == ctx.Err() {
+				return nil, nil, ctx.Err()
+			}
+			if err == context.Canceled || err == context.DeadlineExceeded {
+				return nil, nil, err
+			}
+		} else if resp.StatusCode/100 == 5 {
+			switch resp.StatusCode {
+			case http.StatusInternalServerError, http.StatusServiceUnavailable:
+				// TODO: make sure this is a no leader response
+				cerr.Errors = append(cerr.Errors, fmt.Errorf("client: etcd member %s has no leader", eps[k].String()))
+			default:
+				cerr.Errors = append(cerr.Errors, fmt.Errorf("client: etcd member %s returns server error [%s]", eps[k].String(), http.StatusText(resp.StatusCode)))
+			}
+			err = cerr.Errors[0]
+		}
+		if err != nil {
+			if !isOneShot {
+				continue
+			}
+			c.Lock()
+			c.pinned = (k + 1) % leps
+			c.Unlock()
+			return nil, nil, err
+		}
+		if k != pinned {
+			c.Lock()
+			c.pinned = k
+			c.Unlock()
+		}
+		return resp, body, nil
+	}
+
+	return nil, nil, cerr
+}
+
+func (c *httpClusterClient) Endpoints() []string {
+	c.RLock()
+	defer c.RUnlock()
+
+	eps := make([]string, len(c.endpoints))
+	for i, ep := range c.endpoints {
+		eps[i] = ep.String()
+	}
+
+	return eps
+}
+
+func (c *httpClusterClient) Sync(ctx context.Context) error {
+	mAPI := NewMembersAPI(c)
+	ms, err := mAPI.List(ctx)
+	if err != nil {
+		return err
+	}
+
+	var eps []string
+	for _, m := range ms {
+		eps = append(eps, m.ClientURLs...)
+	}
+
+	neps, err := c.parseEndpoints(eps)
+	if err != nil {
+		return err
+	}
+
+	npin := 0
+
+	switch c.selectionMode {
+	case EndpointSelectionRandom:
+		c.RLock()
+		eq := endpointsEqual(c.endpoints, neps)
+		c.RUnlock()
+
+		if eq {
+			return nil
+		}
+		// When items in the endpoint list changes, we choose a new pin
+		neps = shuffleEndpoints(c.rand, neps)
+	case EndpointSelectionPrioritizeLeader:
+		nle, err := c.getLeaderEndpoint(ctx, neps)
+		if err != nil {
+			return ErrNoLeaderEndpoint
+		}
+
+		for i, n := range neps {
+			if n.String() == nle {
+				npin = i
+				break
+			}
+		}
+	default:
+		return fmt.Errorf("invalid endpoint selection mode: %d", c.selectionMode)
+	}
+
+	c.Lock()
+	defer c.Unlock()
+	c.endpoints = neps
+	c.pinned = npin
+
+	return nil
+}
+
+func (c *httpClusterClient) AutoSync(ctx context.Context, interval time.Duration) error {
+	ticker := time.NewTicker(interval)
+	defer ticker.Stop()
+	for {
+		err := c.Sync(ctx)
+		if err != nil {
+			return err
+		}
+		select {
+		case <-ctx.Done():
+			return ctx.Err()
+		case <-ticker.C:
+		}
+	}
+}
+
+func (c *httpClusterClient) GetVersion(ctx context.Context) (*version.Versions, error) {
+	act := &getAction{Prefix: "/version"}
+
+	resp, body, err := c.Do(ctx, act)
+	if err != nil {
+		return nil, err
+	}
+
+	switch resp.StatusCode {
+	case http.StatusOK:
+		if len(body) == 0 {
+			return nil, ErrEmptyBody
+		}
+		var vresp version.Versions
+		if err := json.Unmarshal(body, &vresp); err != nil {
+			return nil, ErrInvalidJSON
+		}
+		return &vresp, nil
+	default:
+		var etcdErr Error
+		if err := json.Unmarshal(body, &etcdErr); err != nil {
+			return nil, ErrInvalidJSON
+		}
+		return nil, etcdErr
+	}
+}
+
+type roundTripResponse struct {
+	resp *http.Response
+	err  error
+}
+
+type simpleHTTPClient struct {
+	transport     CancelableTransport
+	endpoint      url.URL
+	headerTimeout time.Duration
+}
+
+func (c *simpleHTTPClient) Do(ctx context.Context, act httpAction) (*http.Response, []byte, error) {
+	req := act.HTTPRequest(c.endpoint)
+
+	if err := printcURL(req); err != nil {
+		return nil, nil, err
+	}
+
+	isWait := false
+	if req != nil && req.URL != nil {
+		ws := req.URL.Query().Get("wait")
+		if len(ws) != 0 {
+			var err error
+			isWait, err = strconv.ParseBool(ws)
+			if err != nil {
+				return nil, nil, fmt.Errorf("wrong wait value %s (%v for %+v)", ws, err, req)
+			}
+		}
+	}
+
+	var hctx context.Context
+	var hcancel context.CancelFunc
+	if !isWait && c.headerTimeout > 0 {
+		hctx, hcancel = context.WithTimeout(ctx, c.headerTimeout)
+	} else {
+		hctx, hcancel = context.WithCancel(ctx)
+	}
+	defer hcancel()
+
+	reqcancel := requestCanceler(c.transport, req)
+
+	rtchan := make(chan roundTripResponse, 1)
+	go func() {
+		resp, err := c.transport.RoundTrip(req)
+		rtchan <- roundTripResponse{resp: resp, err: err}
+		close(rtchan)
+	}()
+
+	var resp *http.Response
+	var err error
+
+	select {
+	case rtresp := <-rtchan:
+		resp, err = rtresp.resp, rtresp.err
+	case <-hctx.Done():
+		// cancel and wait for request to actually exit before continuing
+		reqcancel()
+		rtresp := <-rtchan
+		resp = rtresp.resp
+		switch {
+		case ctx.Err() != nil:
+			err = ctx.Err()
+		case hctx.Err() != nil:
+			err = fmt.Errorf("client: endpoint %s exceeded header timeout", c.endpoint.String())
+		default:
+			panic("failed to get error from context")
+		}
+	}
+
+	// always check for resp nil-ness to deal with possible
+	// race conditions between channels above
+	defer func() {
+		if resp != nil {
+			resp.Body.Close()
+		}
+	}()
+
+	if err != nil {
+		return nil, nil, err
+	}
+
+	var body []byte
+	done := make(chan struct{})
+	go func() {
+		body, err = ioutil.ReadAll(resp.Body)
+		done <- struct{}{}
+	}()
+
+	select {
+	case <-ctx.Done():
+		resp.Body.Close()
+		<-done
+		return nil, nil, ctx.Err()
+	case <-done:
+	}
+
+	return resp, body, err
+}
+
+type authedAction struct {
+	act         httpAction
+	credentials credentials
+}
+
+func (a *authedAction) HTTPRequest(url url.URL) *http.Request {
+	r := a.act.HTTPRequest(url)
+	r.SetBasicAuth(a.credentials.username, a.credentials.password)
+	return r
+}
+
+type redirectFollowingHTTPClient struct {
+	client        httpClient
+	checkRedirect CheckRedirectFunc
+}
+
+func (r *redirectFollowingHTTPClient) Do(ctx context.Context, act httpAction) (*http.Response, []byte, error) {
+	next := act
+	for i := 0; i < 100; i++ {
+		if i > 0 {
+			if err := r.checkRedirect(i); err != nil {
+				return nil, nil, err
+			}
+		}
+		resp, body, err := r.client.Do(ctx, next)
+		if err != nil {
+			return nil, nil, err
+		}
+		if resp.StatusCode/100 == 3 {
+			hdr := resp.Header.Get("Location")
+			if hdr == "" {
+				return nil, nil, fmt.Errorf("location header not set")
+			}
+			loc, err := url.Parse(hdr)
+			if err != nil {
+				return nil, nil, fmt.Errorf("location header not valid URL: %s", hdr)
+			}
+			next = &redirectedHTTPAction{
+				action:   act,
+				location: *loc,
+			}
+			continue
+		}
+		return resp, body, nil
+	}
+
+	return nil, nil, errTooManyRedirectChecks
+}
+
+type redirectedHTTPAction struct {
+	action   httpAction
+	location url.URL
+}
+
+func (r *redirectedHTTPAction) HTTPRequest(ep url.URL) *http.Request {
+	orig := r.action.HTTPRequest(ep)
+	orig.URL = &r.location
+	return orig
+}
+
+func shuffleEndpoints(r *rand.Rand, eps []url.URL) []url.URL {
+	// copied from Go 1.9<= rand.Rand.Perm
+	n := len(eps)
+	p := make([]int, n)
+	for i := 0; i < n; i++ {
+		j := r.Intn(i + 1)
+		p[i] = p[j]
+		p[j] = i
+	}
+	neps := make([]url.URL, n)
+	for i, k := range p {
+		neps[i] = eps[k]
+	}
+	return neps
+}
+
+func endpointsEqual(left, right []url.URL) bool {
+	if len(left) != len(right) {
+		return false
+	}
+
+	sLeft := make([]string, len(left))
+	sRight := make([]string, len(right))
+	for i, l := range left {
+		sLeft[i] = l.String()
+	}
+	for i, r := range right {
+		sRight[i] = r.String()
+	}
+
+	sort.Strings(sLeft)
+	sort.Strings(sRight)
+	for i := range sLeft {
+		if sLeft[i] != sRight[i] {
+			return false
+		}
+	}
+	return true
+}
diff --git a/vendor/go.etcd.io/etcd/client/cluster_error.go b/vendor/go.etcd.io/etcd/client/cluster_error.go
new file mode 100644
index 0000000..34618cd
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/client/cluster_error.go
@@ -0,0 +1,37 @@
+// Copyright 2015 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 client
+
+import "fmt"
+
+type ClusterError struct {
+	Errors []error
+}
+
+func (ce *ClusterError) Error() string {
+	s := ErrClusterUnavailable.Error()
+	for i, e := range ce.Errors {
+		s += fmt.Sprintf("; error #%d: %s\n", i, e)
+	}
+	return s
+}
+
+func (ce *ClusterError) Detail() string {
+	s := ""
+	for i, e := range ce.Errors {
+		s += fmt.Sprintf("error #%d: %s\n", i, e)
+	}
+	return s
+}
diff --git a/vendor/go.etcd.io/etcd/client/curl.go b/vendor/go.etcd.io/etcd/client/curl.go
new file mode 100644
index 0000000..c8bc9fb
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/client/curl.go
@@ -0,0 +1,70 @@
+// Copyright 2015 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 client
+
+import (
+	"bytes"
+	"fmt"
+	"io/ioutil"
+	"net/http"
+	"os"
+)
+
+var (
+	cURLDebug = false
+)
+
+func EnablecURLDebug() {
+	cURLDebug = true
+}
+
+func DisablecURLDebug() {
+	cURLDebug = false
+}
+
+// printcURL prints the cURL equivalent request to stderr.
+// It returns an error if the body of the request cannot
+// be read.
+// The caller MUST cancel the request if there is an error.
+func printcURL(req *http.Request) error {
+	if !cURLDebug {
+		return nil
+	}
+	var (
+		command string
+		b       []byte
+		err     error
+	)
+
+	if req.URL != nil {
+		command = fmt.Sprintf("curl -X %s %s", req.Method, req.URL.String())
+	}
+
+	if req.Body != nil {
+		b, err = ioutil.ReadAll(req.Body)
+		if err != nil {
+			return err
+		}
+		command += fmt.Sprintf(" -d %q", string(b))
+	}
+
+	fmt.Fprintf(os.Stderr, "cURL Command: %s\n", command)
+
+	// reset body
+	body := bytes.NewBuffer(b)
+	req.Body = ioutil.NopCloser(body)
+
+	return nil
+}
diff --git a/vendor/go.etcd.io/etcd/client/discover.go b/vendor/go.etcd.io/etcd/client/discover.go
new file mode 100644
index 0000000..580c256
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/client/discover.go
@@ -0,0 +1,40 @@
+// Copyright 2015 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 client
+
+import (
+	"go.etcd.io/etcd/pkg/srv"
+)
+
+// Discoverer is an interface that wraps the Discover method.
+type Discoverer interface {
+	// Discover looks up the etcd servers for the domain.
+	Discover(domain string, serviceName string) ([]string, error)
+}
+
+type srvDiscover struct{}
+
+// NewSRVDiscover constructs a new Discoverer that uses the stdlib to lookup SRV records.
+func NewSRVDiscover() Discoverer {
+	return &srvDiscover{}
+}
+
+func (d *srvDiscover) Discover(domain string, serviceName string) ([]string, error) {
+	srvs, err := srv.GetClient("etcd-client", domain, serviceName)
+	if err != nil {
+		return nil, err
+	}
+	return srvs.Endpoints, nil
+}
diff --git a/vendor/go.etcd.io/etcd/client/doc.go b/vendor/go.etcd.io/etcd/client/doc.go
new file mode 100644
index 0000000..abe5199
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/client/doc.go
@@ -0,0 +1,73 @@
+// Copyright 2015 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 client provides bindings for the etcd APIs.
+
+Create a Config and exchange it for a Client:
+
+	import (
+		"net/http"
+		"context"
+
+		"go.etcd.io/etcd/client"
+	)
+
+	cfg := client.Config{
+		Endpoints: []string{"http://127.0.0.1:2379"},
+		Transport: DefaultTransport,
+	}
+
+	c, err := client.New(cfg)
+	if err != nil {
+		// handle error
+	}
+
+Clients are safe for concurrent use by multiple goroutines.
+
+Create a KeysAPI using the Client, then use it to interact with etcd:
+
+	kAPI := client.NewKeysAPI(c)
+
+	// create a new key /foo with the value "bar"
+	_, err = kAPI.Create(context.Background(), "/foo", "bar")
+	if err != nil {
+		// handle error
+	}
+
+	// delete the newly created key only if the value is still "bar"
+	_, err = kAPI.Delete(context.Background(), "/foo", &DeleteOptions{PrevValue: "bar"})
+	if err != nil {
+		// handle error
+	}
+
+Use a custom context to set timeouts on your operations:
+
+	import "time"
+
+	ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
+	defer cancel()
+
+	// set a new key, ignoring its previous state
+	_, err := kAPI.Set(ctx, "/ping", "pong", nil)
+	if err != nil {
+		if err == context.DeadlineExceeded {
+			// request took longer than 5s
+		} else {
+			// handle error
+		}
+	}
+
+*/
+package client
diff --git a/vendor/go.etcd.io/etcd/client/json.go b/vendor/go.etcd.io/etcd/client/json.go
new file mode 100644
index 0000000..97cdbcd
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/client/json.go
@@ -0,0 +1,72 @@
+// Copyright 2019 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 client
+
+import (
+	"github.com/json-iterator/go"
+	"github.com/modern-go/reflect2"
+	"strconv"
+	"unsafe"
+)
+
+type customNumberExtension struct {
+	jsoniter.DummyExtension
+}
+
+func (cne *customNumberExtension) CreateDecoder(typ reflect2.Type) jsoniter.ValDecoder {
+	if typ.String() == "interface {}" {
+		return customNumberDecoder{}
+	}
+	return nil
+}
+
+type customNumberDecoder struct {
+}
+
+func (customNumberDecoder) Decode(ptr unsafe.Pointer, iter *jsoniter.Iterator) {
+	switch iter.WhatIsNext() {
+	case jsoniter.NumberValue:
+		var number jsoniter.Number
+		iter.ReadVal(&number)
+		i64, err := strconv.ParseInt(string(number), 10, 64)
+		if err == nil {
+			*(*interface{})(ptr) = i64
+			return
+		}
+		f64, err := strconv.ParseFloat(string(number), 64)
+		if err == nil {
+			*(*interface{})(ptr) = f64
+			return
+		}
+		iter.ReportError("DecodeNumber", err.Error())
+	default:
+		*(*interface{})(ptr) = iter.Read()
+	}
+}
+
+// caseSensitiveJsonIterator returns a jsoniterator API that's configured to be
+// case-sensitive when unmarshalling, and otherwise compatible with
+// the encoding/json standard library.
+func caseSensitiveJsonIterator() jsoniter.API {
+	config := jsoniter.Config{
+		EscapeHTML:             true,
+		SortMapKeys:            true,
+		ValidateJsonRawMessage: true,
+		CaseSensitive:          true,
+	}.Froze()
+	// Force jsoniter to decode number to interface{} via int64/float64, if possible.
+	config.RegisterExtension(&customNumberExtension{})
+	return config
+}
diff --git a/vendor/go.etcd.io/etcd/client/keys.go b/vendor/go.etcd.io/etcd/client/keys.go
new file mode 100644
index 0000000..ec53830
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/client/keys.go
@@ -0,0 +1,679 @@
+// Copyright 2015 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 client
+
+import (
+	"context"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"go.etcd.io/etcd/pkg/pathutil"
+	"net/http"
+	"net/url"
+	"strconv"
+	"strings"
+	"time"
+)
+
+const (
+	ErrorCodeKeyNotFound  = 100
+	ErrorCodeTestFailed   = 101
+	ErrorCodeNotFile      = 102
+	ErrorCodeNotDir       = 104
+	ErrorCodeNodeExist    = 105
+	ErrorCodeRootROnly    = 107
+	ErrorCodeDirNotEmpty  = 108
+	ErrorCodeUnauthorized = 110
+
+	ErrorCodePrevValueRequired = 201
+	ErrorCodeTTLNaN            = 202
+	ErrorCodeIndexNaN          = 203
+	ErrorCodeInvalidField      = 209
+	ErrorCodeInvalidForm       = 210
+
+	ErrorCodeRaftInternal = 300
+	ErrorCodeLeaderElect  = 301
+
+	ErrorCodeWatcherCleared    = 400
+	ErrorCodeEventIndexCleared = 401
+)
+
+type Error struct {
+	Code    int    `json:"errorCode"`
+	Message string `json:"message"`
+	Cause   string `json:"cause"`
+	Index   uint64 `json:"index"`
+}
+
+func (e Error) Error() string {
+	return fmt.Sprintf("%v: %v (%v) [%v]", e.Code, e.Message, e.Cause, e.Index)
+}
+
+var (
+	ErrInvalidJSON = errors.New("client: response is invalid json. The endpoint is probably not valid etcd cluster endpoint")
+	ErrEmptyBody   = errors.New("client: response body is empty")
+)
+
+// PrevExistType is used to define an existence condition when setting
+// or deleting Nodes.
+type PrevExistType string
+
+const (
+	PrevIgnore  = PrevExistType("")
+	PrevExist   = PrevExistType("true")
+	PrevNoExist = PrevExistType("false")
+)
+
+var (
+	defaultV2KeysPrefix = "/v2/keys"
+)
+
+// NewKeysAPI builds a KeysAPI that interacts with etcd's key-value
+// API over HTTP.
+func NewKeysAPI(c Client) KeysAPI {
+	return NewKeysAPIWithPrefix(c, defaultV2KeysPrefix)
+}
+
+// NewKeysAPIWithPrefix acts like NewKeysAPI, but allows the caller
+// to provide a custom base URL path. This should only be used in
+// very rare cases.
+func NewKeysAPIWithPrefix(c Client, p string) KeysAPI {
+	return &httpKeysAPI{
+		client: c,
+		prefix: p,
+	}
+}
+
+type KeysAPI interface {
+	// Get retrieves a set of Nodes from etcd
+	Get(ctx context.Context, key string, opts *GetOptions) (*Response, error)
+
+	// Set assigns a new value to a Node identified by a given key. The caller
+	// may define a set of conditions in the SetOptions. If SetOptions.Dir=true
+	// then value is ignored.
+	Set(ctx context.Context, key, value string, opts *SetOptions) (*Response, error)
+
+	// Delete removes a Node identified by the given key, optionally destroying
+	// all of its children as well. The caller may define a set of required
+	// conditions in an DeleteOptions object.
+	Delete(ctx context.Context, key string, opts *DeleteOptions) (*Response, error)
+
+	// Create is an alias for Set w/ PrevExist=false
+	Create(ctx context.Context, key, value string) (*Response, error)
+
+	// CreateInOrder is used to atomically create in-order keys within the given directory.
+	CreateInOrder(ctx context.Context, dir, value string, opts *CreateInOrderOptions) (*Response, error)
+
+	// Update is an alias for Set w/ PrevExist=true
+	Update(ctx context.Context, key, value string) (*Response, error)
+
+	// Watcher builds a new Watcher targeted at a specific Node identified
+	// by the given key. The Watcher may be configured at creation time
+	// through a WatcherOptions object. The returned Watcher is designed
+	// to emit events that happen to a Node, and optionally to its children.
+	Watcher(key string, opts *WatcherOptions) Watcher
+}
+
+type WatcherOptions struct {
+	// AfterIndex defines the index after-which the Watcher should
+	// start emitting events. For example, if a value of 5 is
+	// provided, the first event will have an index >= 6.
+	//
+	// Setting AfterIndex to 0 (default) means that the Watcher
+	// should start watching for events starting at the current
+	// index, whatever that may be.
+	AfterIndex uint64
+
+	// Recursive specifies whether or not the Watcher should emit
+	// events that occur in children of the given keyspace. If set
+	// to false (default), events will be limited to those that
+	// occur for the exact key.
+	Recursive bool
+}
+
+type CreateInOrderOptions struct {
+	// TTL defines a period of time after-which the Node should
+	// expire and no longer exist. Values <= 0 are ignored. Given
+	// that the zero-value is ignored, TTL cannot be used to set
+	// a TTL of 0.
+	TTL time.Duration
+}
+
+type SetOptions struct {
+	// PrevValue specifies what the current value of the Node must
+	// be in order for the Set operation to succeed.
+	//
+	// Leaving this field empty means that the caller wishes to
+	// ignore the current value of the Node. This cannot be used
+	// to compare the Node's current value to an empty string.
+	//
+	// PrevValue is ignored if Dir=true
+	PrevValue string
+
+	// PrevIndex indicates what the current ModifiedIndex of the
+	// Node must be in order for the Set operation to succeed.
+	//
+	// If PrevIndex is set to 0 (default), no comparison is made.
+	PrevIndex uint64
+
+	// PrevExist specifies whether the Node must currently exist
+	// (PrevExist) or not (PrevNoExist). If the caller does not
+	// care about existence, set PrevExist to PrevIgnore, or simply
+	// leave it unset.
+	PrevExist PrevExistType
+
+	// TTL defines a period of time after-which the Node should
+	// expire and no longer exist. Values <= 0 are ignored. Given
+	// that the zero-value is ignored, TTL cannot be used to set
+	// a TTL of 0.
+	TTL time.Duration
+
+	// Refresh set to true means a TTL value can be updated
+	// without firing a watch or changing the node value. A
+	// value must not be provided when refreshing a key.
+	Refresh bool
+
+	// Dir specifies whether or not this Node should be created as a directory.
+	Dir bool
+
+	// NoValueOnSuccess specifies whether the response contains the current value of the Node.
+	// If set, the response will only contain the current value when the request fails.
+	NoValueOnSuccess bool
+}
+
+type GetOptions struct {
+	// Recursive defines whether or not all children of the Node
+	// should be returned.
+	Recursive bool
+
+	// Sort instructs the server whether or not to sort the Nodes.
+	// If true, the Nodes are sorted alphabetically by key in
+	// ascending order (A to z). If false (default), the Nodes will
+	// not be sorted and the ordering used should not be considered
+	// predictable.
+	Sort bool
+
+	// Quorum specifies whether it gets the latest committed value that
+	// has been applied in quorum of members, which ensures external
+	// consistency (or linearizability).
+	Quorum bool
+}
+
+type DeleteOptions struct {
+	// PrevValue specifies what the current value of the Node must
+	// be in order for the Delete operation to succeed.
+	//
+	// Leaving this field empty means that the caller wishes to
+	// ignore the current value of the Node. This cannot be used
+	// to compare the Node's current value to an empty string.
+	PrevValue string
+
+	// PrevIndex indicates what the current ModifiedIndex of the
+	// Node must be in order for the Delete operation to succeed.
+	//
+	// If PrevIndex is set to 0 (default), no comparison is made.
+	PrevIndex uint64
+
+	// Recursive defines whether or not all children of the Node
+	// should be deleted. If set to true, all children of the Node
+	// identified by the given key will be deleted. If left unset
+	// or explicitly set to false, only a single Node will be
+	// deleted.
+	Recursive bool
+
+	// Dir specifies whether or not this Node should be removed as a directory.
+	Dir bool
+}
+
+type Watcher interface {
+	// Next blocks until an etcd event occurs, then returns a Response
+	// representing that event. The behavior of Next depends on the
+	// WatcherOptions used to construct the Watcher. Next is designed to
+	// be called repeatedly, each time blocking until a subsequent event
+	// is available.
+	//
+	// If the provided context is cancelled, Next will return a non-nil
+	// error. Any other failures encountered while waiting for the next
+	// event (connection issues, deserialization failures, etc) will
+	// also result in a non-nil error.
+	Next(context.Context) (*Response, error)
+}
+
+type Response struct {
+	// Action is the name of the operation that occurred. Possible values
+	// include get, set, delete, update, create, compareAndSwap,
+	// compareAndDelete and expire.
+	Action string `json:"action"`
+
+	// Node represents the state of the relevant etcd Node.
+	Node *Node `json:"node"`
+
+	// PrevNode represents the previous state of the Node. PrevNode is non-nil
+	// only if the Node existed before the action occurred and the action
+	// caused a change to the Node.
+	PrevNode *Node `json:"prevNode"`
+
+	// Index holds the cluster-level index at the time the Response was generated.
+	// This index is not tied to the Node(s) contained in this Response.
+	Index uint64 `json:"-"`
+
+	// ClusterID holds the cluster-level ID reported by the server.  This
+	// should be different for different etcd clusters.
+	ClusterID string `json:"-"`
+}
+
+type Node struct {
+	// Key represents the unique location of this Node (e.g. "/foo/bar").
+	Key string `json:"key"`
+
+	// Dir reports whether node describes a directory.
+	Dir bool `json:"dir,omitempty"`
+
+	// Value is the current data stored on this Node. If this Node
+	// is a directory, Value will be empty.
+	Value string `json:"value"`
+
+	// Nodes holds the children of this Node, only if this Node is a directory.
+	// This slice of will be arbitrarily deep (children, grandchildren, great-
+	// grandchildren, etc.) if a recursive Get or Watch request were made.
+	Nodes Nodes `json:"nodes"`
+
+	// CreatedIndex is the etcd index at-which this Node was created.
+	CreatedIndex uint64 `json:"createdIndex"`
+
+	// ModifiedIndex is the etcd index at-which this Node was last modified.
+	ModifiedIndex uint64 `json:"modifiedIndex"`
+
+	// Expiration is the server side expiration time of the key.
+	Expiration *time.Time `json:"expiration,omitempty"`
+
+	// TTL is the time to live of the key in second.
+	TTL int64 `json:"ttl,omitempty"`
+}
+
+func (n *Node) String() string {
+	return fmt.Sprintf("{Key: %s, CreatedIndex: %d, ModifiedIndex: %d, TTL: %d}", n.Key, n.CreatedIndex, n.ModifiedIndex, n.TTL)
+}
+
+// TTLDuration returns the Node's TTL as a time.Duration object
+func (n *Node) TTLDuration() time.Duration {
+	return time.Duration(n.TTL) * time.Second
+}
+
+type Nodes []*Node
+
+// interfaces for sorting
+
+func (ns Nodes) Len() int           { return len(ns) }
+func (ns Nodes) Less(i, j int) bool { return ns[i].Key < ns[j].Key }
+func (ns Nodes) Swap(i, j int)      { ns[i], ns[j] = ns[j], ns[i] }
+
+type httpKeysAPI struct {
+	client httpClient
+	prefix string
+}
+
+func (k *httpKeysAPI) Set(ctx context.Context, key, val string, opts *SetOptions) (*Response, error) {
+	act := &setAction{
+		Prefix: k.prefix,
+		Key:    key,
+		Value:  val,
+	}
+
+	if opts != nil {
+		act.PrevValue = opts.PrevValue
+		act.PrevIndex = opts.PrevIndex
+		act.PrevExist = opts.PrevExist
+		act.TTL = opts.TTL
+		act.Refresh = opts.Refresh
+		act.Dir = opts.Dir
+		act.NoValueOnSuccess = opts.NoValueOnSuccess
+	}
+
+	doCtx := ctx
+	if act.PrevExist == PrevNoExist {
+		doCtx = context.WithValue(doCtx, &oneShotCtxValue, &oneShotCtxValue)
+	}
+	resp, body, err := k.client.Do(doCtx, act)
+	if err != nil {
+		return nil, err
+	}
+
+	return unmarshalHTTPResponse(resp.StatusCode, resp.Header, body)
+}
+
+func (k *httpKeysAPI) Create(ctx context.Context, key, val string) (*Response, error) {
+	return k.Set(ctx, key, val, &SetOptions{PrevExist: PrevNoExist})
+}
+
+func (k *httpKeysAPI) CreateInOrder(ctx context.Context, dir, val string, opts *CreateInOrderOptions) (*Response, error) {
+	act := &createInOrderAction{
+		Prefix: k.prefix,
+		Dir:    dir,
+		Value:  val,
+	}
+
+	if opts != nil {
+		act.TTL = opts.TTL
+	}
+
+	resp, body, err := k.client.Do(ctx, act)
+	if err != nil {
+		return nil, err
+	}
+
+	return unmarshalHTTPResponse(resp.StatusCode, resp.Header, body)
+}
+
+func (k *httpKeysAPI) Update(ctx context.Context, key, val string) (*Response, error) {
+	return k.Set(ctx, key, val, &SetOptions{PrevExist: PrevExist})
+}
+
+func (k *httpKeysAPI) Delete(ctx context.Context, key string, opts *DeleteOptions) (*Response, error) {
+	act := &deleteAction{
+		Prefix: k.prefix,
+		Key:    key,
+	}
+
+	if opts != nil {
+		act.PrevValue = opts.PrevValue
+		act.PrevIndex = opts.PrevIndex
+		act.Dir = opts.Dir
+		act.Recursive = opts.Recursive
+	}
+
+	doCtx := context.WithValue(ctx, &oneShotCtxValue, &oneShotCtxValue)
+	resp, body, err := k.client.Do(doCtx, act)
+	if err != nil {
+		return nil, err
+	}
+
+	return unmarshalHTTPResponse(resp.StatusCode, resp.Header, body)
+}
+
+func (k *httpKeysAPI) Get(ctx context.Context, key string, opts *GetOptions) (*Response, error) {
+	act := &getAction{
+		Prefix: k.prefix,
+		Key:    key,
+	}
+
+	if opts != nil {
+		act.Recursive = opts.Recursive
+		act.Sorted = opts.Sort
+		act.Quorum = opts.Quorum
+	}
+
+	resp, body, err := k.client.Do(ctx, act)
+	if err != nil {
+		return nil, err
+	}
+
+	return unmarshalHTTPResponse(resp.StatusCode, resp.Header, body)
+}
+
+func (k *httpKeysAPI) Watcher(key string, opts *WatcherOptions) Watcher {
+	act := waitAction{
+		Prefix: k.prefix,
+		Key:    key,
+	}
+
+	if opts != nil {
+		act.Recursive = opts.Recursive
+		if opts.AfterIndex > 0 {
+			act.WaitIndex = opts.AfterIndex + 1
+		}
+	}
+
+	return &httpWatcher{
+		client:   k.client,
+		nextWait: act,
+	}
+}
+
+type httpWatcher struct {
+	client   httpClient
+	nextWait waitAction
+}
+
+func (hw *httpWatcher) Next(ctx context.Context) (*Response, error) {
+	for {
+		httpresp, body, err := hw.client.Do(ctx, &hw.nextWait)
+		if err != nil {
+			return nil, err
+		}
+
+		resp, err := unmarshalHTTPResponse(httpresp.StatusCode, httpresp.Header, body)
+		if err != nil {
+			if err == ErrEmptyBody {
+				continue
+			}
+			return nil, err
+		}
+
+		hw.nextWait.WaitIndex = resp.Node.ModifiedIndex + 1
+		return resp, nil
+	}
+}
+
+// v2KeysURL forms a URL representing the location of a key.
+// The endpoint argument represents the base URL of an etcd
+// server. The prefix is the path needed to route from the
+// provided endpoint's path to the root of the keys API
+// (typically "/v2/keys").
+func v2KeysURL(ep url.URL, prefix, key string) *url.URL {
+	// We concatenate all parts together manually. We cannot use
+	// path.Join because it does not reserve trailing slash.
+	// We call CanonicalURLPath to further cleanup the path.
+	if prefix != "" && prefix[0] != '/' {
+		prefix = "/" + prefix
+	}
+	if key != "" && key[0] != '/' {
+		key = "/" + key
+	}
+	ep.Path = pathutil.CanonicalURLPath(ep.Path + prefix + key)
+	return &ep
+}
+
+type getAction struct {
+	Prefix    string
+	Key       string
+	Recursive bool
+	Sorted    bool
+	Quorum    bool
+}
+
+func (g *getAction) HTTPRequest(ep url.URL) *http.Request {
+	u := v2KeysURL(ep, g.Prefix, g.Key)
+
+	params := u.Query()
+	params.Set("recursive", strconv.FormatBool(g.Recursive))
+	params.Set("sorted", strconv.FormatBool(g.Sorted))
+	params.Set("quorum", strconv.FormatBool(g.Quorum))
+	u.RawQuery = params.Encode()
+
+	req, _ := http.NewRequest("GET", u.String(), nil)
+	return req
+}
+
+type waitAction struct {
+	Prefix    string
+	Key       string
+	WaitIndex uint64
+	Recursive bool
+}
+
+func (w *waitAction) HTTPRequest(ep url.URL) *http.Request {
+	u := v2KeysURL(ep, w.Prefix, w.Key)
+
+	params := u.Query()
+	params.Set("wait", "true")
+	params.Set("waitIndex", strconv.FormatUint(w.WaitIndex, 10))
+	params.Set("recursive", strconv.FormatBool(w.Recursive))
+	u.RawQuery = params.Encode()
+
+	req, _ := http.NewRequest("GET", u.String(), nil)
+	return req
+}
+
+type setAction struct {
+	Prefix           string
+	Key              string
+	Value            string
+	PrevValue        string
+	PrevIndex        uint64
+	PrevExist        PrevExistType
+	TTL              time.Duration
+	Refresh          bool
+	Dir              bool
+	NoValueOnSuccess bool
+}
+
+func (a *setAction) HTTPRequest(ep url.URL) *http.Request {
+	u := v2KeysURL(ep, a.Prefix, a.Key)
+
+	params := u.Query()
+	form := url.Values{}
+
+	// we're either creating a directory or setting a key
+	if a.Dir {
+		params.Set("dir", strconv.FormatBool(a.Dir))
+	} else {
+		// These options are only valid for setting a key
+		if a.PrevValue != "" {
+			params.Set("prevValue", a.PrevValue)
+		}
+		form.Add("value", a.Value)
+	}
+
+	// Options which apply to both setting a key and creating a dir
+	if a.PrevIndex != 0 {
+		params.Set("prevIndex", strconv.FormatUint(a.PrevIndex, 10))
+	}
+	if a.PrevExist != PrevIgnore {
+		params.Set("prevExist", string(a.PrevExist))
+	}
+	if a.TTL > 0 {
+		form.Add("ttl", strconv.FormatUint(uint64(a.TTL.Seconds()), 10))
+	}
+
+	if a.Refresh {
+		form.Add("refresh", "true")
+	}
+	if a.NoValueOnSuccess {
+		params.Set("noValueOnSuccess", strconv.FormatBool(a.NoValueOnSuccess))
+	}
+
+	u.RawQuery = params.Encode()
+	body := strings.NewReader(form.Encode())
+
+	req, _ := http.NewRequest("PUT", u.String(), body)
+	req.Header.Set("Content-Type", "application/x-www-form-urlencoded")
+
+	return req
+}
+
+type deleteAction struct {
+	Prefix    string
+	Key       string
+	PrevValue string
+	PrevIndex uint64
+	Dir       bool
+	Recursive bool
+}
+
+func (a *deleteAction) HTTPRequest(ep url.URL) *http.Request {
+	u := v2KeysURL(ep, a.Prefix, a.Key)
+
+	params := u.Query()
+	if a.PrevValue != "" {
+		params.Set("prevValue", a.PrevValue)
+	}
+	if a.PrevIndex != 0 {
+		params.Set("prevIndex", strconv.FormatUint(a.PrevIndex, 10))
+	}
+	if a.Dir {
+		params.Set("dir", "true")
+	}
+	if a.Recursive {
+		params.Set("recursive", "true")
+	}
+	u.RawQuery = params.Encode()
+
+	req, _ := http.NewRequest("DELETE", u.String(), nil)
+	req.Header.Set("Content-Type", "application/x-www-form-urlencoded")
+
+	return req
+}
+
+type createInOrderAction struct {
+	Prefix string
+	Dir    string
+	Value  string
+	TTL    time.Duration
+}
+
+func (a *createInOrderAction) HTTPRequest(ep url.URL) *http.Request {
+	u := v2KeysURL(ep, a.Prefix, a.Dir)
+
+	form := url.Values{}
+	form.Add("value", a.Value)
+	if a.TTL > 0 {
+		form.Add("ttl", strconv.FormatUint(uint64(a.TTL.Seconds()), 10))
+	}
+	body := strings.NewReader(form.Encode())
+
+	req, _ := http.NewRequest("POST", u.String(), body)
+	req.Header.Set("Content-Type", "application/x-www-form-urlencoded")
+	return req
+}
+
+func unmarshalHTTPResponse(code int, header http.Header, body []byte) (res *Response, err error) {
+	switch code {
+	case http.StatusOK, http.StatusCreated:
+		if len(body) == 0 {
+			return nil, ErrEmptyBody
+		}
+		res, err = unmarshalSuccessfulKeysResponse(header, body)
+	default:
+		err = unmarshalFailedKeysResponse(body)
+	}
+	return res, err
+}
+
+var jsonIterator = caseSensitiveJsonIterator()
+
+func unmarshalSuccessfulKeysResponse(header http.Header, body []byte) (*Response, error) {
+	var res Response
+	err := jsonIterator.Unmarshal(body, &res)
+	if err != nil {
+		return nil, ErrInvalidJSON
+	}
+	if header.Get("X-Etcd-Index") != "" {
+		res.Index, err = strconv.ParseUint(header.Get("X-Etcd-Index"), 10, 64)
+		if err != nil {
+			return nil, err
+		}
+	}
+	res.ClusterID = header.Get("X-Etcd-Cluster-ID")
+	return &res, nil
+}
+
+func unmarshalFailedKeysResponse(body []byte) error {
+	var etcdErr Error
+	if err := json.Unmarshal(body, &etcdErr); err != nil {
+		return ErrInvalidJSON
+	}
+	return etcdErr
+}
diff --git a/vendor/go.etcd.io/etcd/client/members.go b/vendor/go.etcd.io/etcd/client/members.go
new file mode 100644
index 0000000..657131a
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/client/members.go
@@ -0,0 +1,303 @@
+// Copyright 2015 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 client
+
+import (
+	"bytes"
+	"context"
+	"encoding/json"
+	"fmt"
+	"net/http"
+	"net/url"
+	"path"
+
+	"go.etcd.io/etcd/pkg/types"
+)
+
+var (
+	defaultV2MembersPrefix = "/v2/members"
+	defaultLeaderSuffix    = "/leader"
+)
+
+type Member struct {
+	// ID is the unique identifier of this Member.
+	ID string `json:"id"`
+
+	// Name is a human-readable, non-unique identifier of this Member.
+	Name string `json:"name"`
+
+	// PeerURLs represents the HTTP(S) endpoints this Member uses to
+	// participate in etcd's consensus protocol.
+	PeerURLs []string `json:"peerURLs"`
+
+	// ClientURLs represents the HTTP(S) endpoints on which this Member
+	// serves its client-facing APIs.
+	ClientURLs []string `json:"clientURLs"`
+}
+
+type memberCollection []Member
+
+func (c *memberCollection) UnmarshalJSON(data []byte) error {
+	d := struct {
+		Members []Member
+	}{}
+
+	if err := json.Unmarshal(data, &d); err != nil {
+		return err
+	}
+
+	if d.Members == nil {
+		*c = make([]Member, 0)
+		return nil
+	}
+
+	*c = d.Members
+	return nil
+}
+
+type memberCreateOrUpdateRequest struct {
+	PeerURLs types.URLs
+}
+
+func (m *memberCreateOrUpdateRequest) MarshalJSON() ([]byte, error) {
+	s := struct {
+		PeerURLs []string `json:"peerURLs"`
+	}{
+		PeerURLs: make([]string, len(m.PeerURLs)),
+	}
+
+	for i, u := range m.PeerURLs {
+		s.PeerURLs[i] = u.String()
+	}
+
+	return json.Marshal(&s)
+}
+
+// NewMembersAPI constructs a new MembersAPI that uses HTTP to
+// interact with etcd's membership API.
+func NewMembersAPI(c Client) MembersAPI {
+	return &httpMembersAPI{
+		client: c,
+	}
+}
+
+type MembersAPI interface {
+	// List enumerates the current cluster membership.
+	List(ctx context.Context) ([]Member, error)
+
+	// Add instructs etcd to accept a new Member into the cluster.
+	Add(ctx context.Context, peerURL string) (*Member, error)
+
+	// Remove demotes an existing Member out of the cluster.
+	Remove(ctx context.Context, mID string) error
+
+	// Update instructs etcd to update an existing Member in the cluster.
+	Update(ctx context.Context, mID string, peerURLs []string) error
+
+	// Leader gets current leader of the cluster
+	Leader(ctx context.Context) (*Member, error)
+}
+
+type httpMembersAPI struct {
+	client httpClient
+}
+
+func (m *httpMembersAPI) List(ctx context.Context) ([]Member, error) {
+	req := &membersAPIActionList{}
+	resp, body, err := m.client.Do(ctx, req)
+	if err != nil {
+		return nil, err
+	}
+
+	if err := assertStatusCode(resp.StatusCode, http.StatusOK); err != nil {
+		return nil, err
+	}
+
+	var mCollection memberCollection
+	if err := json.Unmarshal(body, &mCollection); err != nil {
+		return nil, err
+	}
+
+	return []Member(mCollection), nil
+}
+
+func (m *httpMembersAPI) Add(ctx context.Context, peerURL string) (*Member, error) {
+	urls, err := types.NewURLs([]string{peerURL})
+	if err != nil {
+		return nil, err
+	}
+
+	req := &membersAPIActionAdd{peerURLs: urls}
+	resp, body, err := m.client.Do(ctx, req)
+	if err != nil {
+		return nil, err
+	}
+
+	if err := assertStatusCode(resp.StatusCode, http.StatusCreated, http.StatusConflict); err != nil {
+		return nil, err
+	}
+
+	if resp.StatusCode != http.StatusCreated {
+		var merr membersError
+		if err := json.Unmarshal(body, &merr); err != nil {
+			return nil, err
+		}
+		return nil, merr
+	}
+
+	var memb Member
+	if err := json.Unmarshal(body, &memb); err != nil {
+		return nil, err
+	}
+
+	return &memb, nil
+}
+
+func (m *httpMembersAPI) Update(ctx context.Context, memberID string, peerURLs []string) error {
+	urls, err := types.NewURLs(peerURLs)
+	if err != nil {
+		return err
+	}
+
+	req := &membersAPIActionUpdate{peerURLs: urls, memberID: memberID}
+	resp, body, err := m.client.Do(ctx, req)
+	if err != nil {
+		return err
+	}
+
+	if err := assertStatusCode(resp.StatusCode, http.StatusNoContent, http.StatusNotFound, http.StatusConflict); err != nil {
+		return err
+	}
+
+	if resp.StatusCode != http.StatusNoContent {
+		var merr membersError
+		if err := json.Unmarshal(body, &merr); err != nil {
+			return err
+		}
+		return merr
+	}
+
+	return nil
+}
+
+func (m *httpMembersAPI) Remove(ctx context.Context, memberID string) error {
+	req := &membersAPIActionRemove{memberID: memberID}
+	resp, _, err := m.client.Do(ctx, req)
+	if err != nil {
+		return err
+	}
+
+	return assertStatusCode(resp.StatusCode, http.StatusNoContent, http.StatusGone)
+}
+
+func (m *httpMembersAPI) Leader(ctx context.Context) (*Member, error) {
+	req := &membersAPIActionLeader{}
+	resp, body, err := m.client.Do(ctx, req)
+	if err != nil {
+		return nil, err
+	}
+
+	if err := assertStatusCode(resp.StatusCode, http.StatusOK); err != nil {
+		return nil, err
+	}
+
+	var leader Member
+	if err := json.Unmarshal(body, &leader); err != nil {
+		return nil, err
+	}
+
+	return &leader, nil
+}
+
+type membersAPIActionList struct{}
+
+func (l *membersAPIActionList) HTTPRequest(ep url.URL) *http.Request {
+	u := v2MembersURL(ep)
+	req, _ := http.NewRequest("GET", u.String(), nil)
+	return req
+}
+
+type membersAPIActionRemove struct {
+	memberID string
+}
+
+func (d *membersAPIActionRemove) HTTPRequest(ep url.URL) *http.Request {
+	u := v2MembersURL(ep)
+	u.Path = path.Join(u.Path, d.memberID)
+	req, _ := http.NewRequest("DELETE", u.String(), nil)
+	return req
+}
+
+type membersAPIActionAdd struct {
+	peerURLs types.URLs
+}
+
+func (a *membersAPIActionAdd) HTTPRequest(ep url.URL) *http.Request {
+	u := v2MembersURL(ep)
+	m := memberCreateOrUpdateRequest{PeerURLs: a.peerURLs}
+	b, _ := json.Marshal(&m)
+	req, _ := http.NewRequest("POST", u.String(), bytes.NewReader(b))
+	req.Header.Set("Content-Type", "application/json")
+	return req
+}
+
+type membersAPIActionUpdate struct {
+	memberID string
+	peerURLs types.URLs
+}
+
+func (a *membersAPIActionUpdate) HTTPRequest(ep url.URL) *http.Request {
+	u := v2MembersURL(ep)
+	m := memberCreateOrUpdateRequest{PeerURLs: a.peerURLs}
+	u.Path = path.Join(u.Path, a.memberID)
+	b, _ := json.Marshal(&m)
+	req, _ := http.NewRequest("PUT", u.String(), bytes.NewReader(b))
+	req.Header.Set("Content-Type", "application/json")
+	return req
+}
+
+func assertStatusCode(got int, want ...int) (err error) {
+	for _, w := range want {
+		if w == got {
+			return nil
+		}
+	}
+	return fmt.Errorf("unexpected status code %d", got)
+}
+
+type membersAPIActionLeader struct{}
+
+func (l *membersAPIActionLeader) HTTPRequest(ep url.URL) *http.Request {
+	u := v2MembersURL(ep)
+	u.Path = path.Join(u.Path, defaultLeaderSuffix)
+	req, _ := http.NewRequest("GET", u.String(), nil)
+	return req
+}
+
+// v2MembersURL add the necessary path to the provided endpoint
+// to route requests to the default v2 members API.
+func v2MembersURL(ep url.URL) *url.URL {
+	ep.Path = path.Join(ep.Path, defaultV2MembersPrefix)
+	return &ep
+}
+
+type membersError struct {
+	Message string `json:"message"`
+	Code    int    `json:"-"`
+}
+
+func (e membersError) Error() string {
+	return e.Message
+}
diff --git a/vendor/go.etcd.io/etcd/client/util.go b/vendor/go.etcd.io/etcd/client/util.go
new file mode 100644
index 0000000..15a8bab
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/client/util.go
@@ -0,0 +1,53 @@
+// Copyright 2016 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 client
+
+import (
+	"regexp"
+)
+
+var (
+	roleNotFoundRegExp *regexp.Regexp
+	userNotFoundRegExp *regexp.Regexp
+)
+
+func init() {
+	roleNotFoundRegExp = regexp.MustCompile("auth: Role .* does not exist.")
+	userNotFoundRegExp = regexp.MustCompile("auth: User .* does not exist.")
+}
+
+// IsKeyNotFound returns true if the error code is ErrorCodeKeyNotFound.
+func IsKeyNotFound(err error) bool {
+	if cErr, ok := err.(Error); ok {
+		return cErr.Code == ErrorCodeKeyNotFound
+	}
+	return false
+}
+
+// IsRoleNotFound returns true if the error means role not found of v2 API.
+func IsRoleNotFound(err error) bool {
+	if ae, ok := err.(authError); ok {
+		return roleNotFoundRegExp.MatchString(ae.Message)
+	}
+	return false
+}
+
+// IsUserNotFound returns true if the error means user not found of v2 API.
+func IsUserNotFound(err error) bool {
+	if ae, ok := err.(authError); ok {
+		return userNotFoundRegExp.MatchString(ae.Message)
+	}
+	return false
+}
diff --git a/vendor/go.etcd.io/etcd/embed/config.go b/vendor/go.etcd.io/etcd/embed/config.go
new file mode 100644
index 0000000..2f64d92
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/embed/config.go
@@ -0,0 +1,915 @@
+// Copyright 2016 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 embed
+
+import (
+	"crypto/tls"
+	"fmt"
+	"io/ioutil"
+	"net"
+	"net/http"
+	"net/url"
+	"os"
+	"path/filepath"
+	"strings"
+	"sync"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/api/v3compactor"
+	"go.etcd.io/etcd/pkg/flags"
+	"go.etcd.io/etcd/pkg/logutil"
+	"go.etcd.io/etcd/pkg/netutil"
+	"go.etcd.io/etcd/pkg/srv"
+	"go.etcd.io/etcd/pkg/tlsutil"
+	"go.etcd.io/etcd/pkg/transport"
+	"go.etcd.io/etcd/pkg/types"
+
+	bolt "go.etcd.io/bbolt"
+	"go.uber.org/zap"
+	"go.uber.org/zap/zapcore"
+	"golang.org/x/crypto/bcrypt"
+	"google.golang.org/grpc"
+	"sigs.k8s.io/yaml"
+)
+
+const (
+	ClusterStateFlagNew      = "new"
+	ClusterStateFlagExisting = "existing"
+
+	DefaultName                  = "default"
+	DefaultMaxSnapshots          = 5
+	DefaultMaxWALs               = 5
+	DefaultMaxTxnOps             = uint(128)
+	DefaultMaxRequestBytes       = 1.5 * 1024 * 1024
+	DefaultGRPCKeepAliveMinTime  = 5 * time.Second
+	DefaultGRPCKeepAliveInterval = 2 * time.Hour
+	DefaultGRPCKeepAliveTimeout  = 20 * time.Second
+
+	DefaultListenPeerURLs   = "http://localhost:2380"
+	DefaultListenClientURLs = "http://localhost:2379"
+
+	DefaultLogOutput = "default"
+	JournalLogOutput = "systemd/journal"
+	StdErrLogOutput  = "stderr"
+	StdOutLogOutput  = "stdout"
+
+	// DefaultStrictReconfigCheck is the default value for "--strict-reconfig-check" flag.
+	// It's enabled by default.
+	DefaultStrictReconfigCheck = true
+	// DefaultEnableV2 is the default value for "--enable-v2" flag.
+	// v2 API is disabled by default.
+	DefaultEnableV2 = false
+
+	// maxElectionMs specifies the maximum value of election timeout.
+	// More details are listed in ../Documentation/tuning.md#time-parameters.
+	maxElectionMs = 50000
+	// backend freelist map type
+	freelistMapType = "map"
+)
+
+var (
+	ErrConflictBootstrapFlags = fmt.Errorf("multiple discovery or bootstrap flags are set. " +
+		"Choose one of \"initial-cluster\", \"discovery\" or \"discovery-srv\"")
+	ErrUnsetAdvertiseClientURLsFlag = fmt.Errorf("--advertise-client-urls is required when --listen-client-urls is set explicitly")
+
+	DefaultInitialAdvertisePeerURLs = "http://localhost:2380"
+	DefaultAdvertiseClientURLs      = "http://localhost:2379"
+
+	defaultHostname   string
+	defaultHostStatus error
+)
+
+var (
+	// CompactorModePeriodic is periodic compaction mode
+	// for "Config.AutoCompactionMode" field.
+	// If "AutoCompactionMode" is CompactorModePeriodic and
+	// "AutoCompactionRetention" is "1h", it automatically compacts
+	// compacts storage every hour.
+	CompactorModePeriodic = v3compactor.ModePeriodic
+
+	// CompactorModeRevision is revision-based compaction mode
+	// for "Config.AutoCompactionMode" field.
+	// If "AutoCompactionMode" is CompactorModeRevision and
+	// "AutoCompactionRetention" is "1000", it compacts log on
+	// revision 5000 when the current revision is 6000.
+	// This runs every 5-minute if enough of logs have proceeded.
+	CompactorModeRevision = v3compactor.ModeRevision
+)
+
+func init() {
+	defaultHostname, defaultHostStatus = netutil.GetDefaultHost()
+}
+
+// Config holds the arguments for configuring an etcd server.
+type Config struct {
+	Name   string `json:"name"`
+	Dir    string `json:"data-dir"`
+	WalDir string `json:"wal-dir"`
+
+	SnapshotCount uint64 `json:"snapshot-count"`
+
+	// SnapshotCatchUpEntries is the number of entries for a slow follower
+	// to catch-up after compacting the raft storage entries.
+	// We expect the follower has a millisecond level latency with the leader.
+	// The max throughput is around 10K. Keep a 5K entries is enough for helping
+	// follower to catch up.
+	// WARNING: only change this for tests.
+	// Always use "DefaultSnapshotCatchUpEntries"
+	SnapshotCatchUpEntries uint64
+
+	MaxSnapFiles uint `json:"max-snapshots"`
+	MaxWalFiles  uint `json:"max-wals"`
+
+	// TickMs is the number of milliseconds between heartbeat ticks.
+	// TODO: decouple tickMs and heartbeat tick (current heartbeat tick = 1).
+	// make ticks a cluster wide configuration.
+	TickMs     uint `json:"heartbeat-interval"`
+	ElectionMs uint `json:"election-timeout"`
+
+	// InitialElectionTickAdvance is true, then local member fast-forwards
+	// election ticks to speed up "initial" leader election trigger. This
+	// benefits the case of larger election ticks. For instance, cross
+	// datacenter deployment may require longer election timeout of 10-second.
+	// If true, local node does not need wait up to 10-second. Instead,
+	// forwards its election ticks to 8-second, and have only 2-second left
+	// before leader election.
+	//
+	// Major assumptions are that:
+	//  - cluster has no active leader thus advancing ticks enables faster
+	//    leader election, or
+	//  - cluster already has an established leader, and rejoining follower
+	//    is likely to receive heartbeats from the leader after tick advance
+	//    and before election timeout.
+	//
+	// However, when network from leader to rejoining follower is congested,
+	// and the follower does not receive leader heartbeat within left election
+	// ticks, disruptive election has to happen thus affecting cluster
+	// availabilities.
+	//
+	// Disabling this would slow down initial bootstrap process for cross
+	// datacenter deployments. Make your own tradeoffs by configuring
+	// --initial-election-tick-advance at the cost of slow initial bootstrap.
+	//
+	// If single-node, it advances ticks regardless.
+	//
+	// See https://github.com/etcd-io/etcd/issues/9333 for more detail.
+	InitialElectionTickAdvance bool `json:"initial-election-tick-advance"`
+
+	// BackendBatchInterval is the maximum time before commit the backend transaction.
+	BackendBatchInterval time.Duration `json:"backend-batch-interval"`
+	// BackendBatchLimit is the maximum operations before commit the backend transaction.
+	BackendBatchLimit int   `json:"backend-batch-limit"`
+	QuotaBackendBytes int64 `json:"quota-backend-bytes"`
+	MaxTxnOps         uint  `json:"max-txn-ops"`
+	MaxRequestBytes   uint  `json:"max-request-bytes"`
+
+	LPUrls, LCUrls []url.URL
+	APUrls, ACUrls []url.URL
+	ClientTLSInfo  transport.TLSInfo
+	ClientAutoTLS  bool
+	PeerTLSInfo    transport.TLSInfo
+	PeerAutoTLS    bool
+
+	// CipherSuites is a list of supported TLS cipher suites between
+	// client/server and peers. If empty, Go auto-populates the list.
+	// Note that cipher suites are prioritized in the given order.
+	CipherSuites []string `json:"cipher-suites"`
+
+	ClusterState          string `json:"initial-cluster-state"`
+	DNSCluster            string `json:"discovery-srv"`
+	DNSClusterServiceName string `json:"discovery-srv-name"`
+	Dproxy                string `json:"discovery-proxy"`
+	Durl                  string `json:"discovery"`
+	InitialCluster        string `json:"initial-cluster"`
+	InitialClusterToken   string `json:"initial-cluster-token"`
+	StrictReconfigCheck   bool   `json:"strict-reconfig-check"`
+	EnableV2              bool   `json:"enable-v2"`
+
+	// AutoCompactionMode is either 'periodic' or 'revision'.
+	AutoCompactionMode string `json:"auto-compaction-mode"`
+	// AutoCompactionRetention is either duration string with time unit
+	// (e.g. '5m' for 5-minute), or revision unit (e.g. '5000').
+	// If no time unit is provided and compaction mode is 'periodic',
+	// the unit defaults to hour. For example, '5' translates into 5-hour.
+	AutoCompactionRetention string `json:"auto-compaction-retention"`
+
+	// GRPCKeepAliveMinTime is the minimum interval that a client should
+	// wait before pinging server. When client pings "too fast", server
+	// sends goaway and closes the connection (errors: too_many_pings,
+	// http2.ErrCodeEnhanceYourCalm). When too slow, nothing happens.
+	// Server expects client pings only when there is any active streams
+	// (PermitWithoutStream is set false).
+	GRPCKeepAliveMinTime time.Duration `json:"grpc-keepalive-min-time"`
+	// GRPCKeepAliveInterval is the frequency of server-to-client ping
+	// to check if a connection is alive. Close a non-responsive connection
+	// after an additional duration of Timeout. 0 to disable.
+	GRPCKeepAliveInterval time.Duration `json:"grpc-keepalive-interval"`
+	// GRPCKeepAliveTimeout is the additional duration of wait
+	// before closing a non-responsive connection. 0 to disable.
+	GRPCKeepAliveTimeout time.Duration `json:"grpc-keepalive-timeout"`
+
+	// PreVote is true to enable Raft Pre-Vote.
+	// If enabled, Raft runs an additional election phase
+	// to check whether it would get enough votes to win
+	// an election, thus minimizing disruptions.
+	// TODO: enable by default in 3.5.
+	PreVote bool `json:"pre-vote"`
+
+	CORS map[string]struct{}
+
+	// HostWhitelist lists acceptable hostnames from HTTP client requests.
+	// Client origin policy protects against "DNS Rebinding" attacks
+	// to insecure etcd servers. That is, any website can simply create
+	// an authorized DNS name, and direct DNS to "localhost" (or any
+	// other address). Then, all HTTP endpoints of etcd server listening
+	// on "localhost" becomes accessible, thus vulnerable to DNS rebinding
+	// attacks. See "CVE-2018-5702" for more detail.
+	//
+	// 1. If client connection is secure via HTTPS, allow any hostnames.
+	// 2. If client connection is not secure and "HostWhitelist" is not empty,
+	//    only allow HTTP requests whose Host field is listed in whitelist.
+	//
+	// Note that the client origin policy is enforced whether authentication
+	// is enabled or not, for tighter controls.
+	//
+	// By default, "HostWhitelist" is "*", which allows any hostnames.
+	// Note that when specifying hostnames, loopback addresses are not added
+	// automatically. To allow loopback interfaces, leave it empty or set it "*",
+	// or add them to whitelist manually (e.g. "localhost", "127.0.0.1", etc.).
+	//
+	// CVE-2018-5702 reference:
+	// - https://bugs.chromium.org/p/project-zero/issues/detail?id=1447#c2
+	// - https://github.com/transmission/transmission/pull/468
+	// - https://github.com/etcd-io/etcd/issues/9353
+	HostWhitelist map[string]struct{}
+
+	// UserHandlers is for registering users handlers and only used for
+	// embedding etcd into other applications.
+	// The map key is the route path for the handler, and
+	// you must ensure it can't be conflicted with etcd's.
+	UserHandlers map[string]http.Handler `json:"-"`
+	// ServiceRegister is for registering users' gRPC services. A simple usage example:
+	//	cfg := embed.NewConfig()
+	//	cfg.ServerRegister = func(s *grpc.Server) {
+	//		pb.RegisterFooServer(s, &fooServer{})
+	//		pb.RegisterBarServer(s, &barServer{})
+	//	}
+	//	embed.StartEtcd(cfg)
+	ServiceRegister func(*grpc.Server) `json:"-"`
+
+	AuthToken  string `json:"auth-token"`
+	BcryptCost uint   `json:"bcrypt-cost"`
+
+	ExperimentalInitialCorruptCheck bool          `json:"experimental-initial-corrupt-check"`
+	ExperimentalCorruptCheckTime    time.Duration `json:"experimental-corrupt-check-time"`
+	ExperimentalEnableV2V3          string        `json:"experimental-enable-v2v3"`
+	// ExperimentalBackendFreelistType specifies the type of freelist that boltdb backend uses (array and map are supported types).
+	ExperimentalBackendFreelistType string `json:"experimental-backend-bbolt-freelist-type"`
+	// ExperimentalEnableLeaseCheckpoint enables primary lessor to persist lease remainingTTL to prevent indefinite auto-renewal of long lived leases.
+	ExperimentalEnableLeaseCheckpoint bool `json:"experimental-enable-lease-checkpoint"`
+	ExperimentalCompactionBatchLimit  int  `json:"experimental-compaction-batch-limit"`
+
+	// ForceNewCluster starts a new cluster even if previously started; unsafe.
+	ForceNewCluster bool `json:"force-new-cluster"`
+
+	EnablePprof           bool   `json:"enable-pprof"`
+	Metrics               string `json:"metrics"`
+	ListenMetricsUrls     []url.URL
+	ListenMetricsUrlsJSON string `json:"listen-metrics-urls"`
+
+	// Logger is logger options: "zap", "capnslog".
+	// WARN: "capnslog" is being deprecated in v3.5.
+	Logger string `json:"logger"`
+	// LogLevel configures log level. Only supports debug, info, warn, error, panic, or fatal. Default 'info'.
+	LogLevel string `json:"log-level"`
+	// LogOutputs is either:
+	//  - "default" as os.Stderr,
+	//  - "stderr" as os.Stderr,
+	//  - "stdout" as os.Stdout,
+	//  - file path to append server logs to.
+	// It can be multiple when "Logger" is zap.
+	LogOutputs []string `json:"log-outputs"`
+
+	// ZapLoggerBuilder is used to build the zap logger.
+	ZapLoggerBuilder func(*Config) error
+
+	// logger logs server-side operations. The default is nil,
+	// and "setupLogging" must be called before starting server.
+	// Do not set logger directly.
+	loggerMu *sync.RWMutex
+	logger   *zap.Logger
+
+	// loggerConfig is server logger configuration for Raft logger.
+	// Must be either: "loggerConfig != nil" or "loggerCore != nil && loggerWriteSyncer != nil".
+	loggerConfig *zap.Config
+	// loggerCore is "zapcore.Core" for raft logger.
+	// Must be either: "loggerConfig != nil" or "loggerCore != nil && loggerWriteSyncer != nil".
+	loggerCore        zapcore.Core
+	loggerWriteSyncer zapcore.WriteSyncer
+
+	// EnableGRPCGateway is false to disable grpc gateway.
+	EnableGRPCGateway bool `json:"enable-grpc-gateway"`
+
+	// TO BE DEPRECATED
+
+	// DeprecatedLogOutput is to be deprecated in v3.5.
+	// Just here for safe migration in v3.4.
+	DeprecatedLogOutput []string `json:"log-output"`
+	// Debug is true, to enable debug level logging.
+	// WARNING: to be deprecated in 3.5. Use "--log-level=debug" instead.
+	Debug bool `json:"debug"`
+	// LogPkgLevels is being deprecated in v3.5.
+	// Only valid if "logger" option is "capnslog".
+	// WARN: DO NOT USE THIS!
+	LogPkgLevels string `json:"log-package-levels"`
+}
+
+// configYAML holds the config suitable for yaml parsing
+type configYAML struct {
+	Config
+	configJSON
+}
+
+// configJSON has file options that are translated into Config options
+type configJSON struct {
+	LPUrlsJSON string `json:"listen-peer-urls"`
+	LCUrlsJSON string `json:"listen-client-urls"`
+	APUrlsJSON string `json:"initial-advertise-peer-urls"`
+	ACUrlsJSON string `json:"advertise-client-urls"`
+
+	CORSJSON          string `json:"cors"`
+	HostWhitelistJSON string `json:"host-whitelist"`
+
+	ClientSecurityJSON securityConfig `json:"client-transport-security"`
+	PeerSecurityJSON   securityConfig `json:"peer-transport-security"`
+}
+
+type securityConfig struct {
+	CertFile      string `json:"cert-file"`
+	KeyFile       string `json:"key-file"`
+	CertAuth      bool   `json:"client-cert-auth"`
+	TrustedCAFile string `json:"trusted-ca-file"`
+	AutoTLS       bool   `json:"auto-tls"`
+}
+
+// NewConfig creates a new Config populated with default values.
+func NewConfig() *Config {
+	lpurl, _ := url.Parse(DefaultListenPeerURLs)
+	apurl, _ := url.Parse(DefaultInitialAdvertisePeerURLs)
+	lcurl, _ := url.Parse(DefaultListenClientURLs)
+	acurl, _ := url.Parse(DefaultAdvertiseClientURLs)
+	cfg := &Config{
+		MaxSnapFiles: DefaultMaxSnapshots,
+		MaxWalFiles:  DefaultMaxWALs,
+
+		Name: DefaultName,
+
+		SnapshotCount:          etcdserver.DefaultSnapshotCount,
+		SnapshotCatchUpEntries: etcdserver.DefaultSnapshotCatchUpEntries,
+
+		MaxTxnOps:       DefaultMaxTxnOps,
+		MaxRequestBytes: DefaultMaxRequestBytes,
+
+		GRPCKeepAliveMinTime:  DefaultGRPCKeepAliveMinTime,
+		GRPCKeepAliveInterval: DefaultGRPCKeepAliveInterval,
+		GRPCKeepAliveTimeout:  DefaultGRPCKeepAliveTimeout,
+
+		TickMs:                     100,
+		ElectionMs:                 1000,
+		InitialElectionTickAdvance: true,
+
+		LPUrls: []url.URL{*lpurl},
+		LCUrls: []url.URL{*lcurl},
+		APUrls: []url.URL{*apurl},
+		ACUrls: []url.URL{*acurl},
+
+		ClusterState:        ClusterStateFlagNew,
+		InitialClusterToken: "etcd-cluster",
+
+		StrictReconfigCheck: DefaultStrictReconfigCheck,
+		Metrics:             "basic",
+		EnableV2:            DefaultEnableV2,
+
+		CORS:          map[string]struct{}{"*": {}},
+		HostWhitelist: map[string]struct{}{"*": {}},
+
+		AuthToken:  "simple",
+		BcryptCost: uint(bcrypt.DefaultCost),
+
+		PreVote: false, // TODO: enable by default in v3.5
+
+		loggerMu:            new(sync.RWMutex),
+		logger:              nil,
+		Logger:              "capnslog",
+		DeprecatedLogOutput: []string{DefaultLogOutput},
+		LogOutputs:          []string{DefaultLogOutput},
+		Debug:               false,
+		LogLevel:            logutil.DefaultLogLevel,
+		LogPkgLevels:        "",
+	}
+	cfg.InitialCluster = cfg.InitialClusterFromName(cfg.Name)
+	return cfg
+}
+
+func logTLSHandshakeFailure(conn *tls.Conn, err error) {
+	state := conn.ConnectionState()
+	remoteAddr := conn.RemoteAddr().String()
+	serverName := state.ServerName
+	if len(state.PeerCertificates) > 0 {
+		cert := state.PeerCertificates[0]
+		ips, dns := cert.IPAddresses, cert.DNSNames
+		plog.Infof("rejected connection from %q (error %q, ServerName %q, IPAddresses %q, DNSNames %q)", remoteAddr, err.Error(), serverName, ips, dns)
+	} else {
+		plog.Infof("rejected connection from %q (error %q, ServerName %q)", remoteAddr, err.Error(), serverName)
+	}
+}
+
+func ConfigFromFile(path string) (*Config, error) {
+	cfg := &configYAML{Config: *NewConfig()}
+	if err := cfg.configFromFile(path); err != nil {
+		return nil, err
+	}
+	return &cfg.Config, nil
+}
+
+func (cfg *configYAML) configFromFile(path string) error {
+	b, err := ioutil.ReadFile(path)
+	if err != nil {
+		return err
+	}
+
+	defaultInitialCluster := cfg.InitialCluster
+
+	err = yaml.Unmarshal(b, cfg)
+	if err != nil {
+		return err
+	}
+
+	if cfg.LPUrlsJSON != "" {
+		u, err := types.NewURLs(strings.Split(cfg.LPUrlsJSON, ","))
+		if err != nil {
+			fmt.Fprintf(os.Stderr, "unexpected error setting up listen-peer-urls: %v\n", err)
+			os.Exit(1)
+		}
+		cfg.LPUrls = []url.URL(u)
+	}
+
+	if cfg.LCUrlsJSON != "" {
+		u, err := types.NewURLs(strings.Split(cfg.LCUrlsJSON, ","))
+		if err != nil {
+			fmt.Fprintf(os.Stderr, "unexpected error setting up listen-client-urls: %v\n", err)
+			os.Exit(1)
+		}
+		cfg.LCUrls = []url.URL(u)
+	}
+
+	if cfg.APUrlsJSON != "" {
+		u, err := types.NewURLs(strings.Split(cfg.APUrlsJSON, ","))
+		if err != nil {
+			fmt.Fprintf(os.Stderr, "unexpected error setting up initial-advertise-peer-urls: %v\n", err)
+			os.Exit(1)
+		}
+		cfg.APUrls = []url.URL(u)
+	}
+
+	if cfg.ACUrlsJSON != "" {
+		u, err := types.NewURLs(strings.Split(cfg.ACUrlsJSON, ","))
+		if err != nil {
+			fmt.Fprintf(os.Stderr, "unexpected error setting up advertise-peer-urls: %v\n", err)
+			os.Exit(1)
+		}
+		cfg.ACUrls = []url.URL(u)
+	}
+
+	if cfg.ListenMetricsUrlsJSON != "" {
+		u, err := types.NewURLs(strings.Split(cfg.ListenMetricsUrlsJSON, ","))
+		if err != nil {
+			fmt.Fprintf(os.Stderr, "unexpected error setting up listen-metrics-urls: %v\n", err)
+			os.Exit(1)
+		}
+		cfg.ListenMetricsUrls = []url.URL(u)
+	}
+
+	if cfg.CORSJSON != "" {
+		uv := flags.NewUniqueURLsWithExceptions(cfg.CORSJSON, "*")
+		cfg.CORS = uv.Values
+	}
+
+	if cfg.HostWhitelistJSON != "" {
+		uv := flags.NewUniqueStringsValue(cfg.HostWhitelistJSON)
+		cfg.HostWhitelist = uv.Values
+	}
+
+	// If a discovery flag is set, clear default initial cluster set by InitialClusterFromName
+	if (cfg.Durl != "" || cfg.DNSCluster != "") && cfg.InitialCluster == defaultInitialCluster {
+		cfg.InitialCluster = ""
+	}
+	if cfg.ClusterState == "" {
+		cfg.ClusterState = ClusterStateFlagNew
+	}
+
+	copySecurityDetails := func(tls *transport.TLSInfo, ysc *securityConfig) {
+		tls.CertFile = ysc.CertFile
+		tls.KeyFile = ysc.KeyFile
+		tls.ClientCertAuth = ysc.CertAuth
+		tls.TrustedCAFile = ysc.TrustedCAFile
+	}
+	copySecurityDetails(&cfg.ClientTLSInfo, &cfg.ClientSecurityJSON)
+	copySecurityDetails(&cfg.PeerTLSInfo, &cfg.PeerSecurityJSON)
+	cfg.ClientAutoTLS = cfg.ClientSecurityJSON.AutoTLS
+	cfg.PeerAutoTLS = cfg.PeerSecurityJSON.AutoTLS
+
+	return cfg.Validate()
+}
+
+func updateCipherSuites(tls *transport.TLSInfo, ss []string) error {
+	if len(tls.CipherSuites) > 0 && len(ss) > 0 {
+		return fmt.Errorf("TLSInfo.CipherSuites is already specified (given %v)", ss)
+	}
+	if len(ss) > 0 {
+		cs := make([]uint16, len(ss))
+		for i, s := range ss {
+			var ok bool
+			cs[i], ok = tlsutil.GetCipherSuite(s)
+			if !ok {
+				return fmt.Errorf("unexpected TLS cipher suite %q", s)
+			}
+		}
+		tls.CipherSuites = cs
+	}
+	return nil
+}
+
+// Validate ensures that '*embed.Config' fields are properly configured.
+func (cfg *Config) Validate() error {
+	if err := cfg.setupLogging(); err != nil {
+		return err
+	}
+	if err := checkBindURLs(cfg.LPUrls); err != nil {
+		return err
+	}
+	if err := checkBindURLs(cfg.LCUrls); err != nil {
+		return err
+	}
+	if err := checkBindURLs(cfg.ListenMetricsUrls); err != nil {
+		return err
+	}
+	if err := checkHostURLs(cfg.APUrls); err != nil {
+		addrs := cfg.getAPURLs()
+		return fmt.Errorf(`--initial-advertise-peer-urls %q must be "host:port" (%v)`, strings.Join(addrs, ","), err)
+	}
+	if err := checkHostURLs(cfg.ACUrls); err != nil {
+		addrs := cfg.getACURLs()
+		return fmt.Errorf(`--advertise-client-urls %q must be "host:port" (%v)`, strings.Join(addrs, ","), err)
+	}
+	// Check if conflicting flags are passed.
+	nSet := 0
+	for _, v := range []bool{cfg.Durl != "", cfg.InitialCluster != "", cfg.DNSCluster != ""} {
+		if v {
+			nSet++
+		}
+	}
+
+	if cfg.ClusterState != ClusterStateFlagNew && cfg.ClusterState != ClusterStateFlagExisting {
+		return fmt.Errorf("unexpected clusterState %q", cfg.ClusterState)
+	}
+
+	if nSet > 1 {
+		return ErrConflictBootstrapFlags
+	}
+
+	if cfg.TickMs <= 0 {
+		return fmt.Errorf("--heartbeat-interval must be >0 (set to %dms)", cfg.TickMs)
+	}
+	if cfg.ElectionMs <= 0 {
+		return fmt.Errorf("--election-timeout must be >0 (set to %dms)", cfg.ElectionMs)
+	}
+	if 5*cfg.TickMs > cfg.ElectionMs {
+		return fmt.Errorf("--election-timeout[%vms] should be at least as 5 times as --heartbeat-interval[%vms]", cfg.ElectionMs, cfg.TickMs)
+	}
+	if cfg.ElectionMs > maxElectionMs {
+		return fmt.Errorf("--election-timeout[%vms] is too long, and should be set less than %vms", cfg.ElectionMs, maxElectionMs)
+	}
+
+	// check this last since proxying in etcdmain may make this OK
+	if cfg.LCUrls != nil && cfg.ACUrls == nil {
+		return ErrUnsetAdvertiseClientURLsFlag
+	}
+
+	switch cfg.AutoCompactionMode {
+	case "":
+	case CompactorModeRevision, CompactorModePeriodic:
+	default:
+		return fmt.Errorf("unknown auto-compaction-mode %q", cfg.AutoCompactionMode)
+	}
+
+	return nil
+}
+
+// PeerURLsMapAndToken sets up an initial peer URLsMap and cluster token for bootstrap or discovery.
+func (cfg *Config) PeerURLsMapAndToken(which string) (urlsmap types.URLsMap, token string, err error) {
+	token = cfg.InitialClusterToken
+	switch {
+	case cfg.Durl != "":
+		urlsmap = types.URLsMap{}
+		// If using discovery, generate a temporary cluster based on
+		// self's advertised peer URLs
+		urlsmap[cfg.Name] = cfg.APUrls
+		token = cfg.Durl
+
+	case cfg.DNSCluster != "":
+		clusterStrs, cerr := cfg.GetDNSClusterNames()
+		lg := cfg.logger
+		if cerr != nil {
+			if lg != nil {
+				lg.Warn("failed to resolve during SRV discovery", zap.Error(cerr))
+			} else {
+				plog.Errorf("couldn't resolve during SRV discovery (%v)", cerr)
+			}
+			return nil, "", cerr
+		}
+		for _, s := range clusterStrs {
+			if lg != nil {
+				lg.Info("got bootstrap from DNS for etcd-server", zap.String("node", s))
+			} else {
+				plog.Noticef("got bootstrap from DNS for etcd-server at %s", s)
+			}
+		}
+		clusterStr := strings.Join(clusterStrs, ",")
+		if strings.Contains(clusterStr, "https://") && cfg.PeerTLSInfo.TrustedCAFile == "" {
+			cfg.PeerTLSInfo.ServerName = cfg.DNSCluster
+		}
+		urlsmap, err = types.NewURLsMap(clusterStr)
+		// only etcd member must belong to the discovered cluster.
+		// proxy does not need to belong to the discovered cluster.
+		if which == "etcd" {
+			if _, ok := urlsmap[cfg.Name]; !ok {
+				return nil, "", fmt.Errorf("cannot find local etcd member %q in SRV records", cfg.Name)
+			}
+		}
+
+	default:
+		// We're statically configured, and cluster has appropriately been set.
+		urlsmap, err = types.NewURLsMap(cfg.InitialCluster)
+	}
+	return urlsmap, token, err
+}
+
+// GetDNSClusterNames uses DNS SRV records to get a list of initial nodes for cluster bootstrapping.
+func (cfg *Config) GetDNSClusterNames() ([]string, error) {
+	var (
+		clusterStrs       []string
+		cerr              error
+		serviceNameSuffix string
+	)
+	if cfg.DNSClusterServiceName != "" {
+		serviceNameSuffix = "-" + cfg.DNSClusterServiceName
+	}
+
+	lg := cfg.GetLogger()
+
+	// Use both etcd-server-ssl and etcd-server for discovery.
+	// Combine the results if both are available.
+	clusterStrs, cerr = srv.GetCluster("https", "etcd-server-ssl"+serviceNameSuffix, cfg.Name, cfg.DNSCluster, cfg.APUrls)
+	if cerr != nil {
+		clusterStrs = make([]string, 0)
+	}
+	if lg != nil {
+		lg.Info(
+			"get cluster for etcd-server-ssl SRV",
+			zap.String("service-scheme", "https"),
+			zap.String("service-name", "etcd-server-ssl"+serviceNameSuffix),
+			zap.String("server-name", cfg.Name),
+			zap.String("discovery-srv", cfg.DNSCluster),
+			zap.Strings("advertise-peer-urls", cfg.getAPURLs()),
+			zap.Strings("found-cluster", clusterStrs),
+			zap.Error(cerr),
+		)
+	}
+
+	defaultHTTPClusterStrs, httpCerr := srv.GetCluster("http", "etcd-server"+serviceNameSuffix, cfg.Name, cfg.DNSCluster, cfg.APUrls)
+	if httpCerr != nil {
+		clusterStrs = append(clusterStrs, defaultHTTPClusterStrs...)
+	}
+	if lg != nil {
+		lg.Info(
+			"get cluster for etcd-server SRV",
+			zap.String("service-scheme", "http"),
+			zap.String("service-name", "etcd-server"+serviceNameSuffix),
+			zap.String("server-name", cfg.Name),
+			zap.String("discovery-srv", cfg.DNSCluster),
+			zap.Strings("advertise-peer-urls", cfg.getAPURLs()),
+			zap.Strings("found-cluster", clusterStrs),
+			zap.Error(httpCerr),
+		)
+	}
+
+	return clusterStrs, cerr
+}
+
+func (cfg Config) InitialClusterFromName(name string) (ret string) {
+	if len(cfg.APUrls) == 0 {
+		return ""
+	}
+	n := name
+	if name == "" {
+		n = DefaultName
+	}
+	for i := range cfg.APUrls {
+		ret = ret + "," + n + "=" + cfg.APUrls[i].String()
+	}
+	return ret[1:]
+}
+
+func (cfg Config) IsNewCluster() bool { return cfg.ClusterState == ClusterStateFlagNew }
+func (cfg Config) ElectionTicks() int { return int(cfg.ElectionMs / cfg.TickMs) }
+
+func (cfg Config) defaultPeerHost() bool {
+	return len(cfg.APUrls) == 1 && cfg.APUrls[0].String() == DefaultInitialAdvertisePeerURLs
+}
+
+func (cfg Config) defaultClientHost() bool {
+	return len(cfg.ACUrls) == 1 && cfg.ACUrls[0].String() == DefaultAdvertiseClientURLs
+}
+
+func (cfg *Config) ClientSelfCert() (err error) {
+	if !cfg.ClientAutoTLS {
+		return nil
+	}
+	if !cfg.ClientTLSInfo.Empty() {
+		if cfg.logger != nil {
+			cfg.logger.Warn("ignoring client auto TLS since certs given")
+		} else {
+			plog.Warningf("ignoring client auto TLS since certs given")
+		}
+		return nil
+	}
+	chosts := make([]string, len(cfg.LCUrls))
+	for i, u := range cfg.LCUrls {
+		chosts[i] = u.Host
+	}
+	cfg.ClientTLSInfo, err = transport.SelfCert(cfg.logger, filepath.Join(cfg.Dir, "fixtures", "client"), chosts)
+	if err != nil {
+		return err
+	}
+	return updateCipherSuites(&cfg.ClientTLSInfo, cfg.CipherSuites)
+}
+
+func (cfg *Config) PeerSelfCert() (err error) {
+	if !cfg.PeerAutoTLS {
+		return nil
+	}
+	if !cfg.PeerTLSInfo.Empty() {
+		if cfg.logger != nil {
+			cfg.logger.Warn("ignoring peer auto TLS since certs given")
+		} else {
+			plog.Warningf("ignoring peer auto TLS since certs given")
+		}
+		return nil
+	}
+	phosts := make([]string, len(cfg.LPUrls))
+	for i, u := range cfg.LPUrls {
+		phosts[i] = u.Host
+	}
+	cfg.PeerTLSInfo, err = transport.SelfCert(cfg.logger, filepath.Join(cfg.Dir, "fixtures", "peer"), phosts)
+	if err != nil {
+		return err
+	}
+	return updateCipherSuites(&cfg.PeerTLSInfo, cfg.CipherSuites)
+}
+
+// UpdateDefaultClusterFromName updates cluster advertise URLs with, if available, default host,
+// if advertise URLs are default values(localhost:2379,2380) AND if listen URL is 0.0.0.0.
+// e.g. advertise peer URL localhost:2380 or listen peer URL 0.0.0.0:2380
+// then the advertise peer host would be updated with machine's default host,
+// while keeping the listen URL's port.
+// User can work around this by explicitly setting URL with 127.0.0.1.
+// It returns the default hostname, if used, and the error, if any, from getting the machine's default host.
+// TODO: check whether fields are set instead of whether fields have default value
+func (cfg *Config) UpdateDefaultClusterFromName(defaultInitialCluster string) (string, error) {
+	if defaultHostname == "" || defaultHostStatus != nil {
+		// update 'initial-cluster' when only the name is specified (e.g. 'etcd --name=abc')
+		if cfg.Name != DefaultName && cfg.InitialCluster == defaultInitialCluster {
+			cfg.InitialCluster = cfg.InitialClusterFromName(cfg.Name)
+		}
+		return "", defaultHostStatus
+	}
+
+	used := false
+	pip, pport := cfg.LPUrls[0].Hostname(), cfg.LPUrls[0].Port()
+	if cfg.defaultPeerHost() && pip == "0.0.0.0" {
+		cfg.APUrls[0] = url.URL{Scheme: cfg.APUrls[0].Scheme, Host: fmt.Sprintf("%s:%s", defaultHostname, pport)}
+		used = true
+	}
+	// update 'initial-cluster' when only the name is specified (e.g. 'etcd --name=abc')
+	if cfg.Name != DefaultName && cfg.InitialCluster == defaultInitialCluster {
+		cfg.InitialCluster = cfg.InitialClusterFromName(cfg.Name)
+	}
+
+	cip, cport := cfg.LCUrls[0].Hostname(), cfg.LCUrls[0].Port()
+	if cfg.defaultClientHost() && cip == "0.0.0.0" {
+		cfg.ACUrls[0] = url.URL{Scheme: cfg.ACUrls[0].Scheme, Host: fmt.Sprintf("%s:%s", defaultHostname, cport)}
+		used = true
+	}
+	dhost := defaultHostname
+	if !used {
+		dhost = ""
+	}
+	return dhost, defaultHostStatus
+}
+
+// checkBindURLs returns an error if any URL uses a domain name.
+func checkBindURLs(urls []url.URL) error {
+	for _, url := range urls {
+		if url.Scheme == "unix" || url.Scheme == "unixs" {
+			continue
+		}
+		host, _, err := net.SplitHostPort(url.Host)
+		if err != nil {
+			return err
+		}
+		if host == "localhost" {
+			// special case for local address
+			// TODO: support /etc/hosts ?
+			continue
+		}
+		if net.ParseIP(host) == nil {
+			return fmt.Errorf("expected IP in URL for binding (%s)", url.String())
+		}
+	}
+	return nil
+}
+
+func checkHostURLs(urls []url.URL) error {
+	for _, url := range urls {
+		host, _, err := net.SplitHostPort(url.Host)
+		if err != nil {
+			return err
+		}
+		if host == "" {
+			return fmt.Errorf("unexpected empty host (%s)", url.String())
+		}
+	}
+	return nil
+}
+
+func (cfg *Config) getAPURLs() (ss []string) {
+	ss = make([]string, len(cfg.APUrls))
+	for i := range cfg.APUrls {
+		ss[i] = cfg.APUrls[i].String()
+	}
+	return ss
+}
+
+func (cfg *Config) getLPURLs() (ss []string) {
+	ss = make([]string, len(cfg.LPUrls))
+	for i := range cfg.LPUrls {
+		ss[i] = cfg.LPUrls[i].String()
+	}
+	return ss
+}
+
+func (cfg *Config) getACURLs() (ss []string) {
+	ss = make([]string, len(cfg.ACUrls))
+	for i := range cfg.ACUrls {
+		ss[i] = cfg.ACUrls[i].String()
+	}
+	return ss
+}
+
+func (cfg *Config) getLCURLs() (ss []string) {
+	ss = make([]string, len(cfg.LCUrls))
+	for i := range cfg.LCUrls {
+		ss[i] = cfg.LCUrls[i].String()
+	}
+	return ss
+}
+
+func (cfg *Config) getMetricsURLs() (ss []string) {
+	ss = make([]string, len(cfg.ListenMetricsUrls))
+	for i := range cfg.ListenMetricsUrls {
+		ss[i] = cfg.ListenMetricsUrls[i].String()
+	}
+	return ss
+}
+
+func parseBackendFreelistType(freelistType string) bolt.FreelistType {
+	if freelistType == freelistMapType {
+		return bolt.FreelistMapType
+	}
+
+	return bolt.FreelistArrayType
+}
diff --git a/vendor/go.etcd.io/etcd/embed/config_logging.go b/vendor/go.etcd.io/etcd/embed/config_logging.go
new file mode 100644
index 0000000..e42103c
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/embed/config_logging.go
@@ -0,0 +1,312 @@
+// Copyright 2018 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 embed
+
+import (
+	"crypto/tls"
+	"errors"
+	"fmt"
+	"io/ioutil"
+	"os"
+	"reflect"
+	"sync"
+
+	"go.etcd.io/etcd/pkg/logutil"
+
+	"github.com/coreos/pkg/capnslog"
+	"go.uber.org/zap"
+	"go.uber.org/zap/zapcore"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/grpclog"
+)
+
+// GetLogger returns the logger.
+func (cfg Config) GetLogger() *zap.Logger {
+	cfg.loggerMu.RLock()
+	l := cfg.logger
+	cfg.loggerMu.RUnlock()
+	return l
+}
+
+// for testing
+var grpcLogOnce = new(sync.Once)
+
+// setupLogging initializes etcd logging.
+// Must be called after flag parsing or finishing configuring embed.Config.
+func (cfg *Config) setupLogging() error {
+	// handle "DeprecatedLogOutput" in v3.4
+	// TODO: remove "DeprecatedLogOutput" in v3.5
+	len1 := len(cfg.DeprecatedLogOutput)
+	len2 := len(cfg.LogOutputs)
+	if len1 != len2 {
+		switch {
+		case len1 > len2: // deprecate "log-output" flag is used
+			fmt.Fprintln(os.Stderr, "'--log-output' flag has been deprecated! Please use '--log-outputs'!")
+			cfg.LogOutputs = cfg.DeprecatedLogOutput
+		case len1 < len2: // "--log-outputs" flag has been set with multiple writers
+			cfg.DeprecatedLogOutput = []string{}
+		}
+	} else {
+		if len1 > 1 {
+			return errors.New("both '--log-output' and '--log-outputs' are set; only set '--log-outputs'")
+		}
+		if len1 < 1 {
+			return errors.New("either '--log-output' or '--log-outputs' flag must be set")
+		}
+		if reflect.DeepEqual(cfg.DeprecatedLogOutput, cfg.LogOutputs) && cfg.DeprecatedLogOutput[0] != DefaultLogOutput {
+			return fmt.Errorf("'--log-output=%q' and '--log-outputs=%q' are incompatible; only set --log-outputs", cfg.DeprecatedLogOutput, cfg.LogOutputs)
+		}
+		if !reflect.DeepEqual(cfg.DeprecatedLogOutput, []string{DefaultLogOutput}) {
+			fmt.Fprintf(os.Stderr, "[WARNING] Deprecated '--log-output' flag is set to %q\n", cfg.DeprecatedLogOutput)
+			fmt.Fprintln(os.Stderr, "Please use '--log-outputs' flag")
+		}
+	}
+
+	// TODO: remove after deprecating log related flags in v3.5
+	if cfg.Debug {
+		fmt.Fprintf(os.Stderr, "[WARNING] Deprecated '--debug' flag is set to %v (use '--log-level=debug' instead\n", cfg.Debug)
+	}
+	if cfg.Debug && cfg.LogLevel != "debug" {
+		fmt.Fprintf(os.Stderr, "[WARNING] Deprecated '--debug' flag is set to %v with inconsistent '--log-level=%s' flag\n", cfg.Debug, cfg.LogLevel)
+	}
+	if cfg.Logger == "capnslog" {
+		fmt.Fprintf(os.Stderr, "[WARNING] Deprecated '--logger=%s' flag is set; use '--logger=zap' flag instead\n", cfg.Logger)
+	}
+	if cfg.LogPkgLevels != "" {
+		fmt.Fprintf(os.Stderr, "[WARNING] Deprecated '--log-package-levels=%s' flag is set; use '--logger=zap' flag instead\n", cfg.LogPkgLevels)
+	}
+
+	switch cfg.Logger {
+	case "capnslog": // TODO: deprecate this in v3.5
+		cfg.ClientTLSInfo.HandshakeFailure = logTLSHandshakeFailure
+		cfg.PeerTLSInfo.HandshakeFailure = logTLSHandshakeFailure
+
+		if cfg.Debug {
+			capnslog.SetGlobalLogLevel(capnslog.DEBUG)
+			grpc.EnableTracing = true
+			// enable info, warning, error
+			grpclog.SetLoggerV2(grpclog.NewLoggerV2(os.Stderr, os.Stderr, os.Stderr))
+		} else {
+			capnslog.SetGlobalLogLevel(logutil.ConvertToCapnslogLogLevel(cfg.LogLevel))
+			// only discard info
+			grpclog.SetLoggerV2(grpclog.NewLoggerV2(ioutil.Discard, os.Stderr, os.Stderr))
+		}
+
+		// TODO: deprecate with "capnslog"
+		if cfg.LogPkgLevels != "" {
+			repoLog := capnslog.MustRepoLogger("go.etcd.io/etcd")
+			settings, err := repoLog.ParseLogLevelConfig(cfg.LogPkgLevels)
+			if err != nil {
+				plog.Warningf("couldn't parse log level string: %s, continuing with default levels", err.Error())
+				return nil
+			}
+			repoLog.SetLogLevel(settings)
+		}
+
+		if len(cfg.LogOutputs) != 1 {
+			return fmt.Errorf("--logger=capnslog supports only 1 value in '--log-outputs', got %q", cfg.LogOutputs)
+		}
+		// capnslog initially SetFormatter(NewDefaultFormatter(os.Stderr))
+		// where NewDefaultFormatter returns NewJournaldFormatter when syscall.Getppid() == 1
+		// specify 'stdout' or 'stderr' to skip journald logging even when running under systemd
+		output := cfg.LogOutputs[0]
+		switch output {
+		case StdErrLogOutput:
+			capnslog.SetFormatter(capnslog.NewPrettyFormatter(os.Stderr, cfg.Debug))
+		case StdOutLogOutput:
+			capnslog.SetFormatter(capnslog.NewPrettyFormatter(os.Stdout, cfg.Debug))
+		case DefaultLogOutput:
+		default:
+			return fmt.Errorf("unknown log-output %q (only supports %q, %q, %q)", output, DefaultLogOutput, StdErrLogOutput, StdOutLogOutput)
+		}
+
+	case "zap":
+		if len(cfg.LogOutputs) == 0 {
+			cfg.LogOutputs = []string{DefaultLogOutput}
+		}
+		if len(cfg.LogOutputs) > 1 {
+			for _, v := range cfg.LogOutputs {
+				if v == DefaultLogOutput {
+					return fmt.Errorf("multi logoutput for %q is not supported yet", DefaultLogOutput)
+				}
+			}
+		}
+
+		outputPaths, errOutputPaths := make([]string, 0), make([]string, 0)
+		isJournal := false
+		for _, v := range cfg.LogOutputs {
+			switch v {
+			case DefaultLogOutput:
+				outputPaths = append(outputPaths, StdErrLogOutput)
+				errOutputPaths = append(errOutputPaths, StdErrLogOutput)
+
+			case JournalLogOutput:
+				isJournal = true
+
+			case StdErrLogOutput:
+				outputPaths = append(outputPaths, StdErrLogOutput)
+				errOutputPaths = append(errOutputPaths, StdErrLogOutput)
+
+			case StdOutLogOutput:
+				outputPaths = append(outputPaths, StdOutLogOutput)
+				errOutputPaths = append(errOutputPaths, StdOutLogOutput)
+
+			default:
+				outputPaths = append(outputPaths, v)
+				errOutputPaths = append(errOutputPaths, v)
+			}
+		}
+
+		if !isJournal {
+			copied := logutil.DefaultZapLoggerConfig
+			copied.OutputPaths = outputPaths
+			copied.ErrorOutputPaths = errOutputPaths
+			copied = logutil.MergeOutputPaths(copied)
+			copied.Level = zap.NewAtomicLevelAt(logutil.ConvertToZapLevel(cfg.LogLevel))
+			if cfg.Debug || cfg.LogLevel == "debug" {
+				// enable tracing even when "--debug --log-level info"
+				// in order to keep backward compatibility with <= v3.3
+				// TODO: remove "Debug" check in v3.5
+				grpc.EnableTracing = true
+			}
+			if cfg.ZapLoggerBuilder == nil {
+				cfg.ZapLoggerBuilder = func(c *Config) error {
+					var err error
+					c.logger, err = copied.Build()
+					if err != nil {
+						return err
+					}
+					c.loggerMu.Lock()
+					defer c.loggerMu.Unlock()
+					c.loggerConfig = &copied
+					c.loggerCore = nil
+					c.loggerWriteSyncer = nil
+					grpcLogOnce.Do(func() {
+						// debug true, enable info, warning, error
+						// debug false, only discard info
+						var gl grpclog.LoggerV2
+						gl, err = logutil.NewGRPCLoggerV2(copied)
+						if err == nil {
+							grpclog.SetLoggerV2(gl)
+						}
+					})
+					return nil
+				}
+			}
+		} else {
+			if len(cfg.LogOutputs) > 1 {
+				for _, v := range cfg.LogOutputs {
+					if v != DefaultLogOutput {
+						return fmt.Errorf("running with systemd/journal but other '--log-outputs' values (%q) are configured with 'default'; override 'default' value with something else", cfg.LogOutputs)
+					}
+				}
+			}
+
+			// use stderr as fallback
+			syncer, lerr := getJournalWriteSyncer()
+			if lerr != nil {
+				return lerr
+			}
+
+			lvl := zap.NewAtomicLevelAt(logutil.ConvertToZapLevel(cfg.LogLevel))
+			if cfg.Debug || cfg.LogLevel == "debug" {
+				// enable tracing even when "--debug --log-level info"
+				// in order to keep backward compatibility with <= v3.3
+				// TODO: remove "Debug" check in v3.5
+				grpc.EnableTracing = true
+			}
+
+			// WARN: do not change field names in encoder config
+			// journald logging writer assumes field names of "level" and "caller"
+			cr := zapcore.NewCore(
+				zapcore.NewJSONEncoder(logutil.DefaultZapLoggerConfig.EncoderConfig),
+				syncer,
+				lvl,
+			)
+			if cfg.ZapLoggerBuilder == nil {
+				cfg.ZapLoggerBuilder = func(c *Config) error {
+					c.logger = zap.New(cr, zap.AddCaller(), zap.ErrorOutput(syncer))
+					c.loggerMu.Lock()
+					defer c.loggerMu.Unlock()
+					c.loggerConfig = nil
+					c.loggerCore = cr
+					c.loggerWriteSyncer = syncer
+
+					grpcLogOnce.Do(func() {
+						grpclog.SetLoggerV2(logutil.NewGRPCLoggerV2FromZapCore(cr, syncer))
+					})
+					return nil
+				}
+			}
+		}
+
+		err := cfg.ZapLoggerBuilder(cfg)
+		if err != nil {
+			return err
+		}
+
+		logTLSHandshakeFailure := func(conn *tls.Conn, err error) {
+			state := conn.ConnectionState()
+			remoteAddr := conn.RemoteAddr().String()
+			serverName := state.ServerName
+			if len(state.PeerCertificates) > 0 {
+				cert := state.PeerCertificates[0]
+				ips := make([]string, len(cert.IPAddresses))
+				for i := range cert.IPAddresses {
+					ips[i] = cert.IPAddresses[i].String()
+				}
+				cfg.logger.Warn(
+					"rejected connection",
+					zap.String("remote-addr", remoteAddr),
+					zap.String("server-name", serverName),
+					zap.Strings("ip-addresses", ips),
+					zap.Strings("dns-names", cert.DNSNames),
+					zap.Error(err),
+				)
+			} else {
+				cfg.logger.Warn(
+					"rejected connection",
+					zap.String("remote-addr", remoteAddr),
+					zap.String("server-name", serverName),
+					zap.Error(err),
+				)
+			}
+		}
+		cfg.ClientTLSInfo.HandshakeFailure = logTLSHandshakeFailure
+		cfg.PeerTLSInfo.HandshakeFailure = logTLSHandshakeFailure
+
+	default:
+		return fmt.Errorf("unknown logger option %q", cfg.Logger)
+	}
+
+	return nil
+}
+
+// NewZapCoreLoggerBuilder generates a zap core logger builder.
+func NewZapCoreLoggerBuilder(lg *zap.Logger, cr zapcore.Core, syncer zapcore.WriteSyncer) func(*Config) error {
+	return func(cfg *Config) error {
+		cfg.loggerMu.Lock()
+		defer cfg.loggerMu.Unlock()
+		cfg.logger = lg
+		cfg.loggerConfig = nil
+		cfg.loggerCore = cr
+		cfg.loggerWriteSyncer = syncer
+
+		grpcLogOnce.Do(func() {
+			grpclog.SetLoggerV2(logutil.NewGRPCLoggerV2FromZapCore(cr, syncer))
+		})
+		return nil
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/embed/config_logging_journal_unix.go b/vendor/go.etcd.io/etcd/embed/config_logging_journal_unix.go
new file mode 100644
index 0000000..44a51d6
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/embed/config_logging_journal_unix.go
@@ -0,0 +1,35 @@
+// Copyright 2018 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.
+
+// +build !windows
+
+package embed
+
+import (
+	"fmt"
+	"os"
+
+	"go.etcd.io/etcd/pkg/logutil"
+
+	"go.uber.org/zap/zapcore"
+)
+
+// use stderr as fallback
+func getJournalWriteSyncer() (zapcore.WriteSyncer, error) {
+	jw, err := logutil.NewJournalWriter(os.Stderr)
+	if err != nil {
+		return nil, fmt.Errorf("can't find journal (%v)", err)
+	}
+	return zapcore.AddSync(jw), nil
+}
diff --git a/vendor/go.etcd.io/etcd/embed/config_logging_journal_windows.go b/vendor/go.etcd.io/etcd/embed/config_logging_journal_windows.go
new file mode 100644
index 0000000..5b76256
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/embed/config_logging_journal_windows.go
@@ -0,0 +1,27 @@
+// Copyright 2018 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.
+
+// +build windows
+
+package embed
+
+import (
+	"os"
+
+	"go.uber.org/zap/zapcore"
+)
+
+func getJournalWriteSyncer() (zapcore.WriteSyncer, error) {
+	return zapcore.AddSync(os.Stderr), nil
+}
diff --git a/vendor/go.etcd.io/etcd/embed/doc.go b/vendor/go.etcd.io/etcd/embed/doc.go
new file mode 100644
index 0000000..4811bb6
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/embed/doc.go
@@ -0,0 +1,45 @@
+// Copyright 2016 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 embed provides bindings for embedding an etcd server in a program.
+
+Launch an embedded etcd server using the configuration defaults:
+
+	import (
+		"log"
+		"time"
+
+		"go.etcd.io/etcd/embed"
+	)
+
+	func main() {
+		cfg := embed.NewConfig()
+		cfg.Dir = "default.etcd"
+		e, err := embed.StartEtcd(cfg)
+		if err != nil {
+			log.Fatal(err)
+		}
+		defer e.Close()
+		select {
+		case <-e.Server.ReadyNotify():
+			log.Printf("Server is ready!")
+		case <-time.After(60 * time.Second):
+			e.Server.Stop() // trigger a shutdown
+			log.Printf("Server took too long to start!")
+		}
+		log.Fatal(<-e.Err())
+	}
+*/
+package embed
diff --git a/vendor/go.etcd.io/etcd/embed/etcd.go b/vendor/go.etcd.io/etcd/embed/etcd.go
new file mode 100644
index 0000000..ac7dbc9
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/embed/etcd.go
@@ -0,0 +1,829 @@
+// Copyright 2016 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 embed
+
+import (
+	"context"
+	"crypto/tls"
+	"fmt"
+	"io/ioutil"
+	defaultLog "log"
+	"net"
+	"net/http"
+	"net/url"
+	"runtime"
+	"sort"
+	"strconv"
+	"sync"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/api/etcdhttp"
+	"go.etcd.io/etcd/etcdserver/api/rafthttp"
+	"go.etcd.io/etcd/etcdserver/api/v2http"
+	"go.etcd.io/etcd/etcdserver/api/v2v3"
+	"go.etcd.io/etcd/etcdserver/api/v3client"
+	"go.etcd.io/etcd/etcdserver/api/v3rpc"
+	"go.etcd.io/etcd/pkg/debugutil"
+	runtimeutil "go.etcd.io/etcd/pkg/runtime"
+	"go.etcd.io/etcd/pkg/transport"
+	"go.etcd.io/etcd/pkg/types"
+	"go.etcd.io/etcd/version"
+
+	"github.com/coreos/pkg/capnslog"
+	grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus"
+	"github.com/soheilhy/cmux"
+	"go.uber.org/zap"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/keepalive"
+)
+
+var plog = capnslog.NewPackageLogger("go.etcd.io/etcd", "embed")
+
+const (
+	// internal fd usage includes disk usage and transport usage.
+	// To read/write snapshot, snap pkg needs 1. In normal case, wal pkg needs
+	// at most 2 to read/lock/write WALs. One case that it needs to 2 is to
+	// read all logs after some snapshot index, which locates at the end of
+	// the second last and the head of the last. For purging, it needs to read
+	// directory, so it needs 1. For fd monitor, it needs 1.
+	// For transport, rafthttp builds two long-polling connections and at most
+	// four temporary connections with each member. There are at most 9 members
+	// in a cluster, so it should reserve 96.
+	// For the safety, we set the total reserved number to 150.
+	reservedInternalFDNum = 150
+)
+
+// Etcd contains a running etcd server and its listeners.
+type Etcd struct {
+	Peers   []*peerListener
+	Clients []net.Listener
+	// a map of contexts for the servers that serves client requests.
+	sctxs            map[string]*serveCtx
+	metricsListeners []net.Listener
+
+	Server *etcdserver.EtcdServer
+
+	cfg   Config
+	stopc chan struct{}
+	errc  chan error
+
+	closeOnce sync.Once
+}
+
+type peerListener struct {
+	net.Listener
+	serve func() error
+	close func(context.Context) error
+}
+
+// StartEtcd launches the etcd server and HTTP handlers for client/server communication.
+// The returned Etcd.Server is not guaranteed to have joined the cluster. Wait
+// on the Etcd.Server.ReadyNotify() channel to know when it completes and is ready for use.
+func StartEtcd(inCfg *Config) (e *Etcd, err error) {
+	if err = inCfg.Validate(); err != nil {
+		return nil, err
+	}
+	serving := false
+	e = &Etcd{cfg: *inCfg, stopc: make(chan struct{})}
+	cfg := &e.cfg
+	defer func() {
+		if e == nil || err == nil {
+			return
+		}
+		if !serving {
+			// errored before starting gRPC server for serveCtx.serversC
+			for _, sctx := range e.sctxs {
+				close(sctx.serversC)
+			}
+		}
+		e.Close()
+		e = nil
+	}()
+
+	if e.cfg.logger != nil {
+		e.cfg.logger.Info(
+			"configuring peer listeners",
+			zap.Strings("listen-peer-urls", e.cfg.getLPURLs()),
+		)
+	}
+	if e.Peers, err = configurePeerListeners(cfg); err != nil {
+		return e, err
+	}
+
+	if e.cfg.logger != nil {
+		e.cfg.logger.Info(
+			"configuring client listeners",
+			zap.Strings("listen-client-urls", e.cfg.getLCURLs()),
+		)
+	}
+	if e.sctxs, err = configureClientListeners(cfg); err != nil {
+		return e, err
+	}
+
+	for _, sctx := range e.sctxs {
+		e.Clients = append(e.Clients, sctx.l)
+	}
+
+	var (
+		urlsmap types.URLsMap
+		token   string
+	)
+	memberInitialized := true
+	if !isMemberInitialized(cfg) {
+		memberInitialized = false
+		urlsmap, token, err = cfg.PeerURLsMapAndToken("etcd")
+		if err != nil {
+			return e, fmt.Errorf("error setting up initial cluster: %v", err)
+		}
+	}
+
+	// AutoCompactionRetention defaults to "0" if not set.
+	if len(cfg.AutoCompactionRetention) == 0 {
+		cfg.AutoCompactionRetention = "0"
+	}
+	autoCompactionRetention, err := parseCompactionRetention(cfg.AutoCompactionMode, cfg.AutoCompactionRetention)
+	if err != nil {
+		return e, err
+	}
+
+	backendFreelistType := parseBackendFreelistType(cfg.ExperimentalBackendFreelistType)
+
+	srvcfg := etcdserver.ServerConfig{
+		Name:                       cfg.Name,
+		ClientURLs:                 cfg.ACUrls,
+		PeerURLs:                   cfg.APUrls,
+		DataDir:                    cfg.Dir,
+		DedicatedWALDir:            cfg.WalDir,
+		SnapshotCount:              cfg.SnapshotCount,
+		SnapshotCatchUpEntries:     cfg.SnapshotCatchUpEntries,
+		MaxSnapFiles:               cfg.MaxSnapFiles,
+		MaxWALFiles:                cfg.MaxWalFiles,
+		InitialPeerURLsMap:         urlsmap,
+		InitialClusterToken:        token,
+		DiscoveryURL:               cfg.Durl,
+		DiscoveryProxy:             cfg.Dproxy,
+		NewCluster:                 cfg.IsNewCluster(),
+		PeerTLSInfo:                cfg.PeerTLSInfo,
+		TickMs:                     cfg.TickMs,
+		ElectionTicks:              cfg.ElectionTicks(),
+		InitialElectionTickAdvance: cfg.InitialElectionTickAdvance,
+		AutoCompactionRetention:    autoCompactionRetention,
+		AutoCompactionMode:         cfg.AutoCompactionMode,
+		QuotaBackendBytes:          cfg.QuotaBackendBytes,
+		BackendBatchLimit:          cfg.BackendBatchLimit,
+		BackendFreelistType:        backendFreelistType,
+		BackendBatchInterval:       cfg.BackendBatchInterval,
+		MaxTxnOps:                  cfg.MaxTxnOps,
+		MaxRequestBytes:            cfg.MaxRequestBytes,
+		StrictReconfigCheck:        cfg.StrictReconfigCheck,
+		ClientCertAuthEnabled:      cfg.ClientTLSInfo.ClientCertAuth,
+		AuthToken:                  cfg.AuthToken,
+		BcryptCost:                 cfg.BcryptCost,
+		CORS:                       cfg.CORS,
+		HostWhitelist:              cfg.HostWhitelist,
+		InitialCorruptCheck:        cfg.ExperimentalInitialCorruptCheck,
+		CorruptCheckTime:           cfg.ExperimentalCorruptCheckTime,
+		PreVote:                    cfg.PreVote,
+		Logger:                     cfg.logger,
+		LoggerConfig:               cfg.loggerConfig,
+		LoggerCore:                 cfg.loggerCore,
+		LoggerWriteSyncer:          cfg.loggerWriteSyncer,
+		Debug:                      cfg.Debug,
+		ForceNewCluster:            cfg.ForceNewCluster,
+		EnableGRPCGateway:          cfg.EnableGRPCGateway,
+		EnableLeaseCheckpoint:      cfg.ExperimentalEnableLeaseCheckpoint,
+		CompactionBatchLimit:       cfg.ExperimentalCompactionBatchLimit,
+	}
+	print(e.cfg.logger, *cfg, srvcfg, memberInitialized)
+	if e.Server, err = etcdserver.NewServer(srvcfg); err != nil {
+		return e, err
+	}
+
+	// buffer channel so goroutines on closed connections won't wait forever
+	e.errc = make(chan error, len(e.Peers)+len(e.Clients)+2*len(e.sctxs))
+
+	// newly started member ("memberInitialized==false")
+	// does not need corruption check
+	if memberInitialized {
+		if err = e.Server.CheckInitialHashKV(); err != nil {
+			// set "EtcdServer" to nil, so that it does not block on "EtcdServer.Close()"
+			// (nothing to close since rafthttp transports have not been started)
+			e.Server = nil
+			return e, err
+		}
+	}
+	e.Server.Start()
+
+	if err = e.servePeers(); err != nil {
+		return e, err
+	}
+	if err = e.serveClients(); err != nil {
+		return e, err
+	}
+	if err = e.serveMetrics(); err != nil {
+		return e, err
+	}
+
+	if e.cfg.logger != nil {
+		e.cfg.logger.Info(
+			"now serving peer/client/metrics",
+			zap.String("local-member-id", e.Server.ID().String()),
+			zap.Strings("initial-advertise-peer-urls", e.cfg.getAPURLs()),
+			zap.Strings("listen-peer-urls", e.cfg.getLPURLs()),
+			zap.Strings("advertise-client-urls", e.cfg.getACURLs()),
+			zap.Strings("listen-client-urls", e.cfg.getLCURLs()),
+			zap.Strings("listen-metrics-urls", e.cfg.getMetricsURLs()),
+		)
+	}
+	serving = true
+	return e, nil
+}
+
+func print(lg *zap.Logger, ec Config, sc etcdserver.ServerConfig, memberInitialized bool) {
+	// TODO: remove this after dropping "capnslog"
+	if lg == nil {
+		plog.Infof("name = %s", ec.Name)
+		if sc.ForceNewCluster {
+			plog.Infof("force new cluster")
+		}
+		plog.Infof("data dir = %s", sc.DataDir)
+		plog.Infof("member dir = %s", sc.MemberDir())
+		if sc.DedicatedWALDir != "" {
+			plog.Infof("dedicated WAL dir = %s", sc.DedicatedWALDir)
+		}
+		plog.Infof("heartbeat = %dms", sc.TickMs)
+		plog.Infof("election = %dms", sc.ElectionTicks*int(sc.TickMs))
+		plog.Infof("snapshot count = %d", sc.SnapshotCount)
+		if len(sc.DiscoveryURL) != 0 {
+			plog.Infof("discovery URL= %s", sc.DiscoveryURL)
+			if len(sc.DiscoveryProxy) != 0 {
+				plog.Infof("discovery proxy = %s", sc.DiscoveryProxy)
+			}
+		}
+		plog.Infof("advertise client URLs = %s", sc.ClientURLs)
+		if memberInitialized {
+			plog.Infof("initial advertise peer URLs = %s", sc.PeerURLs)
+			plog.Infof("initial cluster = %s", sc.InitialPeerURLsMap)
+		}
+	} else {
+		cors := make([]string, 0, len(ec.CORS))
+		for v := range ec.CORS {
+			cors = append(cors, v)
+		}
+		sort.Strings(cors)
+
+		hss := make([]string, 0, len(ec.HostWhitelist))
+		for v := range ec.HostWhitelist {
+			hss = append(hss, v)
+		}
+		sort.Strings(hss)
+
+		quota := ec.QuotaBackendBytes
+		if quota == 0 {
+			quota = etcdserver.DefaultQuotaBytes
+		}
+
+		lg.Info(
+			"starting an etcd server",
+			zap.String("etcd-version", version.Version),
+			zap.String("git-sha", version.GitSHA),
+			zap.String("go-version", runtime.Version()),
+			zap.String("go-os", runtime.GOOS),
+			zap.String("go-arch", runtime.GOARCH),
+			zap.Int("max-cpu-set", runtime.GOMAXPROCS(0)),
+			zap.Int("max-cpu-available", runtime.NumCPU()),
+			zap.Bool("member-initialized", memberInitialized),
+			zap.String("name", sc.Name),
+			zap.String("data-dir", sc.DataDir),
+			zap.String("wal-dir", ec.WalDir),
+			zap.String("wal-dir-dedicated", sc.DedicatedWALDir),
+			zap.String("member-dir", sc.MemberDir()),
+			zap.Bool("force-new-cluster", sc.ForceNewCluster),
+			zap.String("heartbeat-interval", fmt.Sprintf("%v", time.Duration(sc.TickMs)*time.Millisecond)),
+			zap.String("election-timeout", fmt.Sprintf("%v", time.Duration(sc.ElectionTicks*int(sc.TickMs))*time.Millisecond)),
+			zap.Bool("initial-election-tick-advance", sc.InitialElectionTickAdvance),
+			zap.Uint64("snapshot-count", sc.SnapshotCount),
+			zap.Uint64("snapshot-catchup-entries", sc.SnapshotCatchUpEntries),
+			zap.Strings("initial-advertise-peer-urls", ec.getAPURLs()),
+			zap.Strings("listen-peer-urls", ec.getLPURLs()),
+			zap.Strings("advertise-client-urls", ec.getACURLs()),
+			zap.Strings("listen-client-urls", ec.getLCURLs()),
+			zap.Strings("listen-metrics-urls", ec.getMetricsURLs()),
+			zap.Strings("cors", cors),
+			zap.Strings("host-whitelist", hss),
+			zap.String("initial-cluster", sc.InitialPeerURLsMap.String()),
+			zap.String("initial-cluster-state", ec.ClusterState),
+			zap.String("initial-cluster-token", sc.InitialClusterToken),
+			zap.Int64("quota-size-bytes", quota),
+			zap.Bool("pre-vote", sc.PreVote),
+			zap.Bool("initial-corrupt-check", sc.InitialCorruptCheck),
+			zap.String("corrupt-check-time-interval", sc.CorruptCheckTime.String()),
+			zap.String("auto-compaction-mode", sc.AutoCompactionMode),
+			zap.Duration("auto-compaction-retention", sc.AutoCompactionRetention),
+			zap.String("auto-compaction-interval", sc.AutoCompactionRetention.String()),
+			zap.String("discovery-url", sc.DiscoveryURL),
+			zap.String("discovery-proxy", sc.DiscoveryProxy),
+		)
+	}
+}
+
+// Config returns the current configuration.
+func (e *Etcd) Config() Config {
+	return e.cfg
+}
+
+// Close gracefully shuts down all servers/listeners.
+// Client requests will be terminated with request timeout.
+// After timeout, enforce remaning requests be closed immediately.
+func (e *Etcd) Close() {
+	fields := []zap.Field{
+		zap.String("name", e.cfg.Name),
+		zap.String("data-dir", e.cfg.Dir),
+		zap.Strings("advertise-peer-urls", e.cfg.getAPURLs()),
+		zap.Strings("advertise-client-urls", e.cfg.getACURLs()),
+	}
+	lg := e.GetLogger()
+	if lg != nil {
+		lg.Info("closing etcd server", fields...)
+	}
+	defer func() {
+		if lg != nil {
+			lg.Info("closed etcd server", fields...)
+			lg.Sync()
+		}
+	}()
+
+	e.closeOnce.Do(func() { close(e.stopc) })
+
+	// close client requests with request timeout
+	timeout := 2 * time.Second
+	if e.Server != nil {
+		timeout = e.Server.Cfg.ReqTimeout()
+	}
+	for _, sctx := range e.sctxs {
+		for ss := range sctx.serversC {
+			ctx, cancel := context.WithTimeout(context.Background(), timeout)
+			stopServers(ctx, ss)
+			cancel()
+		}
+	}
+
+	for _, sctx := range e.sctxs {
+		sctx.cancel()
+	}
+
+	for i := range e.Clients {
+		if e.Clients[i] != nil {
+			e.Clients[i].Close()
+		}
+	}
+
+	for i := range e.metricsListeners {
+		e.metricsListeners[i].Close()
+	}
+
+	// close rafthttp transports
+	if e.Server != nil {
+		e.Server.Stop()
+	}
+
+	// close all idle connections in peer handler (wait up to 1-second)
+	for i := range e.Peers {
+		if e.Peers[i] != nil && e.Peers[i].close != nil {
+			ctx, cancel := context.WithTimeout(context.Background(), time.Second)
+			e.Peers[i].close(ctx)
+			cancel()
+		}
+	}
+}
+
+func stopServers(ctx context.Context, ss *servers) {
+	shutdownNow := func() {
+		// first, close the http.Server
+		ss.http.Shutdown(ctx)
+		// then close grpc.Server; cancels all active RPCs
+		ss.grpc.Stop()
+	}
+
+	// do not grpc.Server.GracefulStop with TLS enabled etcd server
+	// See https://github.com/grpc/grpc-go/issues/1384#issuecomment-317124531
+	// and https://github.com/etcd-io/etcd/issues/8916
+	if ss.secure {
+		shutdownNow()
+		return
+	}
+
+	ch := make(chan struct{})
+	go func() {
+		defer close(ch)
+		// close listeners to stop accepting new connections,
+		// will block on any existing transports
+		ss.grpc.GracefulStop()
+	}()
+
+	// wait until all pending RPCs are finished
+	select {
+	case <-ch:
+	case <-ctx.Done():
+		// took too long, manually close open transports
+		// e.g. watch streams
+		shutdownNow()
+
+		// concurrent GracefulStop should be interrupted
+		<-ch
+	}
+}
+
+func (e *Etcd) Err() <-chan error { return e.errc }
+
+func configurePeerListeners(cfg *Config) (peers []*peerListener, err error) {
+	if err = updateCipherSuites(&cfg.PeerTLSInfo, cfg.CipherSuites); err != nil {
+		return nil, err
+	}
+	if err = cfg.PeerSelfCert(); err != nil {
+		if cfg.logger != nil {
+			cfg.logger.Fatal("failed to get peer self-signed certs", zap.Error(err))
+		} else {
+			plog.Fatalf("could not get certs (%v)", err)
+		}
+	}
+	if !cfg.PeerTLSInfo.Empty() {
+		if cfg.logger != nil {
+			cfg.logger.Info(
+				"starting with peer TLS",
+				zap.String("tls-info", fmt.Sprintf("%+v", cfg.PeerTLSInfo)),
+				zap.Strings("cipher-suites", cfg.CipherSuites),
+			)
+		} else {
+			plog.Infof("peerTLS: %s", cfg.PeerTLSInfo)
+		}
+	}
+
+	peers = make([]*peerListener, len(cfg.LPUrls))
+	defer func() {
+		if err == nil {
+			return
+		}
+		for i := range peers {
+			if peers[i] != nil && peers[i].close != nil {
+				if cfg.logger != nil {
+					cfg.logger.Warn(
+						"closing peer listener",
+						zap.String("address", cfg.LPUrls[i].String()),
+						zap.Error(err),
+					)
+				} else {
+					plog.Info("stopping listening for peers on ", cfg.LPUrls[i].String())
+				}
+				ctx, cancel := context.WithTimeout(context.Background(), time.Second)
+				peers[i].close(ctx)
+				cancel()
+			}
+		}
+	}()
+
+	for i, u := range cfg.LPUrls {
+		if u.Scheme == "http" {
+			if !cfg.PeerTLSInfo.Empty() {
+				if cfg.logger != nil {
+					cfg.logger.Warn("scheme is HTTP while key and cert files are present; ignoring key and cert files", zap.String("peer-url", u.String()))
+				} else {
+					plog.Warningf("The scheme of peer url %s is HTTP while peer key/cert files are presented. Ignored peer key/cert files.", u.String())
+				}
+			}
+			if cfg.PeerTLSInfo.ClientCertAuth {
+				if cfg.logger != nil {
+					cfg.logger.Warn("scheme is HTTP while --peer-client-cert-auth is enabled; ignoring client cert auth for this URL", zap.String("peer-url", u.String()))
+				} else {
+					plog.Warningf("The scheme of peer url %s is HTTP while client cert auth (--peer-client-cert-auth) is enabled. Ignored client cert auth for this url.", u.String())
+				}
+			}
+		}
+		peers[i] = &peerListener{close: func(context.Context) error { return nil }}
+		peers[i].Listener, err = rafthttp.NewListener(u, &cfg.PeerTLSInfo)
+		if err != nil {
+			return nil, err
+		}
+		// once serve, overwrite with 'http.Server.Shutdown'
+		peers[i].close = func(context.Context) error {
+			return peers[i].Listener.Close()
+		}
+	}
+	return peers, nil
+}
+
+// configure peer handlers after rafthttp.Transport started
+func (e *Etcd) servePeers() (err error) {
+	ph := etcdhttp.NewPeerHandler(e.GetLogger(), e.Server)
+	var peerTLScfg *tls.Config
+	if !e.cfg.PeerTLSInfo.Empty() {
+		if peerTLScfg, err = e.cfg.PeerTLSInfo.ServerConfig(); err != nil {
+			return err
+		}
+	}
+
+	for _, p := range e.Peers {
+		u := p.Listener.Addr().String()
+		gs := v3rpc.Server(e.Server, peerTLScfg)
+		m := cmux.New(p.Listener)
+		go gs.Serve(m.Match(cmux.HTTP2()))
+		srv := &http.Server{
+			Handler:     grpcHandlerFunc(gs, ph),
+			ReadTimeout: 5 * time.Minute,
+			ErrorLog:    defaultLog.New(ioutil.Discard, "", 0), // do not log user error
+		}
+		go srv.Serve(m.Match(cmux.Any()))
+		p.serve = func() error { return m.Serve() }
+		p.close = func(ctx context.Context) error {
+			// gracefully shutdown http.Server
+			// close open listeners, idle connections
+			// until context cancel or time-out
+			if e.cfg.logger != nil {
+				e.cfg.logger.Info(
+					"stopping serving peer traffic",
+					zap.String("address", u),
+				)
+			}
+			stopServers(ctx, &servers{secure: peerTLScfg != nil, grpc: gs, http: srv})
+			if e.cfg.logger != nil {
+				e.cfg.logger.Info(
+					"stopped serving peer traffic",
+					zap.String("address", u),
+				)
+			}
+			return nil
+		}
+	}
+
+	// start peer servers in a goroutine
+	for _, pl := range e.Peers {
+		go func(l *peerListener) {
+			u := l.Addr().String()
+			if e.cfg.logger != nil {
+				e.cfg.logger.Info(
+					"serving peer traffic",
+					zap.String("address", u),
+				)
+			} else {
+				plog.Info("listening for peers on ", u)
+			}
+			e.errHandler(l.serve())
+		}(pl)
+	}
+	return nil
+}
+
+func configureClientListeners(cfg *Config) (sctxs map[string]*serveCtx, err error) {
+	if err = updateCipherSuites(&cfg.ClientTLSInfo, cfg.CipherSuites); err != nil {
+		return nil, err
+	}
+	if err = cfg.ClientSelfCert(); err != nil {
+		if cfg.logger != nil {
+			cfg.logger.Fatal("failed to get client self-signed certs", zap.Error(err))
+		} else {
+			plog.Fatalf("could not get certs (%v)", err)
+		}
+	}
+	if cfg.EnablePprof {
+		if cfg.logger != nil {
+			cfg.logger.Info("pprof is enabled", zap.String("path", debugutil.HTTPPrefixPProf))
+		} else {
+			plog.Infof("pprof is enabled under %s", debugutil.HTTPPrefixPProf)
+		}
+	}
+
+	sctxs = make(map[string]*serveCtx)
+	for _, u := range cfg.LCUrls {
+		sctx := newServeCtx(cfg.logger)
+		if u.Scheme == "http" || u.Scheme == "unix" {
+			if !cfg.ClientTLSInfo.Empty() {
+				if cfg.logger != nil {
+					cfg.logger.Warn("scheme is HTTP while key and cert files are present; ignoring key and cert files", zap.String("client-url", u.String()))
+				} else {
+					plog.Warningf("The scheme of client url %s is HTTP while peer key/cert files are presented. Ignored key/cert files.", u.String())
+				}
+			}
+			if cfg.ClientTLSInfo.ClientCertAuth {
+				if cfg.logger != nil {
+					cfg.logger.Warn("scheme is HTTP while --client-cert-auth is enabled; ignoring client cert auth for this URL", zap.String("client-url", u.String()))
+				} else {
+					plog.Warningf("The scheme of client url %s is HTTP while client cert auth (--client-cert-auth) is enabled. Ignored client cert auth for this url.", u.String())
+				}
+			}
+		}
+		if (u.Scheme == "https" || u.Scheme == "unixs") && cfg.ClientTLSInfo.Empty() {
+			return nil, fmt.Errorf("TLS key/cert (--cert-file, --key-file) must be provided for client url %s with HTTPS scheme", u.String())
+		}
+
+		network := "tcp"
+		addr := u.Host
+		if u.Scheme == "unix" || u.Scheme == "unixs" {
+			network = "unix"
+			addr = u.Host + u.Path
+		}
+		sctx.network = network
+
+		sctx.secure = u.Scheme == "https" || u.Scheme == "unixs"
+		sctx.insecure = !sctx.secure
+		if oldctx := sctxs[addr]; oldctx != nil {
+			oldctx.secure = oldctx.secure || sctx.secure
+			oldctx.insecure = oldctx.insecure || sctx.insecure
+			continue
+		}
+
+		if sctx.l, err = net.Listen(network, addr); err != nil {
+			return nil, err
+		}
+		// net.Listener will rewrite ipv4 0.0.0.0 to ipv6 [::], breaking
+		// hosts that disable ipv6. So, use the address given by the user.
+		sctx.addr = addr
+
+		if fdLimit, fderr := runtimeutil.FDLimit(); fderr == nil {
+			if fdLimit <= reservedInternalFDNum {
+				if cfg.logger != nil {
+					cfg.logger.Fatal(
+						"file descriptor limit of etcd process is too low; please set higher",
+						zap.Uint64("limit", fdLimit),
+						zap.Int("recommended-limit", reservedInternalFDNum),
+					)
+				} else {
+					plog.Fatalf("file descriptor limit[%d] of etcd process is too low, and should be set higher than %d to ensure internal usage", fdLimit, reservedInternalFDNum)
+				}
+			}
+			sctx.l = transport.LimitListener(sctx.l, int(fdLimit-reservedInternalFDNum))
+		}
+
+		if network == "tcp" {
+			if sctx.l, err = transport.NewKeepAliveListener(sctx.l, network, nil); err != nil {
+				return nil, err
+			}
+		}
+
+		defer func() {
+			if err == nil {
+				return
+			}
+			sctx.l.Close()
+			if cfg.logger != nil {
+				cfg.logger.Warn(
+					"closing peer listener",
+					zap.String("address", u.Host),
+					zap.Error(err),
+				)
+			} else {
+				plog.Info("stopping listening for client requests on ", u.Host)
+			}
+		}()
+		for k := range cfg.UserHandlers {
+			sctx.userHandlers[k] = cfg.UserHandlers[k]
+		}
+		sctx.serviceRegister = cfg.ServiceRegister
+		if cfg.EnablePprof || cfg.Debug {
+			sctx.registerPprof()
+		}
+		if cfg.Debug {
+			sctx.registerTrace()
+		}
+		sctxs[addr] = sctx
+	}
+	return sctxs, nil
+}
+
+func (e *Etcd) serveClients() (err error) {
+	if !e.cfg.ClientTLSInfo.Empty() {
+		if e.cfg.logger != nil {
+			e.cfg.logger.Info(
+				"starting with client TLS",
+				zap.String("tls-info", fmt.Sprintf("%+v", e.cfg.ClientTLSInfo)),
+				zap.Strings("cipher-suites", e.cfg.CipherSuites),
+			)
+		} else {
+			plog.Infof("ClientTLS: %s", e.cfg.ClientTLSInfo)
+		}
+	}
+
+	// Start a client server goroutine for each listen address
+	var h http.Handler
+	if e.Config().EnableV2 {
+		if len(e.Config().ExperimentalEnableV2V3) > 0 {
+			srv := v2v3.NewServer(e.cfg.logger, v3client.New(e.Server), e.cfg.ExperimentalEnableV2V3)
+			h = v2http.NewClientHandler(e.GetLogger(), srv, e.Server.Cfg.ReqTimeout())
+		} else {
+			h = v2http.NewClientHandler(e.GetLogger(), e.Server, e.Server.Cfg.ReqTimeout())
+		}
+	} else {
+		mux := http.NewServeMux()
+		etcdhttp.HandleBasic(mux, e.Server)
+		h = mux
+	}
+
+	gopts := []grpc.ServerOption{}
+	if e.cfg.GRPCKeepAliveMinTime > time.Duration(0) {
+		gopts = append(gopts, grpc.KeepaliveEnforcementPolicy(keepalive.EnforcementPolicy{
+			MinTime:             e.cfg.GRPCKeepAliveMinTime,
+			PermitWithoutStream: false,
+		}))
+	}
+	if e.cfg.GRPCKeepAliveInterval > time.Duration(0) &&
+		e.cfg.GRPCKeepAliveTimeout > time.Duration(0) {
+		gopts = append(gopts, grpc.KeepaliveParams(keepalive.ServerParameters{
+			Time:    e.cfg.GRPCKeepAliveInterval,
+			Timeout: e.cfg.GRPCKeepAliveTimeout,
+		}))
+	}
+
+	// start client servers in each goroutine
+	for _, sctx := range e.sctxs {
+		go func(s *serveCtx) {
+			e.errHandler(s.serve(e.Server, &e.cfg.ClientTLSInfo, h, e.errHandler, gopts...))
+		}(sctx)
+	}
+	return nil
+}
+
+func (e *Etcd) serveMetrics() (err error) {
+	if e.cfg.Metrics == "extensive" {
+		grpc_prometheus.EnableHandlingTimeHistogram()
+	}
+
+	if len(e.cfg.ListenMetricsUrls) > 0 {
+		metricsMux := http.NewServeMux()
+		etcdhttp.HandleMetricsHealth(metricsMux, e.Server)
+
+		for _, murl := range e.cfg.ListenMetricsUrls {
+			tlsInfo := &e.cfg.ClientTLSInfo
+			if murl.Scheme == "http" {
+				tlsInfo = nil
+			}
+			ml, err := transport.NewListener(murl.Host, murl.Scheme, tlsInfo)
+			if err != nil {
+				return err
+			}
+			e.metricsListeners = append(e.metricsListeners, ml)
+			go func(u url.URL, ln net.Listener) {
+				if e.cfg.logger != nil {
+					e.cfg.logger.Info(
+						"serving metrics",
+						zap.String("address", u.String()),
+					)
+				} else {
+					plog.Info("listening for metrics on ", u.String())
+				}
+				e.errHandler(http.Serve(ln, metricsMux))
+			}(murl, ml)
+		}
+	}
+	return nil
+}
+
+func (e *Etcd) errHandler(err error) {
+	select {
+	case <-e.stopc:
+		return
+	default:
+	}
+	select {
+	case <-e.stopc:
+	case e.errc <- err:
+	}
+}
+
+// GetLogger returns the logger.
+func (e *Etcd) GetLogger() *zap.Logger {
+	e.cfg.loggerMu.RLock()
+	l := e.cfg.logger
+	e.cfg.loggerMu.RUnlock()
+	return l
+}
+
+func parseCompactionRetention(mode, retention string) (ret time.Duration, err error) {
+	h, err := strconv.Atoi(retention)
+	if err == nil {
+		switch mode {
+		case CompactorModeRevision:
+			ret = time.Duration(int64(h))
+		case CompactorModePeriodic:
+			ret = time.Duration(int64(h)) * time.Hour
+		}
+	} else {
+		// periodic compaction
+		ret, err = time.ParseDuration(retention)
+		if err != nil {
+			return 0, fmt.Errorf("error parsing CompactionRetention: %v", err)
+		}
+	}
+	return ret, nil
+}
diff --git a/vendor/go.etcd.io/etcd/embed/serve.go b/vendor/go.etcd.io/etcd/embed/serve.go
new file mode 100644
index 0000000..a3b20c4
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/embed/serve.go
@@ -0,0 +1,435 @@
+// Copyright 2015 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 embed
+
+import (
+	"context"
+	"fmt"
+	"io/ioutil"
+	defaultLog "log"
+	"net"
+	"net/http"
+	"strings"
+
+	"go.etcd.io/etcd/clientv3/credentials"
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/api/v3client"
+	"go.etcd.io/etcd/etcdserver/api/v3election"
+	"go.etcd.io/etcd/etcdserver/api/v3election/v3electionpb"
+	v3electiongw "go.etcd.io/etcd/etcdserver/api/v3election/v3electionpb/gw"
+	"go.etcd.io/etcd/etcdserver/api/v3lock"
+	"go.etcd.io/etcd/etcdserver/api/v3lock/v3lockpb"
+	v3lockgw "go.etcd.io/etcd/etcdserver/api/v3lock/v3lockpb/gw"
+	"go.etcd.io/etcd/etcdserver/api/v3rpc"
+	etcdservergw "go.etcd.io/etcd/etcdserver/etcdserverpb/gw"
+	"go.etcd.io/etcd/pkg/debugutil"
+	"go.etcd.io/etcd/pkg/httputil"
+	"go.etcd.io/etcd/pkg/transport"
+
+	gw "github.com/grpc-ecosystem/grpc-gateway/runtime"
+	"github.com/soheilhy/cmux"
+	"github.com/tmc/grpc-websocket-proxy/wsproxy"
+	"go.uber.org/zap"
+	"golang.org/x/net/trace"
+	"google.golang.org/grpc"
+)
+
+type serveCtx struct {
+	lg       *zap.Logger
+	l        net.Listener
+	addr     string
+	network  string
+	secure   bool
+	insecure bool
+
+	ctx    context.Context
+	cancel context.CancelFunc
+
+	userHandlers    map[string]http.Handler
+	serviceRegister func(*grpc.Server)
+	serversC        chan *servers
+}
+
+type servers struct {
+	secure bool
+	grpc   *grpc.Server
+	http   *http.Server
+}
+
+func newServeCtx(lg *zap.Logger) *serveCtx {
+	ctx, cancel := context.WithCancel(context.Background())
+	return &serveCtx{
+		lg:           lg,
+		ctx:          ctx,
+		cancel:       cancel,
+		userHandlers: make(map[string]http.Handler),
+		serversC:     make(chan *servers, 2), // in case sctx.insecure,sctx.secure true
+	}
+}
+
+// serve accepts incoming connections on the listener l,
+// creating a new service goroutine for each. The service goroutines
+// read requests and then call handler to reply to them.
+func (sctx *serveCtx) serve(
+	s *etcdserver.EtcdServer,
+	tlsinfo *transport.TLSInfo,
+	handler http.Handler,
+	errHandler func(error),
+	gopts ...grpc.ServerOption) (err error) {
+	logger := defaultLog.New(ioutil.Discard, "etcdhttp", 0)
+	<-s.ReadyNotify()
+
+	if sctx.lg == nil {
+		plog.Info("ready to serve client requests")
+	}
+
+	m := cmux.New(sctx.l)
+	v3c := v3client.New(s)
+	servElection := v3election.NewElectionServer(v3c)
+	servLock := v3lock.NewLockServer(v3c)
+
+	var gs *grpc.Server
+	defer func() {
+		if err != nil && gs != nil {
+			gs.Stop()
+		}
+	}()
+
+	if sctx.insecure {
+		gs = v3rpc.Server(s, nil, gopts...)
+		v3electionpb.RegisterElectionServer(gs, servElection)
+		v3lockpb.RegisterLockServer(gs, servLock)
+		if sctx.serviceRegister != nil {
+			sctx.serviceRegister(gs)
+		}
+		grpcl := m.Match(cmux.HTTP2())
+		go func() { errHandler(gs.Serve(grpcl)) }()
+
+		var gwmux *gw.ServeMux
+		if s.Cfg.EnableGRPCGateway {
+			gwmux, err = sctx.registerGateway([]grpc.DialOption{grpc.WithInsecure()})
+			if err != nil {
+				return err
+			}
+		}
+
+		httpmux := sctx.createMux(gwmux, handler)
+
+		srvhttp := &http.Server{
+			Handler:  createAccessController(sctx.lg, s, httpmux),
+			ErrorLog: logger, // do not log user error
+		}
+		httpl := m.Match(cmux.HTTP1())
+		go func() { errHandler(srvhttp.Serve(httpl)) }()
+
+		sctx.serversC <- &servers{grpc: gs, http: srvhttp}
+		if sctx.lg != nil {
+			sctx.lg.Info(
+				"serving client traffic insecurely; this is strongly discouraged!",
+				zap.String("address", sctx.l.Addr().String()),
+			)
+		} else {
+			plog.Noticef("serving insecure client requests on %s, this is strongly discouraged!", sctx.l.Addr().String())
+		}
+	}
+
+	if sctx.secure {
+		tlscfg, tlsErr := tlsinfo.ServerConfig()
+		if tlsErr != nil {
+			return tlsErr
+		}
+		gs = v3rpc.Server(s, tlscfg, gopts...)
+		v3electionpb.RegisterElectionServer(gs, servElection)
+		v3lockpb.RegisterLockServer(gs, servLock)
+		if sctx.serviceRegister != nil {
+			sctx.serviceRegister(gs)
+		}
+		handler = grpcHandlerFunc(gs, handler)
+
+		var gwmux *gw.ServeMux
+		if s.Cfg.EnableGRPCGateway {
+			dtls := tlscfg.Clone()
+			// trust local server
+			dtls.InsecureSkipVerify = true
+			bundle := credentials.NewBundle(credentials.Config{TLSConfig: dtls})
+			opts := []grpc.DialOption{grpc.WithTransportCredentials(bundle.TransportCredentials())}
+			gwmux, err = sctx.registerGateway(opts)
+			if err != nil {
+				return err
+			}
+		}
+
+		var tlsl net.Listener
+		tlsl, err = transport.NewTLSListener(m.Match(cmux.Any()), tlsinfo)
+		if err != nil {
+			return err
+		}
+		// TODO: add debug flag; enable logging when debug flag is set
+		httpmux := sctx.createMux(gwmux, handler)
+
+		srv := &http.Server{
+			Handler:   createAccessController(sctx.lg, s, httpmux),
+			TLSConfig: tlscfg,
+			ErrorLog:  logger, // do not log user error
+		}
+		go func() { errHandler(srv.Serve(tlsl)) }()
+
+		sctx.serversC <- &servers{secure: true, grpc: gs, http: srv}
+		if sctx.lg != nil {
+			sctx.lg.Info(
+				"serving client traffic securely",
+				zap.String("address", sctx.l.Addr().String()),
+			)
+		} else {
+			plog.Infof("serving client requests on %s", sctx.l.Addr().String())
+		}
+	}
+
+	close(sctx.serversC)
+	return m.Serve()
+}
+
+// grpcHandlerFunc returns an http.Handler that delegates to grpcServer on incoming gRPC
+// connections or otherHandler otherwise. Given in gRPC docs.
+func grpcHandlerFunc(grpcServer *grpc.Server, otherHandler http.Handler) http.Handler {
+	if otherHandler == nil {
+		return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+			grpcServer.ServeHTTP(w, r)
+		})
+	}
+	return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		if r.ProtoMajor == 2 && strings.Contains(r.Header.Get("Content-Type"), "application/grpc") {
+			grpcServer.ServeHTTP(w, r)
+		} else {
+			otherHandler.ServeHTTP(w, r)
+		}
+	})
+}
+
+type registerHandlerFunc func(context.Context, *gw.ServeMux, *grpc.ClientConn) error
+
+func (sctx *serveCtx) registerGateway(opts []grpc.DialOption) (*gw.ServeMux, error) {
+	ctx := sctx.ctx
+
+	addr := sctx.addr
+	if network := sctx.network; network == "unix" {
+		// explicitly define unix network for gRPC socket support
+		addr = fmt.Sprintf("%s://%s", network, addr)
+	}
+
+	conn, err := grpc.DialContext(ctx, addr, opts...)
+	if err != nil {
+		return nil, err
+	}
+	gwmux := gw.NewServeMux()
+
+	handlers := []registerHandlerFunc{
+		etcdservergw.RegisterKVHandler,
+		etcdservergw.RegisterWatchHandler,
+		etcdservergw.RegisterLeaseHandler,
+		etcdservergw.RegisterClusterHandler,
+		etcdservergw.RegisterMaintenanceHandler,
+		etcdservergw.RegisterAuthHandler,
+		v3lockgw.RegisterLockHandler,
+		v3electiongw.RegisterElectionHandler,
+	}
+	for _, h := range handlers {
+		if err := h(ctx, gwmux, conn); err != nil {
+			return nil, err
+		}
+	}
+	go func() {
+		<-ctx.Done()
+		if cerr := conn.Close(); cerr != nil {
+			if sctx.lg != nil {
+				sctx.lg.Warn(
+					"failed to close connection",
+					zap.String("address", sctx.l.Addr().String()),
+					zap.Error(cerr),
+				)
+			} else {
+				plog.Warningf("failed to close conn to %s: %v", sctx.l.Addr().String(), cerr)
+			}
+		}
+	}()
+
+	return gwmux, nil
+}
+
+func (sctx *serveCtx) createMux(gwmux *gw.ServeMux, handler http.Handler) *http.ServeMux {
+	httpmux := http.NewServeMux()
+	for path, h := range sctx.userHandlers {
+		httpmux.Handle(path, h)
+	}
+
+	if gwmux != nil {
+		httpmux.Handle(
+			"/v3/",
+			wsproxy.WebsocketProxy(
+				gwmux,
+				wsproxy.WithRequestMutator(
+					// Default to the POST method for streams
+					func(_ *http.Request, outgoing *http.Request) *http.Request {
+						outgoing.Method = "POST"
+						return outgoing
+					},
+				),
+			),
+		)
+	}
+	if handler != nil {
+		httpmux.Handle("/", handler)
+	}
+	return httpmux
+}
+
+// createAccessController wraps HTTP multiplexer:
+// - mutate gRPC gateway request paths
+// - check hostname whitelist
+// client HTTP requests goes here first
+func createAccessController(lg *zap.Logger, s *etcdserver.EtcdServer, mux *http.ServeMux) http.Handler {
+	return &accessController{lg: lg, s: s, mux: mux}
+}
+
+type accessController struct {
+	lg  *zap.Logger
+	s   *etcdserver.EtcdServer
+	mux *http.ServeMux
+}
+
+func (ac *accessController) ServeHTTP(rw http.ResponseWriter, req *http.Request) {
+	// redirect for backward compatibilities
+	if req != nil && req.URL != nil && strings.HasPrefix(req.URL.Path, "/v3beta/") {
+		req.URL.Path = strings.Replace(req.URL.Path, "/v3beta/", "/v3/", 1)
+	}
+
+	if req.TLS == nil { // check origin if client connection is not secure
+		host := httputil.GetHostname(req)
+		if !ac.s.AccessController.IsHostWhitelisted(host) {
+			if ac.lg != nil {
+				ac.lg.Warn(
+					"rejecting HTTP request to prevent DNS rebinding attacks",
+					zap.String("host", host),
+				)
+			} else {
+				plog.Warningf("rejecting HTTP request from %q to prevent DNS rebinding attacks", host)
+			}
+			// TODO: use Go's "http.StatusMisdirectedRequest" (421)
+			// https://github.com/golang/go/commit/4b8a7eafef039af1834ef9bfa879257c4a72b7b5
+			http.Error(rw, errCVE20185702(host), 421)
+			return
+		}
+	} else if ac.s.Cfg.ClientCertAuthEnabled && ac.s.Cfg.EnableGRPCGateway &&
+		ac.s.AuthStore().IsAuthEnabled() && strings.HasPrefix(req.URL.Path, "/v3/") {
+		for _, chains := range req.TLS.VerifiedChains {
+			if len(chains) < 1 {
+				continue
+			}
+			if len(chains[0].Subject.CommonName) != 0 {
+				http.Error(rw, "CommonName of client sending a request against gateway will be ignored and not used as expected", 400)
+				return
+			}
+		}
+	}
+
+	// Write CORS header.
+	if ac.s.AccessController.OriginAllowed("*") {
+		addCORSHeader(rw, "*")
+	} else if origin := req.Header.Get("Origin"); ac.s.OriginAllowed(origin) {
+		addCORSHeader(rw, origin)
+	}
+
+	if req.Method == "OPTIONS" {
+		rw.WriteHeader(http.StatusOK)
+		return
+	}
+
+	ac.mux.ServeHTTP(rw, req)
+}
+
+// addCORSHeader adds the correct cors headers given an origin
+func addCORSHeader(w http.ResponseWriter, origin string) {
+	w.Header().Add("Access-Control-Allow-Methods", "POST, GET, OPTIONS, PUT, DELETE")
+	w.Header().Add("Access-Control-Allow-Origin", origin)
+	w.Header().Add("Access-Control-Allow-Headers", "accept, content-type, authorization")
+}
+
+// https://github.com/transmission/transmission/pull/468
+func errCVE20185702(host string) string {
+	return fmt.Sprintf(`
+etcd received your request, but the Host header was unrecognized.
+
+To fix this, choose one of the following options:
+- Enable TLS, then any HTTPS request will be allowed.
+- Add the hostname you want to use to the whitelist in settings.
+  - e.g. etcd --host-whitelist %q
+
+This requirement has been added to help prevent "DNS Rebinding" attacks (CVE-2018-5702).
+`, host)
+}
+
+// WrapCORS wraps existing handler with CORS.
+// TODO: deprecate this after v2 proxy deprecate
+func WrapCORS(cors map[string]struct{}, h http.Handler) http.Handler {
+	return &corsHandler{
+		ac: &etcdserver.AccessController{CORS: cors},
+		h:  h,
+	}
+}
+
+type corsHandler struct {
+	ac *etcdserver.AccessController
+	h  http.Handler
+}
+
+func (ch *corsHandler) ServeHTTP(rw http.ResponseWriter, req *http.Request) {
+	if ch.ac.OriginAllowed("*") {
+		addCORSHeader(rw, "*")
+	} else if origin := req.Header.Get("Origin"); ch.ac.OriginAllowed(origin) {
+		addCORSHeader(rw, origin)
+	}
+
+	if req.Method == "OPTIONS" {
+		rw.WriteHeader(http.StatusOK)
+		return
+	}
+
+	ch.h.ServeHTTP(rw, req)
+}
+
+func (sctx *serveCtx) registerUserHandler(s string, h http.Handler) {
+	if sctx.userHandlers[s] != nil {
+		if sctx.lg != nil {
+			sctx.lg.Warn("path is already registered by user handler", zap.String("path", s))
+		} else {
+			plog.Warningf("path %s already registered by user handler", s)
+		}
+		return
+	}
+	sctx.userHandlers[s] = h
+}
+
+func (sctx *serveCtx) registerPprof() {
+	for p, h := range debugutil.PProfHandlers() {
+		sctx.registerUserHandler(p, h)
+	}
+}
+
+func (sctx *serveCtx) registerTrace() {
+	reqf := func(w http.ResponseWriter, r *http.Request) { trace.Render(w, r, true) }
+	sctx.registerUserHandler("/debug/requests", http.HandlerFunc(reqf))
+	evf := func(w http.ResponseWriter, r *http.Request) { trace.RenderEvents(w, r, true) }
+	sctx.registerUserHandler("/debug/events", http.HandlerFunc(evf))
+}
diff --git a/vendor/go.etcd.io/etcd/embed/util.go b/vendor/go.etcd.io/etcd/embed/util.go
new file mode 100644
index 0000000..40f3ce9
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/embed/util.go
@@ -0,0 +1,29 @@
+// Copyright 2016 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 embed
+
+import (
+	"path/filepath"
+
+	"go.etcd.io/etcd/wal"
+)
+
+func isMemberInitialized(cfg *Config) bool {
+	waldir := cfg.WalDir
+	if waldir == "" {
+		waldir = filepath.Join(cfg.Dir, "member", "wal")
+	}
+	return wal.Exist(waldir)
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/capability.go b/vendor/go.etcd.io/etcd/etcdserver/api/capability.go
new file mode 100644
index 0000000..09b754d
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/capability.go
@@ -0,0 +1,99 @@
+// Copyright 2015 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 api
+
+import (
+	"sync"
+
+	"go.etcd.io/etcd/version"
+	"go.uber.org/zap"
+
+	"github.com/coreos/go-semver/semver"
+	"github.com/coreos/pkg/capnslog"
+)
+
+type Capability string
+
+const (
+	AuthCapability  Capability = "auth"
+	V3rpcCapability Capability = "v3rpc"
+)
+
+var (
+	plog = capnslog.NewPackageLogger("go.etcd.io/etcd", "etcdserver/api")
+
+	// capabilityMaps is a static map of version to capability map.
+	capabilityMaps = map[string]map[Capability]bool{
+		"3.0.0": {AuthCapability: true, V3rpcCapability: true},
+		"3.1.0": {AuthCapability: true, V3rpcCapability: true},
+		"3.2.0": {AuthCapability: true, V3rpcCapability: true},
+		"3.3.0": {AuthCapability: true, V3rpcCapability: true},
+		"3.4.0": {AuthCapability: true, V3rpcCapability: true},
+		"3.5.0": {AuthCapability: true, V3rpcCapability: true},
+	}
+
+	enableMapMu sync.RWMutex
+	// enabledMap points to a map in capabilityMaps
+	enabledMap map[Capability]bool
+
+	curVersion *semver.Version
+)
+
+func init() {
+	enabledMap = map[Capability]bool{
+		AuthCapability:  true,
+		V3rpcCapability: true,
+	}
+}
+
+// UpdateCapability updates the enabledMap when the cluster version increases.
+func UpdateCapability(lg *zap.Logger, v *semver.Version) {
+	if v == nil {
+		// if recovered but version was never set by cluster
+		return
+	}
+	enableMapMu.Lock()
+	if curVersion != nil && !curVersion.LessThan(*v) {
+		enableMapMu.Unlock()
+		return
+	}
+	curVersion = v
+	enabledMap = capabilityMaps[curVersion.String()]
+	enableMapMu.Unlock()
+
+	if lg != nil {
+		lg.Info(
+			"enabled capabilities for version",
+			zap.String("cluster-version", version.Cluster(v.String())),
+		)
+	} else {
+		plog.Infof("enabled capabilities for version %s", version.Cluster(v.String()))
+	}
+}
+
+func IsCapabilityEnabled(c Capability) bool {
+	enableMapMu.RLock()
+	defer enableMapMu.RUnlock()
+	if enabledMap == nil {
+		return false
+	}
+	return enabledMap[c]
+}
+
+func EnableCapability(c Capability) {
+	enableMapMu.Lock()
+	defer enableMapMu.Unlock()
+	enabledMap[c] = true
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/cluster.go b/vendor/go.etcd.io/etcd/etcdserver/api/cluster.go
new file mode 100644
index 0000000..901be9d
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/cluster.go
@@ -0,0 +1,38 @@
+// Copyright 2016 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 api
+
+import (
+	"go.etcd.io/etcd/etcdserver/api/membership"
+	"go.etcd.io/etcd/pkg/types"
+
+	"github.com/coreos/go-semver/semver"
+)
+
+// Cluster is an interface representing a collection of members in one etcd cluster.
+type Cluster interface {
+	// ID returns the cluster ID
+	ID() types.ID
+	// ClientURLs returns an aggregate set of all URLs on which this
+	// cluster is listening for client requests
+	ClientURLs() []string
+	// Members returns a slice of members sorted by their ID
+	Members() []*membership.Member
+	// Member retrieves a particular member based on ID, or nil if the
+	// member does not exist in the cluster
+	Member(id types.ID) *membership.Member
+	// Version is the cluster-wide minimum major.minor version.
+	Version() *semver.Version
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/doc.go b/vendor/go.etcd.io/etcd/etcdserver/api/doc.go
new file mode 100644
index 0000000..f44881b
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/doc.go
@@ -0,0 +1,16 @@
+// Copyright 2016 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 api manages the capabilities and features that are exposed to clients by the etcd cluster.
+package api
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/etcdhttp/base.go b/vendor/go.etcd.io/etcd/etcdserver/api/etcdhttp/base.go
new file mode 100644
index 0000000..c9df62e
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/etcdhttp/base.go
@@ -0,0 +1,203 @@
+// Copyright 2015 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 etcdhttp
+
+import (
+	"encoding/json"
+	"expvar"
+	"fmt"
+	"net/http"
+	"strings"
+
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/api"
+	"go.etcd.io/etcd/etcdserver/api/v2error"
+	"go.etcd.io/etcd/etcdserver/api/v2http/httptypes"
+	"go.etcd.io/etcd/pkg/logutil"
+	"go.etcd.io/etcd/version"
+
+	"github.com/coreos/pkg/capnslog"
+	"go.uber.org/zap"
+)
+
+var (
+	plog = capnslog.NewPackageLogger("go.etcd.io/etcd", "etcdserver/api/etcdhttp")
+	mlog = logutil.NewMergeLogger(plog)
+)
+
+const (
+	configPath  = "/config"
+	varsPath    = "/debug/vars"
+	versionPath = "/version"
+)
+
+// HandleBasic adds handlers to a mux for serving JSON etcd client requests
+// that do not access the v2 store.
+func HandleBasic(mux *http.ServeMux, server etcdserver.ServerPeer) {
+	mux.HandleFunc(varsPath, serveVars)
+
+	// TODO: deprecate '/config/local/log' in v3.5
+	mux.HandleFunc(configPath+"/local/log", logHandleFunc)
+
+	HandleMetricsHealth(mux, server)
+	mux.HandleFunc(versionPath, versionHandler(server.Cluster(), serveVersion))
+}
+
+func versionHandler(c api.Cluster, fn func(http.ResponseWriter, *http.Request, string)) http.HandlerFunc {
+	return func(w http.ResponseWriter, r *http.Request) {
+		v := c.Version()
+		if v != nil {
+			fn(w, r, v.String())
+		} else {
+			fn(w, r, "not_decided")
+		}
+	}
+}
+
+func serveVersion(w http.ResponseWriter, r *http.Request, clusterV string) {
+	if !allowMethod(w, r, "GET") {
+		return
+	}
+	vs := version.Versions{
+		Server:  version.Version,
+		Cluster: clusterV,
+	}
+
+	w.Header().Set("Content-Type", "application/json")
+	b, err := json.Marshal(&vs)
+	if err != nil {
+		plog.Panicf("cannot marshal versions to json (%v)", err)
+	}
+	w.Write(b)
+}
+
+// TODO: deprecate '/config/local/log' in v3.5
+func logHandleFunc(w http.ResponseWriter, r *http.Request) {
+	if !allowMethod(w, r, "PUT") {
+		return
+	}
+
+	in := struct{ Level string }{}
+
+	d := json.NewDecoder(r.Body)
+	if err := d.Decode(&in); err != nil {
+		WriteError(nil, w, r, httptypes.NewHTTPError(http.StatusBadRequest, "Invalid json body"))
+		return
+	}
+
+	logl, err := capnslog.ParseLevel(strings.ToUpper(in.Level))
+	if err != nil {
+		WriteError(nil, w, r, httptypes.NewHTTPError(http.StatusBadRequest, "Invalid log level "+in.Level))
+		return
+	}
+
+	plog.Noticef("globalLogLevel set to %q", logl.String())
+	capnslog.SetGlobalLogLevel(logl)
+	w.WriteHeader(http.StatusNoContent)
+}
+
+func serveVars(w http.ResponseWriter, r *http.Request) {
+	if !allowMethod(w, r, "GET") {
+		return
+	}
+
+	w.Header().Set("Content-Type", "application/json; charset=utf-8")
+	fmt.Fprintf(w, "{\n")
+	first := true
+	expvar.Do(func(kv expvar.KeyValue) {
+		if !first {
+			fmt.Fprintf(w, ",\n")
+		}
+		first = false
+		fmt.Fprintf(w, "%q: %s", kv.Key, kv.Value)
+	})
+	fmt.Fprintf(w, "\n}\n")
+}
+
+func allowMethod(w http.ResponseWriter, r *http.Request, m string) bool {
+	if m == r.Method {
+		return true
+	}
+	w.Header().Set("Allow", m)
+	http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed)
+	return false
+}
+
+// WriteError logs and writes the given Error to the ResponseWriter
+// If Error is an etcdErr, it is rendered to the ResponseWriter
+// Otherwise, it is assumed to be a StatusInternalServerError
+func WriteError(lg *zap.Logger, w http.ResponseWriter, r *http.Request, err error) {
+	if err == nil {
+		return
+	}
+	switch e := err.(type) {
+	case *v2error.Error:
+		e.WriteTo(w)
+
+	case *httptypes.HTTPError:
+		if et := e.WriteTo(w); et != nil {
+			if lg != nil {
+				lg.Debug(
+					"failed to write v2 HTTP error",
+					zap.String("remote-addr", r.RemoteAddr),
+					zap.String("internal-server-error", e.Error()),
+					zap.Error(et),
+				)
+			} else {
+				plog.Debugf("error writing HTTPError (%v) to %s", et, r.RemoteAddr)
+			}
+		}
+
+	default:
+		switch err {
+		case etcdserver.ErrTimeoutDueToLeaderFail, etcdserver.ErrTimeoutDueToConnectionLost, etcdserver.ErrNotEnoughStartedMembers,
+			etcdserver.ErrUnhealthy:
+			if lg != nil {
+				lg.Warn(
+					"v2 response error",
+					zap.String("remote-addr", r.RemoteAddr),
+					zap.String("internal-server-error", err.Error()),
+				)
+			} else {
+				mlog.MergeError(err)
+			}
+
+		default:
+			if lg != nil {
+				lg.Warn(
+					"unexpected v2 response error",
+					zap.String("remote-addr", r.RemoteAddr),
+					zap.String("internal-server-error", err.Error()),
+				)
+			} else {
+				mlog.MergeErrorf("got unexpected response error (%v)", err)
+			}
+		}
+
+		herr := httptypes.NewHTTPError(http.StatusInternalServerError, "Internal Server Error")
+		if et := herr.WriteTo(w); et != nil {
+			if lg != nil {
+				lg.Debug(
+					"failed to write v2 HTTP error",
+					zap.String("remote-addr", r.RemoteAddr),
+					zap.String("internal-server-error", err.Error()),
+					zap.Error(et),
+				)
+			} else {
+				plog.Debugf("error writing HTTPError (%v) to %s", et, r.RemoteAddr)
+			}
+		}
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/etcdhttp/doc.go b/vendor/go.etcd.io/etcd/etcdserver/api/etcdhttp/doc.go
new file mode 100644
index 0000000..a03b626
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/etcdhttp/doc.go
@@ -0,0 +1,16 @@
+// 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 etcdhttp implements HTTP transportation layer for etcdserver.
+package etcdhttp
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/etcdhttp/metrics.go b/vendor/go.etcd.io/etcd/etcdserver/api/etcdhttp/metrics.go
new file mode 100644
index 0000000..f455e40
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/etcdhttp/metrics.go
@@ -0,0 +1,123 @@
+// 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 etcdhttp
+
+import (
+	"context"
+	"encoding/json"
+	"net/http"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/raft"
+
+	"github.com/prometheus/client_golang/prometheus"
+	"github.com/prometheus/client_golang/prometheus/promhttp"
+)
+
+const (
+	PathMetrics = "/metrics"
+	PathHealth  = "/health"
+)
+
+// HandleMetricsHealth registers metrics and health handlers.
+func HandleMetricsHealth(mux *http.ServeMux, srv etcdserver.ServerV2) {
+	mux.Handle(PathMetrics, promhttp.Handler())
+	mux.Handle(PathHealth, NewHealthHandler(func() Health { return checkHealth(srv) }))
+}
+
+// HandlePrometheus registers prometheus handler on '/metrics'.
+func HandlePrometheus(mux *http.ServeMux) {
+	mux.Handle(PathMetrics, promhttp.Handler())
+}
+
+// NewHealthHandler handles '/health' requests.
+func NewHealthHandler(hfunc func() Health) http.HandlerFunc {
+	return func(w http.ResponseWriter, r *http.Request) {
+		if r.Method != http.MethodGet {
+			w.Header().Set("Allow", http.MethodGet)
+			http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed)
+			return
+		}
+		h := hfunc()
+		d, _ := json.Marshal(h)
+		if h.Health != "true" {
+			http.Error(w, string(d), http.StatusServiceUnavailable)
+			return
+		}
+		w.WriteHeader(http.StatusOK)
+		w.Write(d)
+	}
+}
+
+var (
+	healthSuccess = prometheus.NewCounter(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "health_success",
+		Help:      "The total number of successful health checks",
+	})
+	healthFailed = prometheus.NewCounter(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "health_failures",
+		Help:      "The total number of failed health checks",
+	})
+)
+
+func init() {
+	prometheus.MustRegister(healthSuccess)
+	prometheus.MustRegister(healthFailed)
+}
+
+// Health defines etcd server health status.
+// TODO: remove manual parsing in etcdctl cluster-health
+type Health struct {
+	Health string `json:"health"`
+}
+
+// TODO: server NOSPACE, etcdserver.ErrNoLeader in health API
+
+func checkHealth(srv etcdserver.ServerV2) Health {
+	h := Health{Health: "true"}
+
+	as := srv.Alarms()
+	if len(as) > 0 {
+		h.Health = "false"
+	}
+
+	if h.Health == "true" {
+		if uint64(srv.Leader()) == raft.None {
+			h.Health = "false"
+		}
+	}
+
+	if h.Health == "true" {
+		ctx, cancel := context.WithTimeout(context.Background(), time.Second)
+		_, err := srv.Do(ctx, etcdserverpb.Request{Method: "QGET"})
+		cancel()
+		if err != nil {
+			h.Health = "false"
+		}
+	}
+
+	if h.Health == "true" {
+		healthSuccess.Inc()
+	} else {
+		healthFailed.Inc()
+	}
+	return h
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/etcdhttp/peer.go b/vendor/go.etcd.io/etcd/etcdserver/api/etcdhttp/peer.go
new file mode 100644
index 0000000..6c61bf5
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/etcdhttp/peer.go
@@ -0,0 +1,159 @@
+// Copyright 2015 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 etcdhttp
+
+import (
+	"encoding/json"
+	"fmt"
+	"net/http"
+	"strconv"
+	"strings"
+
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/api"
+	"go.etcd.io/etcd/etcdserver/api/membership"
+	"go.etcd.io/etcd/etcdserver/api/rafthttp"
+	"go.etcd.io/etcd/lease/leasehttp"
+	"go.etcd.io/etcd/pkg/types"
+
+	"go.uber.org/zap"
+)
+
+const (
+	peerMembersPath         = "/members"
+	peerMemberPromotePrefix = "/members/promote/"
+)
+
+// NewPeerHandler generates an http.Handler to handle etcd peer requests.
+func NewPeerHandler(lg *zap.Logger, s etcdserver.ServerPeer) http.Handler {
+	return newPeerHandler(lg, s, s.RaftHandler(), s.LeaseHandler())
+}
+
+func newPeerHandler(lg *zap.Logger, s etcdserver.Server, raftHandler http.Handler, leaseHandler http.Handler) http.Handler {
+	peerMembersHandler := newPeerMembersHandler(lg, s.Cluster())
+	peerMemberPromoteHandler := newPeerMemberPromoteHandler(lg, s)
+
+	mux := http.NewServeMux()
+	mux.HandleFunc("/", http.NotFound)
+	mux.Handle(rafthttp.RaftPrefix, raftHandler)
+	mux.Handle(rafthttp.RaftPrefix+"/", raftHandler)
+	mux.Handle(peerMembersPath, peerMembersHandler)
+	mux.Handle(peerMemberPromotePrefix, peerMemberPromoteHandler)
+	if leaseHandler != nil {
+		mux.Handle(leasehttp.LeasePrefix, leaseHandler)
+		mux.Handle(leasehttp.LeaseInternalPrefix, leaseHandler)
+	}
+	mux.HandleFunc(versionPath, versionHandler(s.Cluster(), serveVersion))
+	return mux
+}
+
+func newPeerMembersHandler(lg *zap.Logger, cluster api.Cluster) http.Handler {
+	return &peerMembersHandler{
+		lg:      lg,
+		cluster: cluster,
+	}
+}
+
+type peerMembersHandler struct {
+	lg      *zap.Logger
+	cluster api.Cluster
+}
+
+func newPeerMemberPromoteHandler(lg *zap.Logger, s etcdserver.Server) http.Handler {
+	return &peerMemberPromoteHandler{
+		lg:      lg,
+		cluster: s.Cluster(),
+		server:  s,
+	}
+}
+
+type peerMemberPromoteHandler struct {
+	lg      *zap.Logger
+	cluster api.Cluster
+	server  etcdserver.Server
+}
+
+func (h *peerMembersHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
+	if !allowMethod(w, r, "GET") {
+		return
+	}
+	w.Header().Set("X-Etcd-Cluster-ID", h.cluster.ID().String())
+
+	if r.URL.Path != peerMembersPath {
+		http.Error(w, "bad path", http.StatusBadRequest)
+		return
+	}
+	ms := h.cluster.Members()
+	w.Header().Set("Content-Type", "application/json")
+	if err := json.NewEncoder(w).Encode(ms); err != nil {
+		if h.lg != nil {
+			h.lg.Warn("failed to encode membership members", zap.Error(err))
+		} else {
+			plog.Warningf("failed to encode members response (%v)", err)
+		}
+	}
+}
+
+func (h *peerMemberPromoteHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
+	if !allowMethod(w, r, "POST") {
+		return
+	}
+	w.Header().Set("X-Etcd-Cluster-ID", h.cluster.ID().String())
+
+	if !strings.HasPrefix(r.URL.Path, peerMemberPromotePrefix) {
+		http.Error(w, "bad path", http.StatusBadRequest)
+		return
+	}
+	idStr := strings.TrimPrefix(r.URL.Path, peerMemberPromotePrefix)
+	id, err := strconv.ParseUint(idStr, 10, 64)
+	if err != nil {
+		http.Error(w, fmt.Sprintf("member %s not found in cluster", idStr), http.StatusNotFound)
+		return
+	}
+
+	resp, err := h.server.PromoteMember(r.Context(), id)
+	if err != nil {
+		switch err {
+		case membership.ErrIDNotFound:
+			http.Error(w, err.Error(), http.StatusNotFound)
+		case membership.ErrMemberNotLearner:
+			http.Error(w, err.Error(), http.StatusPreconditionFailed)
+		case etcdserver.ErrLearnerNotReady:
+			http.Error(w, err.Error(), http.StatusPreconditionFailed)
+		default:
+			WriteError(h.lg, w, r, err)
+		}
+		if h.lg != nil {
+			h.lg.Warn(
+				"failed to promote a member",
+				zap.String("member-id", types.ID(id).String()),
+				zap.Error(err),
+			)
+		} else {
+			plog.Errorf("error promoting member %s (%v)", types.ID(id).String(), err)
+		}
+		return
+	}
+
+	w.Header().Set("Content-Type", "application/json")
+	w.WriteHeader(http.StatusOK)
+	if err := json.NewEncoder(w).Encode(resp); err != nil {
+		if h.lg != nil {
+			h.lg.Warn("failed to encode members response", zap.Error(err))
+		} else {
+			plog.Warningf("failed to encode members response (%v)", err)
+		}
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/membership/cluster.go b/vendor/go.etcd.io/etcd/etcdserver/api/membership/cluster.go
new file mode 100644
index 0000000..89a6edd
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/membership/cluster.go
@@ -0,0 +1,841 @@
+// Copyright 2015 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 membership
+
+import (
+	"bytes"
+	"context"
+	"crypto/sha1"
+	"encoding/binary"
+	"encoding/json"
+	"fmt"
+	"path"
+	"sort"
+	"strings"
+	"sync"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver/api/v2store"
+	"go.etcd.io/etcd/mvcc/backend"
+	"go.etcd.io/etcd/pkg/netutil"
+	"go.etcd.io/etcd/pkg/types"
+	"go.etcd.io/etcd/raft"
+	"go.etcd.io/etcd/raft/raftpb"
+	"go.etcd.io/etcd/version"
+
+	"github.com/coreos/go-semver/semver"
+	"github.com/prometheus/client_golang/prometheus"
+	"go.uber.org/zap"
+)
+
+const maxLearners = 1
+
+// RaftCluster is a list of Members that belong to the same raft cluster
+type RaftCluster struct {
+	lg *zap.Logger
+
+	localID types.ID
+	cid     types.ID
+	token   string
+
+	v2store v2store.Store
+	be      backend.Backend
+
+	sync.Mutex // guards the fields below
+	version    *semver.Version
+	members    map[types.ID]*Member
+	// removed contains the ids of removed members in the cluster.
+	// removed id cannot be reused.
+	removed map[types.ID]bool
+}
+
+// ConfigChangeContext represents a context for confChange.
+type ConfigChangeContext struct {
+	Member
+	// IsPromote indicates if the config change is for promoting a learner member.
+	// This flag is needed because both adding a new member and promoting a learner member
+	// uses the same config change type 'ConfChangeAddNode'.
+	IsPromote bool `json:"isPromote"`
+}
+
+// NewClusterFromURLsMap creates a new raft cluster using provided urls map. Currently, it does not support creating
+// cluster with raft learner member.
+func NewClusterFromURLsMap(lg *zap.Logger, token string, urlsmap types.URLsMap) (*RaftCluster, error) {
+	c := NewCluster(lg, token)
+	for name, urls := range urlsmap {
+		m := NewMember(name, urls, token, nil)
+		if _, ok := c.members[m.ID]; ok {
+			return nil, fmt.Errorf("member exists with identical ID %v", m)
+		}
+		if uint64(m.ID) == raft.None {
+			return nil, fmt.Errorf("cannot use %x as member id", raft.None)
+		}
+		c.members[m.ID] = m
+	}
+	c.genID()
+	return c, nil
+}
+
+func NewClusterFromMembers(lg *zap.Logger, token string, id types.ID, membs []*Member) *RaftCluster {
+	c := NewCluster(lg, token)
+	c.cid = id
+	for _, m := range membs {
+		c.members[m.ID] = m
+	}
+	return c
+}
+
+func NewCluster(lg *zap.Logger, token string) *RaftCluster {
+	return &RaftCluster{
+		lg:      lg,
+		token:   token,
+		members: make(map[types.ID]*Member),
+		removed: make(map[types.ID]bool),
+	}
+}
+
+func (c *RaftCluster) ID() types.ID { return c.cid }
+
+func (c *RaftCluster) Members() []*Member {
+	c.Lock()
+	defer c.Unlock()
+	var ms MembersByID
+	for _, m := range c.members {
+		ms = append(ms, m.Clone())
+	}
+	sort.Sort(ms)
+	return []*Member(ms)
+}
+
+func (c *RaftCluster) Member(id types.ID) *Member {
+	c.Lock()
+	defer c.Unlock()
+	return c.members[id].Clone()
+}
+
+func (c *RaftCluster) VotingMembers() []*Member {
+	c.Lock()
+	defer c.Unlock()
+	var ms MembersByID
+	for _, m := range c.members {
+		if !m.IsLearner {
+			ms = append(ms, m.Clone())
+		}
+	}
+	sort.Sort(ms)
+	return []*Member(ms)
+}
+
+// MemberByName returns a Member with the given name if exists.
+// If more than one member has the given name, it will panic.
+func (c *RaftCluster) MemberByName(name string) *Member {
+	c.Lock()
+	defer c.Unlock()
+	var memb *Member
+	for _, m := range c.members {
+		if m.Name == name {
+			if memb != nil {
+				if c.lg != nil {
+					c.lg.Panic("two member with same name found", zap.String("name", name))
+				} else {
+					plog.Panicf("two members with the given name %q exist", name)
+				}
+			}
+			memb = m
+		}
+	}
+	return memb.Clone()
+}
+
+func (c *RaftCluster) MemberIDs() []types.ID {
+	c.Lock()
+	defer c.Unlock()
+	var ids []types.ID
+	for _, m := range c.members {
+		ids = append(ids, m.ID)
+	}
+	sort.Sort(types.IDSlice(ids))
+	return ids
+}
+
+func (c *RaftCluster) IsIDRemoved(id types.ID) bool {
+	c.Lock()
+	defer c.Unlock()
+	return c.removed[id]
+}
+
+// PeerURLs returns a list of all peer addresses.
+// The returned list is sorted in ascending lexicographical order.
+func (c *RaftCluster) PeerURLs() []string {
+	c.Lock()
+	defer c.Unlock()
+	urls := make([]string, 0)
+	for _, p := range c.members {
+		urls = append(urls, p.PeerURLs...)
+	}
+	sort.Strings(urls)
+	return urls
+}
+
+// ClientURLs returns a list of all client addresses.
+// The returned list is sorted in ascending lexicographical order.
+func (c *RaftCluster) ClientURLs() []string {
+	c.Lock()
+	defer c.Unlock()
+	urls := make([]string, 0)
+	for _, p := range c.members {
+		urls = append(urls, p.ClientURLs...)
+	}
+	sort.Strings(urls)
+	return urls
+}
+
+func (c *RaftCluster) String() string {
+	c.Lock()
+	defer c.Unlock()
+	b := &bytes.Buffer{}
+	fmt.Fprintf(b, "{ClusterID:%s ", c.cid)
+	var ms []string
+	for _, m := range c.members {
+		ms = append(ms, fmt.Sprintf("%+v", m))
+	}
+	fmt.Fprintf(b, "Members:[%s] ", strings.Join(ms, " "))
+	var ids []string
+	for id := range c.removed {
+		ids = append(ids, id.String())
+	}
+	fmt.Fprintf(b, "RemovedMemberIDs:[%s]}", strings.Join(ids, " "))
+	return b.String()
+}
+
+func (c *RaftCluster) genID() {
+	mIDs := c.MemberIDs()
+	b := make([]byte, 8*len(mIDs))
+	for i, id := range mIDs {
+		binary.BigEndian.PutUint64(b[8*i:], uint64(id))
+	}
+	hash := sha1.Sum(b)
+	c.cid = types.ID(binary.BigEndian.Uint64(hash[:8]))
+}
+
+func (c *RaftCluster) SetID(localID, cid types.ID) {
+	c.localID = localID
+	c.cid = cid
+}
+
+func (c *RaftCluster) SetStore(st v2store.Store) { c.v2store = st }
+
+func (c *RaftCluster) SetBackend(be backend.Backend) {
+	c.be = be
+	mustCreateBackendBuckets(c.be)
+}
+
+func (c *RaftCluster) Recover(onSet func(*zap.Logger, *semver.Version)) {
+	c.Lock()
+	defer c.Unlock()
+
+	c.members, c.removed = membersFromStore(c.lg, c.v2store)
+	c.version = clusterVersionFromStore(c.lg, c.v2store)
+	mustDetectDowngrade(c.lg, c.version)
+	onSet(c.lg, c.version)
+
+	for _, m := range c.members {
+		if c.lg != nil {
+			c.lg.Info(
+				"recovered/added member from store",
+				zap.String("cluster-id", c.cid.String()),
+				zap.String("local-member-id", c.localID.String()),
+				zap.String("recovered-remote-peer-id", m.ID.String()),
+				zap.Strings("recovered-remote-peer-urls", m.PeerURLs),
+			)
+		} else {
+			plog.Infof("added member %s %v to cluster %s from store", m.ID, m.PeerURLs, c.cid)
+		}
+	}
+	if c.version != nil {
+		if c.lg != nil {
+			c.lg.Info(
+				"set cluster version from store",
+				zap.String("cluster-version", version.Cluster(c.version.String())),
+			)
+		} else {
+			plog.Infof("set the cluster version to %v from store", version.Cluster(c.version.String()))
+		}
+	}
+}
+
+// ValidateConfigurationChange takes a proposed ConfChange and
+// ensures that it is still valid.
+func (c *RaftCluster) ValidateConfigurationChange(cc raftpb.ConfChange) error {
+	members, removed := membersFromStore(c.lg, c.v2store)
+	id := types.ID(cc.NodeID)
+	if removed[id] {
+		return ErrIDRemoved
+	}
+	switch cc.Type {
+	case raftpb.ConfChangeAddNode, raftpb.ConfChangeAddLearnerNode:
+		confChangeContext := new(ConfigChangeContext)
+		if err := json.Unmarshal(cc.Context, confChangeContext); err != nil {
+			if c.lg != nil {
+				c.lg.Panic("failed to unmarshal confChangeContext", zap.Error(err))
+			} else {
+				plog.Panicf("unmarshal confChangeContext should never fail: %v", err)
+			}
+		}
+
+		if confChangeContext.IsPromote { // promoting a learner member to voting member
+			if members[id] == nil {
+				return ErrIDNotFound
+			}
+			if !members[id].IsLearner {
+				return ErrMemberNotLearner
+			}
+		} else { // adding a new member
+			if members[id] != nil {
+				return ErrIDExists
+			}
+
+			urls := make(map[string]bool)
+			for _, m := range members {
+				for _, u := range m.PeerURLs {
+					urls[u] = true
+				}
+			}
+			for _, u := range confChangeContext.Member.PeerURLs {
+				if urls[u] {
+					return ErrPeerURLexists
+				}
+			}
+
+			if confChangeContext.Member.IsLearner { // the new member is a learner
+				numLearners := 0
+				for _, m := range members {
+					if m.IsLearner {
+						numLearners++
+					}
+				}
+				if numLearners+1 > maxLearners {
+					return ErrTooManyLearners
+				}
+			}
+		}
+	case raftpb.ConfChangeRemoveNode:
+		if members[id] == nil {
+			return ErrIDNotFound
+		}
+
+	case raftpb.ConfChangeUpdateNode:
+		if members[id] == nil {
+			return ErrIDNotFound
+		}
+		urls := make(map[string]bool)
+		for _, m := range members {
+			if m.ID == id {
+				continue
+			}
+			for _, u := range m.PeerURLs {
+				urls[u] = true
+			}
+		}
+		m := new(Member)
+		if err := json.Unmarshal(cc.Context, m); err != nil {
+			if c.lg != nil {
+				c.lg.Panic("failed to unmarshal member", zap.Error(err))
+			} else {
+				plog.Panicf("unmarshal member should never fail: %v", err)
+			}
+		}
+		for _, u := range m.PeerURLs {
+			if urls[u] {
+				return ErrPeerURLexists
+			}
+		}
+
+	default:
+		if c.lg != nil {
+			c.lg.Panic("unknown ConfChange type", zap.String("type", cc.Type.String()))
+		} else {
+			plog.Panicf("ConfChange type should be either AddNode, RemoveNode or UpdateNode")
+		}
+	}
+	return nil
+}
+
+// AddMember adds a new Member into the cluster, and saves the given member's
+// raftAttributes into the store. The given member should have empty attributes.
+// A Member with a matching id must not exist.
+func (c *RaftCluster) AddMember(m *Member) {
+	c.Lock()
+	defer c.Unlock()
+	if c.v2store != nil {
+		mustSaveMemberToStore(c.v2store, m)
+	}
+	if c.be != nil {
+		mustSaveMemberToBackend(c.be, m)
+	}
+
+	c.members[m.ID] = m
+
+	if c.lg != nil {
+		c.lg.Info(
+			"added member",
+			zap.String("cluster-id", c.cid.String()),
+			zap.String("local-member-id", c.localID.String()),
+			zap.String("added-peer-id", m.ID.String()),
+			zap.Strings("added-peer-peer-urls", m.PeerURLs),
+		)
+	} else {
+		plog.Infof("added member %s %v to cluster %s", m.ID, m.PeerURLs, c.cid)
+	}
+}
+
+// RemoveMember removes a member from the store.
+// The given id MUST exist, or the function panics.
+func (c *RaftCluster) RemoveMember(id types.ID) {
+	c.Lock()
+	defer c.Unlock()
+	if c.v2store != nil {
+		mustDeleteMemberFromStore(c.v2store, id)
+	}
+	if c.be != nil {
+		mustDeleteMemberFromBackend(c.be, id)
+	}
+
+	m, ok := c.members[id]
+	delete(c.members, id)
+	c.removed[id] = true
+
+	if c.lg != nil {
+		if ok {
+			c.lg.Info(
+				"removed member",
+				zap.String("cluster-id", c.cid.String()),
+				zap.String("local-member-id", c.localID.String()),
+				zap.String("removed-remote-peer-id", id.String()),
+				zap.Strings("removed-remote-peer-urls", m.PeerURLs),
+			)
+		} else {
+			c.lg.Warn(
+				"skipped removing already removed member",
+				zap.String("cluster-id", c.cid.String()),
+				zap.String("local-member-id", c.localID.String()),
+				zap.String("removed-remote-peer-id", id.String()),
+			)
+		}
+	} else {
+		plog.Infof("removed member %s from cluster %s", id, c.cid)
+	}
+}
+
+func (c *RaftCluster) UpdateAttributes(id types.ID, attr Attributes) {
+	c.Lock()
+	defer c.Unlock()
+
+	if m, ok := c.members[id]; ok {
+		m.Attributes = attr
+		if c.v2store != nil {
+			mustUpdateMemberAttrInStore(c.v2store, m)
+		}
+		if c.be != nil {
+			mustSaveMemberToBackend(c.be, m)
+		}
+		return
+	}
+
+	_, ok := c.removed[id]
+	if !ok {
+		if c.lg != nil {
+			c.lg.Panic(
+				"failed to update; member unknown",
+				zap.String("cluster-id", c.cid.String()),
+				zap.String("local-member-id", c.localID.String()),
+				zap.String("unknown-remote-peer-id", id.String()),
+			)
+		} else {
+			plog.Panicf("error updating attributes of unknown member %s", id)
+		}
+	}
+
+	if c.lg != nil {
+		c.lg.Warn(
+			"skipped attributes update of removed member",
+			zap.String("cluster-id", c.cid.String()),
+			zap.String("local-member-id", c.localID.String()),
+			zap.String("updated-peer-id", id.String()),
+		)
+	} else {
+		plog.Warningf("skipped updating attributes of removed member %s", id)
+	}
+}
+
+// PromoteMember marks the member's IsLearner RaftAttributes to false.
+func (c *RaftCluster) PromoteMember(id types.ID) {
+	c.Lock()
+	defer c.Unlock()
+
+	c.members[id].RaftAttributes.IsLearner = false
+	if c.v2store != nil {
+		mustUpdateMemberInStore(c.v2store, c.members[id])
+	}
+	if c.be != nil {
+		mustSaveMemberToBackend(c.be, c.members[id])
+	}
+
+	if c.lg != nil {
+		c.lg.Info(
+			"promote member",
+			zap.String("cluster-id", c.cid.String()),
+			zap.String("local-member-id", c.localID.String()),
+		)
+	} else {
+		plog.Noticef("promote member %s in cluster %s", id, c.cid)
+	}
+}
+
+func (c *RaftCluster) UpdateRaftAttributes(id types.ID, raftAttr RaftAttributes) {
+	c.Lock()
+	defer c.Unlock()
+
+	c.members[id].RaftAttributes = raftAttr
+	if c.v2store != nil {
+		mustUpdateMemberInStore(c.v2store, c.members[id])
+	}
+	if c.be != nil {
+		mustSaveMemberToBackend(c.be, c.members[id])
+	}
+
+	if c.lg != nil {
+		c.lg.Info(
+			"updated member",
+			zap.String("cluster-id", c.cid.String()),
+			zap.String("local-member-id", c.localID.String()),
+			zap.String("updated-remote-peer-id", id.String()),
+			zap.Strings("updated-remote-peer-urls", raftAttr.PeerURLs),
+		)
+	} else {
+		plog.Noticef("updated member %s %v in cluster %s", id, raftAttr.PeerURLs, c.cid)
+	}
+}
+
+func (c *RaftCluster) Version() *semver.Version {
+	c.Lock()
+	defer c.Unlock()
+	if c.version == nil {
+		return nil
+	}
+	return semver.Must(semver.NewVersion(c.version.String()))
+}
+
+func (c *RaftCluster) SetVersion(ver *semver.Version, onSet func(*zap.Logger, *semver.Version)) {
+	c.Lock()
+	defer c.Unlock()
+	if c.version != nil {
+		if c.lg != nil {
+			c.lg.Info(
+				"updated cluster version",
+				zap.String("cluster-id", c.cid.String()),
+				zap.String("local-member-id", c.localID.String()),
+				zap.String("from", version.Cluster(c.version.String())),
+				zap.String("from", version.Cluster(ver.String())),
+			)
+		} else {
+			plog.Noticef("updated the cluster version from %v to %v", version.Cluster(c.version.String()), version.Cluster(ver.String()))
+		}
+	} else {
+		if c.lg != nil {
+			c.lg.Info(
+				"set initial cluster version",
+				zap.String("cluster-id", c.cid.String()),
+				zap.String("local-member-id", c.localID.String()),
+				zap.String("cluster-version", version.Cluster(ver.String())),
+			)
+		} else {
+			plog.Noticef("set the initial cluster version to %v", version.Cluster(ver.String()))
+		}
+	}
+	c.version = ver
+	mustDetectDowngrade(c.lg, c.version)
+	if c.v2store != nil {
+		mustSaveClusterVersionToStore(c.v2store, ver)
+	}
+	if c.be != nil {
+		mustSaveClusterVersionToBackend(c.be, ver)
+	}
+	ClusterVersionMetrics.With(prometheus.Labels{"cluster_version": ver.String()}).Set(1)
+	onSet(c.lg, ver)
+}
+
+func (c *RaftCluster) IsReadyToAddVotingMember() bool {
+	nmembers := 1
+	nstarted := 0
+
+	for _, member := range c.VotingMembers() {
+		if member.IsStarted() {
+			nstarted++
+		}
+		nmembers++
+	}
+
+	if nstarted == 1 && nmembers == 2 {
+		// a case of adding a new node to 1-member cluster for restoring cluster data
+		// https://github.com/etcd-io/etcd/blob/master/Documentation/v2/admin_guide.md#restoring-the-cluster
+		if c.lg != nil {
+			c.lg.Debug("number of started member is 1; can accept add member request")
+		} else {
+			plog.Debugf("The number of started member is 1. This cluster can accept add member request.")
+		}
+		return true
+	}
+
+	nquorum := nmembers/2 + 1
+	if nstarted < nquorum {
+		if c.lg != nil {
+			c.lg.Warn(
+				"rejecting member add; started member will be less than quorum",
+				zap.Int("number-of-started-member", nstarted),
+				zap.Int("quorum", nquorum),
+				zap.String("cluster-id", c.cid.String()),
+				zap.String("local-member-id", c.localID.String()),
+			)
+		} else {
+			plog.Warningf("Reject add member request: the number of started member (%d) will be less than the quorum number of the cluster (%d)", nstarted, nquorum)
+		}
+		return false
+	}
+
+	return true
+}
+
+func (c *RaftCluster) IsReadyToRemoveVotingMember(id uint64) bool {
+	nmembers := 0
+	nstarted := 0
+
+	for _, member := range c.VotingMembers() {
+		if uint64(member.ID) == id {
+			continue
+		}
+
+		if member.IsStarted() {
+			nstarted++
+		}
+		nmembers++
+	}
+
+	nquorum := nmembers/2 + 1
+	if nstarted < nquorum {
+		if c.lg != nil {
+			c.lg.Warn(
+				"rejecting member remove; started member will be less than quorum",
+				zap.Int("number-of-started-member", nstarted),
+				zap.Int("quorum", nquorum),
+				zap.String("cluster-id", c.cid.String()),
+				zap.String("local-member-id", c.localID.String()),
+			)
+		} else {
+			plog.Warningf("Reject remove member request: the number of started member (%d) will be less than the quorum number of the cluster (%d)", nstarted, nquorum)
+		}
+		return false
+	}
+
+	return true
+}
+
+func (c *RaftCluster) IsReadyToPromoteMember(id uint64) bool {
+	nmembers := 1
+	nstarted := 0
+
+	for _, member := range c.VotingMembers() {
+		if member.IsStarted() {
+			nstarted++
+		}
+		nmembers++
+	}
+
+	nquorum := nmembers/2 + 1
+	if nstarted < nquorum {
+		if c.lg != nil {
+			c.lg.Warn(
+				"rejecting member promote; started member will be less than quorum",
+				zap.Int("number-of-started-member", nstarted),
+				zap.Int("quorum", nquorum),
+				zap.String("cluster-id", c.cid.String()),
+				zap.String("local-member-id", c.localID.String()),
+			)
+		} else {
+			plog.Warningf("Reject promote member request: the number of started member (%d) will be less than the quorum number of the cluster (%d)", nstarted, nquorum)
+		}
+		return false
+	}
+
+	return true
+}
+
+func membersFromStore(lg *zap.Logger, st v2store.Store) (map[types.ID]*Member, map[types.ID]bool) {
+	members := make(map[types.ID]*Member)
+	removed := make(map[types.ID]bool)
+	e, err := st.Get(StoreMembersPrefix, true, true)
+	if err != nil {
+		if isKeyNotFound(err) {
+			return members, removed
+		}
+		if lg != nil {
+			lg.Panic("failed to get members from store", zap.String("path", StoreMembersPrefix), zap.Error(err))
+		} else {
+			plog.Panicf("get storeMembers should never fail: %v", err)
+		}
+	}
+	for _, n := range e.Node.Nodes {
+		var m *Member
+		m, err = nodeToMember(n)
+		if err != nil {
+			if lg != nil {
+				lg.Panic("failed to nodeToMember", zap.Error(err))
+			} else {
+				plog.Panicf("nodeToMember should never fail: %v", err)
+			}
+		}
+		members[m.ID] = m
+	}
+
+	e, err = st.Get(storeRemovedMembersPrefix, true, true)
+	if err != nil {
+		if isKeyNotFound(err) {
+			return members, removed
+		}
+		if lg != nil {
+			lg.Panic(
+				"failed to get removed members from store",
+				zap.String("path", storeRemovedMembersPrefix),
+				zap.Error(err),
+			)
+		} else {
+			plog.Panicf("get storeRemovedMembers should never fail: %v", err)
+		}
+	}
+	for _, n := range e.Node.Nodes {
+		removed[MustParseMemberIDFromKey(n.Key)] = true
+	}
+	return members, removed
+}
+
+func clusterVersionFromStore(lg *zap.Logger, st v2store.Store) *semver.Version {
+	e, err := st.Get(path.Join(storePrefix, "version"), false, false)
+	if err != nil {
+		if isKeyNotFound(err) {
+			return nil
+		}
+		if lg != nil {
+			lg.Panic(
+				"failed to get cluster version from store",
+				zap.String("path", path.Join(storePrefix, "version")),
+				zap.Error(err),
+			)
+		} else {
+			plog.Panicf("unexpected error (%v) when getting cluster version from store", err)
+		}
+	}
+	return semver.Must(semver.NewVersion(*e.Node.Value))
+}
+
+// ValidateClusterAndAssignIDs validates the local cluster by matching the PeerURLs
+// with the existing cluster. If the validation succeeds, it assigns the IDs
+// from the existing cluster to the local cluster.
+// If the validation fails, an error will be returned.
+func ValidateClusterAndAssignIDs(lg *zap.Logger, local *RaftCluster, existing *RaftCluster) error {
+	ems := existing.Members()
+	lms := local.Members()
+	if len(ems) != len(lms) {
+		return fmt.Errorf("member count is unequal")
+	}
+
+	ctx, cancel := context.WithTimeout(context.TODO(), 30*time.Second)
+	defer cancel()
+	for i := range ems {
+		var err error
+		ok := false
+		for j := range lms {
+			if ok, err = netutil.URLStringsEqual(ctx, lg, ems[i].PeerURLs, lms[j].PeerURLs); ok {
+				lms[j].ID = ems[i].ID
+				break
+			}
+		}
+		if !ok {
+			return fmt.Errorf("PeerURLs: no match found for existing member (%v, %v), last resolver error (%v)", ems[i].ID, ems[i].PeerURLs, err)
+		}
+	}
+	local.members = make(map[types.ID]*Member)
+	for _, m := range lms {
+		local.members[m.ID] = m
+	}
+	return nil
+}
+
+func mustDetectDowngrade(lg *zap.Logger, cv *semver.Version) {
+	lv := semver.Must(semver.NewVersion(version.Version))
+	// only keep major.minor version for comparison against cluster version
+	lv = &semver.Version{Major: lv.Major, Minor: lv.Minor}
+	if cv != nil && lv.LessThan(*cv) {
+		if lg != nil {
+			lg.Fatal(
+				"invalid downgrade; server version is lower than determined cluster version",
+				zap.String("current-server-version", version.Version),
+				zap.String("determined-cluster-version", version.Cluster(cv.String())),
+			)
+		} else {
+			plog.Fatalf("cluster cannot be downgraded (current version: %s is lower than determined cluster version: %s).", version.Version, version.Cluster(cv.String()))
+		}
+	}
+}
+
+// IsLocalMemberLearner returns if the local member is raft learner
+func (c *RaftCluster) IsLocalMemberLearner() bool {
+	c.Lock()
+	defer c.Unlock()
+	localMember, ok := c.members[c.localID]
+	if !ok {
+		if c.lg != nil {
+			c.lg.Panic(
+				"failed to find local ID in cluster members",
+				zap.String("cluster-id", c.cid.String()),
+				zap.String("local-member-id", c.localID.String()),
+			)
+		} else {
+			plog.Panicf("failed to find local ID %s in cluster %s", c.localID.String(), c.cid.String())
+		}
+	}
+	return localMember.IsLearner
+}
+
+// IsMemberExist returns if the member with the given id exists in cluster.
+func (c *RaftCluster) IsMemberExist(id types.ID) bool {
+	c.Lock()
+	defer c.Unlock()
+	_, ok := c.members[id]
+	return ok
+}
+
+// VotingMemberIDs returns the ID of voting members in cluster.
+func (c *RaftCluster) VotingMemberIDs() []types.ID {
+	c.Lock()
+	defer c.Unlock()
+	var ids []types.ID
+	for _, m := range c.members {
+		if !m.IsLearner {
+			ids = append(ids, m.ID)
+		}
+	}
+	sort.Sort(types.IDSlice(ids))
+	return ids
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/membership/doc.go b/vendor/go.etcd.io/etcd/etcdserver/api/membership/doc.go
new file mode 100644
index 0000000..b07fb2d
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/membership/doc.go
@@ -0,0 +1,16 @@
+// 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 membership describes individual etcd members and clusters of members.
+package membership
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/membership/errors.go b/vendor/go.etcd.io/etcd/etcdserver/api/membership/errors.go
new file mode 100644
index 0000000..8f6fe50
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/membership/errors.go
@@ -0,0 +1,35 @@
+// Copyright 2016 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 membership
+
+import (
+	"errors"
+
+	"go.etcd.io/etcd/etcdserver/api/v2error"
+)
+
+var (
+	ErrIDRemoved        = errors.New("membership: ID removed")
+	ErrIDExists         = errors.New("membership: ID exists")
+	ErrIDNotFound       = errors.New("membership: ID not found")
+	ErrPeerURLexists    = errors.New("membership: peerURL exists")
+	ErrMemberNotLearner = errors.New("membership: can only promote a learner member")
+	ErrTooManyLearners  = errors.New("membership: too many learner members in cluster")
+)
+
+func isKeyNotFound(err error) bool {
+	e, ok := err.(*v2error.Error)
+	return ok && e.ErrorCode == v2error.EcodeKeyNotFound
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/membership/member.go b/vendor/go.etcd.io/etcd/etcdserver/api/membership/member.go
new file mode 100644
index 0000000..896cb36
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/membership/member.go
@@ -0,0 +1,142 @@
+// Copyright 2015 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 membership
+
+import (
+	"crypto/sha1"
+	"encoding/binary"
+	"fmt"
+	"math/rand"
+	"sort"
+	"time"
+
+	"github.com/coreos/pkg/capnslog"
+	"go.etcd.io/etcd/pkg/types"
+)
+
+var (
+	plog = capnslog.NewPackageLogger("go.etcd.io/etcd/v3", "etcdserver/membership")
+)
+
+// RaftAttributes represents the raft related attributes of an etcd member.
+type RaftAttributes struct {
+	// PeerURLs is the list of peers in the raft cluster.
+	// TODO(philips): ensure these are URLs
+	PeerURLs []string `json:"peerURLs"`
+	// IsLearner indicates if the member is raft learner.
+	IsLearner bool `json:"isLearner,omitempty"`
+}
+
+// Attributes represents all the non-raft related attributes of an etcd member.
+type Attributes struct {
+	Name       string   `json:"name,omitempty"`
+	ClientURLs []string `json:"clientURLs,omitempty"`
+}
+
+type Member struct {
+	ID types.ID `json:"id"`
+	RaftAttributes
+	Attributes
+}
+
+// NewMember creates a Member without an ID and generates one based on the
+// cluster name, peer URLs, and time. This is used for bootstrapping/adding new member.
+func NewMember(name string, peerURLs types.URLs, clusterName string, now *time.Time) *Member {
+	return newMember(name, peerURLs, clusterName, now, false)
+}
+
+// NewMemberAsLearner creates a learner Member without an ID and generates one based on the
+// cluster name, peer URLs, and time. This is used for adding new learner member.
+func NewMemberAsLearner(name string, peerURLs types.URLs, clusterName string, now *time.Time) *Member {
+	return newMember(name, peerURLs, clusterName, now, true)
+}
+
+func newMember(name string, peerURLs types.URLs, clusterName string, now *time.Time, isLearner bool) *Member {
+	m := &Member{
+		RaftAttributes: RaftAttributes{
+			PeerURLs:  peerURLs.StringSlice(),
+			IsLearner: isLearner,
+		},
+		Attributes: Attributes{Name: name},
+	}
+
+	var b []byte
+	sort.Strings(m.PeerURLs)
+	for _, p := range m.PeerURLs {
+		b = append(b, []byte(p)...)
+	}
+
+	b = append(b, []byte(clusterName)...)
+	if now != nil {
+		b = append(b, []byte(fmt.Sprintf("%d", now.Unix()))...)
+	}
+
+	hash := sha1.Sum(b)
+	m.ID = types.ID(binary.BigEndian.Uint64(hash[:8]))
+	return m
+}
+
+// PickPeerURL chooses a random address from a given Member's PeerURLs.
+// It will panic if there is no PeerURLs available in Member.
+func (m *Member) PickPeerURL() string {
+	if len(m.PeerURLs) == 0 {
+		panic("member should always have some peer url")
+	}
+	return m.PeerURLs[rand.Intn(len(m.PeerURLs))]
+}
+
+func (m *Member) Clone() *Member {
+	if m == nil {
+		return nil
+	}
+	mm := &Member{
+		ID: m.ID,
+		RaftAttributes: RaftAttributes{
+			IsLearner: m.IsLearner,
+		},
+		Attributes: Attributes{
+			Name: m.Name,
+		},
+	}
+	if m.PeerURLs != nil {
+		mm.PeerURLs = make([]string, len(m.PeerURLs))
+		copy(mm.PeerURLs, m.PeerURLs)
+	}
+	if m.ClientURLs != nil {
+		mm.ClientURLs = make([]string, len(m.ClientURLs))
+		copy(mm.ClientURLs, m.ClientURLs)
+	}
+	return mm
+}
+
+func (m *Member) IsStarted() bool {
+	return len(m.Name) != 0
+}
+
+// MembersByID implements sort by ID interface
+type MembersByID []*Member
+
+func (ms MembersByID) Len() int           { return len(ms) }
+func (ms MembersByID) Less(i, j int) bool { return ms[i].ID < ms[j].ID }
+func (ms MembersByID) Swap(i, j int)      { ms[i], ms[j] = ms[j], ms[i] }
+
+// MembersByPeerURLs implements sort by peer urls interface
+type MembersByPeerURLs []*Member
+
+func (ms MembersByPeerURLs) Len() int { return len(ms) }
+func (ms MembersByPeerURLs) Less(i, j int) bool {
+	return ms[i].PeerURLs[0] < ms[j].PeerURLs[0]
+}
+func (ms MembersByPeerURLs) Swap(i, j int) { ms[i], ms[j] = ms[j], ms[i] }
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/membership/metrics.go b/vendor/go.etcd.io/etcd/etcdserver/api/membership/metrics.go
new file mode 100644
index 0000000..b3212bc
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/membership/metrics.go
@@ -0,0 +1,31 @@
+// Copyright 2018 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 membership
+
+import "github.com/prometheus/client_golang/prometheus"
+
+var (
+	ClusterVersionMetrics = prometheus.NewGaugeVec(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "cluster",
+		Name:      "version",
+		Help:      "Which version is running. 1 for 'cluster_version' label with current cluster version",
+	},
+		[]string{"cluster_version"})
+)
+
+func init() {
+	prometheus.MustRegister(ClusterVersionMetrics)
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/membership/store.go b/vendor/go.etcd.io/etcd/etcdserver/api/membership/store.go
new file mode 100644
index 0000000..14ab119
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/membership/store.go
@@ -0,0 +1,193 @@
+// Copyright 2016 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 membership
+
+import (
+	"encoding/json"
+	"fmt"
+	"path"
+
+	"go.etcd.io/etcd/etcdserver/api/v2store"
+	"go.etcd.io/etcd/mvcc/backend"
+	"go.etcd.io/etcd/pkg/types"
+
+	"github.com/coreos/go-semver/semver"
+)
+
+const (
+	attributesSuffix     = "attributes"
+	raftAttributesSuffix = "raftAttributes"
+
+	// the prefix for stroing membership related information in store provided by store pkg.
+	storePrefix = "/0"
+)
+
+var (
+	membersBucketName        = []byte("members")
+	membersRemovedBucketName = []byte("members_removed")
+	clusterBucketName        = []byte("cluster")
+
+	StoreMembersPrefix        = path.Join(storePrefix, "members")
+	storeRemovedMembersPrefix = path.Join(storePrefix, "removed_members")
+)
+
+func mustSaveMemberToBackend(be backend.Backend, m *Member) {
+	mkey := backendMemberKey(m.ID)
+	mvalue, err := json.Marshal(m)
+	if err != nil {
+		plog.Panicf("marshal raftAttributes should never fail: %v", err)
+	}
+
+	tx := be.BatchTx()
+	tx.Lock()
+	tx.UnsafePut(membersBucketName, mkey, mvalue)
+	tx.Unlock()
+}
+
+func mustDeleteMemberFromBackend(be backend.Backend, id types.ID) {
+	mkey := backendMemberKey(id)
+
+	tx := be.BatchTx()
+	tx.Lock()
+	tx.UnsafeDelete(membersBucketName, mkey)
+	tx.UnsafePut(membersRemovedBucketName, mkey, []byte("removed"))
+	tx.Unlock()
+}
+
+func mustSaveClusterVersionToBackend(be backend.Backend, ver *semver.Version) {
+	ckey := backendClusterVersionKey()
+
+	tx := be.BatchTx()
+	tx.Lock()
+	defer tx.Unlock()
+	tx.UnsafePut(clusterBucketName, ckey, []byte(ver.String()))
+}
+
+func mustSaveMemberToStore(s v2store.Store, m *Member) {
+	b, err := json.Marshal(m.RaftAttributes)
+	if err != nil {
+		plog.Panicf("marshal raftAttributes should never fail: %v", err)
+	}
+	p := path.Join(MemberStoreKey(m.ID), raftAttributesSuffix)
+	if _, err := s.Create(p, false, string(b), false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent}); err != nil {
+		plog.Panicf("create raftAttributes should never fail: %v", err)
+	}
+}
+
+func mustDeleteMemberFromStore(s v2store.Store, id types.ID) {
+	if _, err := s.Delete(MemberStoreKey(id), true, true); err != nil {
+		plog.Panicf("delete member should never fail: %v", err)
+	}
+	if _, err := s.Create(RemovedMemberStoreKey(id), false, "", false, v2store.TTLOptionSet{ExpireTime: v2store.Permanent}); err != nil {
+		plog.Panicf("create removedMember should never fail: %v", err)
+	}
+}
+
+func mustUpdateMemberInStore(s v2store.Store, m *Member) {
+	b, err := json.Marshal(m.RaftAttributes)
+	if err != nil {
+		plog.Panicf("marshal raftAttributes should never fail: %v", err)
+	}
+	p := path.Join(MemberStoreKey(m.ID), raftAttributesSuffix)
+	if _, err := s.Update(p, string(b), v2store.TTLOptionSet{ExpireTime: v2store.Permanent}); err != nil {
+		plog.Panicf("update raftAttributes should never fail: %v", err)
+	}
+}
+
+func mustUpdateMemberAttrInStore(s v2store.Store, m *Member) {
+	b, err := json.Marshal(m.Attributes)
+	if err != nil {
+		plog.Panicf("marshal raftAttributes should never fail: %v", err)
+	}
+	p := path.Join(MemberStoreKey(m.ID), attributesSuffix)
+	if _, err := s.Set(p, false, string(b), v2store.TTLOptionSet{ExpireTime: v2store.Permanent}); err != nil {
+		plog.Panicf("update raftAttributes should never fail: %v", err)
+	}
+}
+
+func mustSaveClusterVersionToStore(s v2store.Store, ver *semver.Version) {
+	if _, err := s.Set(StoreClusterVersionKey(), false, ver.String(), v2store.TTLOptionSet{ExpireTime: v2store.Permanent}); err != nil {
+		plog.Panicf("save cluster version should never fail: %v", err)
+	}
+}
+
+// nodeToMember builds member from a key value node.
+// the child nodes of the given node MUST be sorted by key.
+func nodeToMember(n *v2store.NodeExtern) (*Member, error) {
+	m := &Member{ID: MustParseMemberIDFromKey(n.Key)}
+	attrs := make(map[string][]byte)
+	raftAttrKey := path.Join(n.Key, raftAttributesSuffix)
+	attrKey := path.Join(n.Key, attributesSuffix)
+	for _, nn := range n.Nodes {
+		if nn.Key != raftAttrKey && nn.Key != attrKey {
+			return nil, fmt.Errorf("unknown key %q", nn.Key)
+		}
+		attrs[nn.Key] = []byte(*nn.Value)
+	}
+	if data := attrs[raftAttrKey]; data != nil {
+		if err := json.Unmarshal(data, &m.RaftAttributes); err != nil {
+			return nil, fmt.Errorf("unmarshal raftAttributes error: %v", err)
+		}
+	} else {
+		return nil, fmt.Errorf("raftAttributes key doesn't exist")
+	}
+	if data := attrs[attrKey]; data != nil {
+		if err := json.Unmarshal(data, &m.Attributes); err != nil {
+			return m, fmt.Errorf("unmarshal attributes error: %v", err)
+		}
+	}
+	return m, nil
+}
+
+func backendMemberKey(id types.ID) []byte {
+	return []byte(id.String())
+}
+
+func backendClusterVersionKey() []byte {
+	return []byte("clusterVersion")
+}
+
+func mustCreateBackendBuckets(be backend.Backend) {
+	tx := be.BatchTx()
+	tx.Lock()
+	defer tx.Unlock()
+	tx.UnsafeCreateBucket(membersBucketName)
+	tx.UnsafeCreateBucket(membersRemovedBucketName)
+	tx.UnsafeCreateBucket(clusterBucketName)
+}
+
+func MemberStoreKey(id types.ID) string {
+	return path.Join(StoreMembersPrefix, id.String())
+}
+
+func StoreClusterVersionKey() string {
+	return path.Join(storePrefix, "version")
+}
+
+func MemberAttributesStorePath(id types.ID) string {
+	return path.Join(MemberStoreKey(id), attributesSuffix)
+}
+
+func MustParseMemberIDFromKey(key string) types.ID {
+	id, err := types.IDFromString(path.Base(key))
+	if err != nil {
+		plog.Panicf("unexpected parse member id error: %v", err)
+	}
+	return id
+}
+
+func RemovedMemberStoreKey(id types.ID) string {
+	return path.Join(storeRemovedMembersPrefix, id.String())
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/coder.go b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/coder.go
new file mode 100644
index 0000000..12c3e44
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/coder.go
@@ -0,0 +1,27 @@
+// Copyright 2015 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 rafthttp
+
+import "go.etcd.io/etcd/raft/raftpb"
+
+type encoder interface {
+	// encode encodes the given message to an output stream.
+	encode(m *raftpb.Message) error
+}
+
+type decoder interface {
+	// decode decodes the message from an input stream.
+	decode() (raftpb.Message, error)
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/doc.go b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/doc.go
new file mode 100644
index 0000000..a9486a8
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/doc.go
@@ -0,0 +1,16 @@
+// Copyright 2015 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 rafthttp implements HTTP transportation layer for etcd/raft pkg.
+package rafthttp
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/http.go b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/http.go
new file mode 100644
index 0000000..d0e0c81
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/http.go
@@ -0,0 +1,577 @@
+// Copyright 2015 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 rafthttp
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"io/ioutil"
+	"net/http"
+	"path"
+	"strings"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver/api/snap"
+	pioutil "go.etcd.io/etcd/pkg/ioutil"
+	"go.etcd.io/etcd/pkg/types"
+	"go.etcd.io/etcd/raft/raftpb"
+	"go.etcd.io/etcd/version"
+
+	humanize "github.com/dustin/go-humanize"
+	"go.uber.org/zap"
+)
+
+const (
+	// connReadLimitByte limits the number of bytes
+	// a single read can read out.
+	//
+	// 64KB should be large enough for not causing
+	// throughput bottleneck as well as small enough
+	// for not causing a read timeout.
+	connReadLimitByte = 64 * 1024
+)
+
+var (
+	RaftPrefix         = "/raft"
+	ProbingPrefix      = path.Join(RaftPrefix, "probing")
+	RaftStreamPrefix   = path.Join(RaftPrefix, "stream")
+	RaftSnapshotPrefix = path.Join(RaftPrefix, "snapshot")
+
+	errIncompatibleVersion = errors.New("incompatible version")
+	errClusterIDMismatch   = errors.New("cluster ID mismatch")
+)
+
+type peerGetter interface {
+	Get(id types.ID) Peer
+}
+
+type writerToResponse interface {
+	WriteTo(w http.ResponseWriter)
+}
+
+type pipelineHandler struct {
+	lg      *zap.Logger
+	localID types.ID
+	tr      Transporter
+	r       Raft
+	cid     types.ID
+}
+
+// newPipelineHandler returns a handler for handling raft messages
+// from pipeline for RaftPrefix.
+//
+// The handler reads out the raft message from request body,
+// and forwards it to the given raft state machine for processing.
+func newPipelineHandler(t *Transport, r Raft, cid types.ID) http.Handler {
+	return &pipelineHandler{
+		lg:      t.Logger,
+		localID: t.ID,
+		tr:      t,
+		r:       r,
+		cid:     cid,
+	}
+}
+
+func (h *pipelineHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
+	if r.Method != "POST" {
+		w.Header().Set("Allow", "POST")
+		http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed)
+		return
+	}
+
+	w.Header().Set("X-Etcd-Cluster-ID", h.cid.String())
+
+	if err := checkClusterCompatibilityFromHeader(h.lg, h.localID, r.Header, h.cid); err != nil {
+		http.Error(w, err.Error(), http.StatusPreconditionFailed)
+		return
+	}
+
+	addRemoteFromRequest(h.tr, r)
+
+	// Limit the data size that could be read from the request body, which ensures that read from
+	// connection will not time out accidentally due to possible blocking in underlying implementation.
+	limitedr := pioutil.NewLimitedBufferReader(r.Body, connReadLimitByte)
+	b, err := ioutil.ReadAll(limitedr)
+	if err != nil {
+		if h.lg != nil {
+			h.lg.Warn(
+				"failed to read Raft message",
+				zap.String("local-member-id", h.localID.String()),
+				zap.Error(err),
+			)
+		} else {
+			plog.Errorf("failed to read raft message (%v)", err)
+		}
+		http.Error(w, "error reading raft message", http.StatusBadRequest)
+		recvFailures.WithLabelValues(r.RemoteAddr).Inc()
+		return
+	}
+
+	var m raftpb.Message
+	if err := m.Unmarshal(b); err != nil {
+		if h.lg != nil {
+			h.lg.Warn(
+				"failed to unmarshal Raft message",
+				zap.String("local-member-id", h.localID.String()),
+				zap.Error(err),
+			)
+		} else {
+			plog.Errorf("failed to unmarshal raft message (%v)", err)
+		}
+		http.Error(w, "error unmarshalling raft message", http.StatusBadRequest)
+		recvFailures.WithLabelValues(r.RemoteAddr).Inc()
+		return
+	}
+
+	receivedBytes.WithLabelValues(types.ID(m.From).String()).Add(float64(len(b)))
+
+	if err := h.r.Process(context.TODO(), m); err != nil {
+		switch v := err.(type) {
+		case writerToResponse:
+			v.WriteTo(w)
+		default:
+			if h.lg != nil {
+				h.lg.Warn(
+					"failed to process Raft message",
+					zap.String("local-member-id", h.localID.String()),
+					zap.Error(err),
+				)
+			} else {
+				plog.Warningf("failed to process raft message (%v)", err)
+			}
+			http.Error(w, "error processing raft message", http.StatusInternalServerError)
+			w.(http.Flusher).Flush()
+			// disconnect the http stream
+			panic(err)
+		}
+		return
+	}
+
+	// Write StatusNoContent header after the message has been processed by
+	// raft, which facilitates the client to report MsgSnap status.
+	w.WriteHeader(http.StatusNoContent)
+}
+
+type snapshotHandler struct {
+	lg          *zap.Logger
+	tr          Transporter
+	r           Raft
+	snapshotter *snap.Snapshotter
+
+	localID types.ID
+	cid     types.ID
+}
+
+func newSnapshotHandler(t *Transport, r Raft, snapshotter *snap.Snapshotter, cid types.ID) http.Handler {
+	return &snapshotHandler{
+		lg:          t.Logger,
+		tr:          t,
+		r:           r,
+		snapshotter: snapshotter,
+		localID:     t.ID,
+		cid:         cid,
+	}
+}
+
+const unknownSnapshotSender = "UNKNOWN_SNAPSHOT_SENDER"
+
+// ServeHTTP serves HTTP request to receive and process snapshot message.
+//
+// If request sender dies without closing underlying TCP connection,
+// the handler will keep waiting for the request body until TCP keepalive
+// finds out that the connection is broken after several minutes.
+// This is acceptable because
+// 1. snapshot messages sent through other TCP connections could still be
+// received and processed.
+// 2. this case should happen rarely, so no further optimization is done.
+func (h *snapshotHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
+	start := time.Now()
+
+	if r.Method != "POST" {
+		w.Header().Set("Allow", "POST")
+		http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed)
+		snapshotReceiveFailures.WithLabelValues(unknownSnapshotSender).Inc()
+		return
+	}
+
+	w.Header().Set("X-Etcd-Cluster-ID", h.cid.String())
+
+	if err := checkClusterCompatibilityFromHeader(h.lg, h.localID, r.Header, h.cid); err != nil {
+		http.Error(w, err.Error(), http.StatusPreconditionFailed)
+		snapshotReceiveFailures.WithLabelValues(unknownSnapshotSender).Inc()
+		return
+	}
+
+	addRemoteFromRequest(h.tr, r)
+
+	dec := &messageDecoder{r: r.Body}
+	// let snapshots be very large since they can exceed 512MB for large installations
+	m, err := dec.decodeLimit(uint64(1 << 63))
+	from := types.ID(m.From).String()
+	if err != nil {
+		msg := fmt.Sprintf("failed to decode raft message (%v)", err)
+		if h.lg != nil {
+			h.lg.Warn(
+				"failed to decode Raft message",
+				zap.String("local-member-id", h.localID.String()),
+				zap.String("remote-snapshot-sender-id", from),
+				zap.Error(err),
+			)
+		} else {
+			plog.Error(msg)
+		}
+		http.Error(w, msg, http.StatusBadRequest)
+		recvFailures.WithLabelValues(r.RemoteAddr).Inc()
+		snapshotReceiveFailures.WithLabelValues(from).Inc()
+		return
+	}
+
+	msgSize := m.Size()
+	receivedBytes.WithLabelValues(from).Add(float64(msgSize))
+
+	if m.Type != raftpb.MsgSnap {
+		if h.lg != nil {
+			h.lg.Warn(
+				"unexpected Raft message type",
+				zap.String("local-member-id", h.localID.String()),
+				zap.String("remote-snapshot-sender-id", from),
+				zap.String("message-type", m.Type.String()),
+			)
+		} else {
+			plog.Errorf("unexpected raft message type %s on snapshot path", m.Type)
+		}
+		http.Error(w, "wrong raft message type", http.StatusBadRequest)
+		snapshotReceiveFailures.WithLabelValues(from).Inc()
+		return
+	}
+
+	snapshotReceiveInflights.WithLabelValues(from).Inc()
+	defer func() {
+		snapshotReceiveInflights.WithLabelValues(from).Dec()
+	}()
+
+	if h.lg != nil {
+		h.lg.Info(
+			"receiving database snapshot",
+			zap.String("local-member-id", h.localID.String()),
+			zap.String("remote-snapshot-sender-id", from),
+			zap.Uint64("incoming-snapshot-index", m.Snapshot.Metadata.Index),
+			zap.Int("incoming-snapshot-message-size-bytes", msgSize),
+			zap.String("incoming-snapshot-message-size", humanize.Bytes(uint64(msgSize))),
+		)
+	} else {
+		plog.Infof("receiving database snapshot [index:%d, from %s] ...", m.Snapshot.Metadata.Index, types.ID(m.From))
+	}
+
+	// save incoming database snapshot.
+	n, err := h.snapshotter.SaveDBFrom(r.Body, m.Snapshot.Metadata.Index)
+	if err != nil {
+		msg := fmt.Sprintf("failed to save KV snapshot (%v)", err)
+		if h.lg != nil {
+			h.lg.Warn(
+				"failed to save incoming database snapshot",
+				zap.String("local-member-id", h.localID.String()),
+				zap.String("remote-snapshot-sender-id", from),
+				zap.Uint64("incoming-snapshot-index", m.Snapshot.Metadata.Index),
+				zap.Error(err),
+			)
+		} else {
+			plog.Error(msg)
+		}
+		http.Error(w, msg, http.StatusInternalServerError)
+		snapshotReceiveFailures.WithLabelValues(from).Inc()
+		return
+	}
+
+	receivedBytes.WithLabelValues(from).Add(float64(n))
+
+	if h.lg != nil {
+		h.lg.Info(
+			"received and saved database snapshot",
+			zap.String("local-member-id", h.localID.String()),
+			zap.String("remote-snapshot-sender-id", from),
+			zap.Uint64("incoming-snapshot-index", m.Snapshot.Metadata.Index),
+			zap.Int64("incoming-snapshot-size-bytes", n),
+			zap.String("incoming-snapshot-size", humanize.Bytes(uint64(n))),
+		)
+	} else {
+		plog.Infof("received and saved database snapshot [index: %d, from: %s] successfully", m.Snapshot.Metadata.Index, types.ID(m.From))
+	}
+
+	if err := h.r.Process(context.TODO(), m); err != nil {
+		switch v := err.(type) {
+		// Process may return writerToResponse error when doing some
+		// additional checks before calling raft.Node.Step.
+		case writerToResponse:
+			v.WriteTo(w)
+		default:
+			msg := fmt.Sprintf("failed to process raft message (%v)", err)
+			if h.lg != nil {
+				h.lg.Warn(
+					"failed to process Raft message",
+					zap.String("local-member-id", h.localID.String()),
+					zap.String("remote-snapshot-sender-id", from),
+					zap.Error(err),
+				)
+			} else {
+				plog.Error(msg)
+			}
+			http.Error(w, msg, http.StatusInternalServerError)
+			snapshotReceiveFailures.WithLabelValues(from).Inc()
+		}
+		return
+	}
+
+	// Write StatusNoContent header after the message has been processed by
+	// raft, which facilitates the client to report MsgSnap status.
+	w.WriteHeader(http.StatusNoContent)
+
+	snapshotReceive.WithLabelValues(from).Inc()
+	snapshotReceiveSeconds.WithLabelValues(from).Observe(time.Since(start).Seconds())
+}
+
+type streamHandler struct {
+	lg         *zap.Logger
+	tr         *Transport
+	peerGetter peerGetter
+	r          Raft
+	id         types.ID
+	cid        types.ID
+}
+
+func newStreamHandler(t *Transport, pg peerGetter, r Raft, id, cid types.ID) http.Handler {
+	return &streamHandler{
+		lg:         t.Logger,
+		tr:         t,
+		peerGetter: pg,
+		r:          r,
+		id:         id,
+		cid:        cid,
+	}
+}
+
+func (h *streamHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
+	if r.Method != "GET" {
+		w.Header().Set("Allow", "GET")
+		http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed)
+		return
+	}
+
+	w.Header().Set("X-Server-Version", version.Version)
+	w.Header().Set("X-Etcd-Cluster-ID", h.cid.String())
+
+	if err := checkClusterCompatibilityFromHeader(h.lg, h.tr.ID, r.Header, h.cid); err != nil {
+		http.Error(w, err.Error(), http.StatusPreconditionFailed)
+		return
+	}
+
+	var t streamType
+	switch path.Dir(r.URL.Path) {
+	case streamTypeMsgAppV2.endpoint():
+		t = streamTypeMsgAppV2
+	case streamTypeMessage.endpoint():
+		t = streamTypeMessage
+	default:
+		if h.lg != nil {
+			h.lg.Debug(
+				"ignored unexpected streaming request path",
+				zap.String("local-member-id", h.tr.ID.String()),
+				zap.String("remote-peer-id-stream-handler", h.id.String()),
+				zap.String("path", r.URL.Path),
+			)
+		} else {
+			plog.Debugf("ignored unexpected streaming request path %s", r.URL.Path)
+		}
+		http.Error(w, "invalid path", http.StatusNotFound)
+		return
+	}
+
+	fromStr := path.Base(r.URL.Path)
+	from, err := types.IDFromString(fromStr)
+	if err != nil {
+		if h.lg != nil {
+			h.lg.Warn(
+				"failed to parse path into ID",
+				zap.String("local-member-id", h.tr.ID.String()),
+				zap.String("remote-peer-id-stream-handler", h.id.String()),
+				zap.String("path", fromStr),
+				zap.Error(err),
+			)
+		} else {
+			plog.Errorf("failed to parse from %s into ID (%v)", fromStr, err)
+		}
+		http.Error(w, "invalid from", http.StatusNotFound)
+		return
+	}
+	if h.r.IsIDRemoved(uint64(from)) {
+		if h.lg != nil {
+			h.lg.Warn(
+				"rejected stream from remote peer because it was removed",
+				zap.String("local-member-id", h.tr.ID.String()),
+				zap.String("remote-peer-id-stream-handler", h.id.String()),
+				zap.String("remote-peer-id-from", from.String()),
+			)
+		} else {
+			plog.Warningf("rejected the stream from peer %s since it was removed", from)
+		}
+		http.Error(w, "removed member", http.StatusGone)
+		return
+	}
+	p := h.peerGetter.Get(from)
+	if p == nil {
+		// This may happen in following cases:
+		// 1. user starts a remote peer that belongs to a different cluster
+		// with the same cluster ID.
+		// 2. local etcd falls behind of the cluster, and cannot recognize
+		// the members that joined after its current progress.
+		if urls := r.Header.Get("X-PeerURLs"); urls != "" {
+			h.tr.AddRemote(from, strings.Split(urls, ","))
+		}
+		if h.lg != nil {
+			h.lg.Warn(
+				"failed to find remote peer in cluster",
+				zap.String("local-member-id", h.tr.ID.String()),
+				zap.String("remote-peer-id-stream-handler", h.id.String()),
+				zap.String("remote-peer-id-from", from.String()),
+				zap.String("cluster-id", h.cid.String()),
+			)
+		} else {
+			plog.Errorf("failed to find member %s in cluster %s", from, h.cid)
+		}
+		http.Error(w, "error sender not found", http.StatusNotFound)
+		return
+	}
+
+	wto := h.id.String()
+	if gto := r.Header.Get("X-Raft-To"); gto != wto {
+		if h.lg != nil {
+			h.lg.Warn(
+				"ignored streaming request; ID mismatch",
+				zap.String("local-member-id", h.tr.ID.String()),
+				zap.String("remote-peer-id-stream-handler", h.id.String()),
+				zap.String("remote-peer-id-header", gto),
+				zap.String("remote-peer-id-from", from.String()),
+				zap.String("cluster-id", h.cid.String()),
+			)
+		} else {
+			plog.Errorf("streaming request ignored (ID mismatch got %s want %s)", gto, wto)
+		}
+		http.Error(w, "to field mismatch", http.StatusPreconditionFailed)
+		return
+	}
+
+	w.WriteHeader(http.StatusOK)
+	w.(http.Flusher).Flush()
+
+	c := newCloseNotifier()
+	conn := &outgoingConn{
+		t:       t,
+		Writer:  w,
+		Flusher: w.(http.Flusher),
+		Closer:  c,
+		localID: h.tr.ID,
+		peerID:  h.id,
+	}
+	p.attachOutgoingConn(conn)
+	<-c.closeNotify()
+}
+
+// checkClusterCompatibilityFromHeader checks the cluster compatibility of
+// the local member from the given header.
+// It checks whether the version of local member is compatible with
+// the versions in the header, and whether the cluster ID of local member
+// matches the one in the header.
+func checkClusterCompatibilityFromHeader(lg *zap.Logger, localID types.ID, header http.Header, cid types.ID) error {
+	remoteName := header.Get("X-Server-From")
+
+	remoteServer := serverVersion(header)
+	remoteVs := ""
+	if remoteServer != nil {
+		remoteVs = remoteServer.String()
+	}
+
+	remoteMinClusterVer := minClusterVersion(header)
+	remoteMinClusterVs := ""
+	if remoteMinClusterVer != nil {
+		remoteMinClusterVs = remoteMinClusterVer.String()
+	}
+
+	localServer, localMinCluster, err := checkVersionCompatibility(remoteName, remoteServer, remoteMinClusterVer)
+
+	localVs := ""
+	if localServer != nil {
+		localVs = localServer.String()
+	}
+	localMinClusterVs := ""
+	if localMinCluster != nil {
+		localMinClusterVs = localMinCluster.String()
+	}
+
+	if err != nil {
+		if lg != nil {
+			lg.Warn(
+				"failed to check version compatibility",
+				zap.String("local-member-id", localID.String()),
+				zap.String("local-member-cluster-id", cid.String()),
+				zap.String("local-member-server-version", localVs),
+				zap.String("local-member-server-minimum-cluster-version", localMinClusterVs),
+				zap.String("remote-peer-server-name", remoteName),
+				zap.String("remote-peer-server-version", remoteVs),
+				zap.String("remote-peer-server-minimum-cluster-version", remoteMinClusterVs),
+				zap.Error(err),
+			)
+		} else {
+			plog.Errorf("request version incompatibility (%v)", err)
+		}
+		return errIncompatibleVersion
+	}
+	if gcid := header.Get("X-Etcd-Cluster-ID"); gcid != cid.String() {
+		if lg != nil {
+			lg.Warn(
+				"request cluster ID mismatch",
+				zap.String("local-member-id", localID.String()),
+				zap.String("local-member-cluster-id", cid.String()),
+				zap.String("local-member-server-version", localVs),
+				zap.String("local-member-server-minimum-cluster-version", localMinClusterVs),
+				zap.String("remote-peer-server-name", remoteName),
+				zap.String("remote-peer-server-version", remoteVs),
+				zap.String("remote-peer-server-minimum-cluster-version", remoteMinClusterVs),
+				zap.String("remote-peer-cluster-id", gcid),
+			)
+		} else {
+			plog.Errorf("request cluster ID mismatch (got %s want %s)", gcid, cid)
+		}
+		return errClusterIDMismatch
+	}
+	return nil
+}
+
+type closeNotifier struct {
+	done chan struct{}
+}
+
+func newCloseNotifier() *closeNotifier {
+	return &closeNotifier{
+		done: make(chan struct{}),
+	}
+}
+
+func (n *closeNotifier) Close() error {
+	close(n.done)
+	return nil
+}
+
+func (n *closeNotifier) closeNotify() <-chan struct{} { return n.done }
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/metrics.go b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/metrics.go
new file mode 100644
index 0000000..02fff84
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/metrics.go
@@ -0,0 +1,186 @@
+// Copyright 2015 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 rafthttp
+
+import "github.com/prometheus/client_golang/prometheus"
+
+var (
+	activePeers = prometheus.NewGaugeVec(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "active_peers",
+		Help:      "The current number of active peer connections.",
+	},
+		[]string{"Local", "Remote"},
+	)
+
+	disconnectedPeers = prometheus.NewCounterVec(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "disconnected_peers_total",
+		Help:      "The total number of disconnected peers.",
+	},
+		[]string{"Local", "Remote"},
+	)
+
+	sentBytes = prometheus.NewCounterVec(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "peer_sent_bytes_total",
+		Help:      "The total number of bytes sent to peers.",
+	},
+		[]string{"To"},
+	)
+
+	receivedBytes = prometheus.NewCounterVec(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "peer_received_bytes_total",
+		Help:      "The total number of bytes received from peers.",
+	},
+		[]string{"From"},
+	)
+
+	sentFailures = prometheus.NewCounterVec(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "peer_sent_failures_total",
+		Help:      "The total number of send failures from peers.",
+	},
+		[]string{"To"},
+	)
+
+	recvFailures = prometheus.NewCounterVec(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "peer_received_failures_total",
+		Help:      "The total number of receive failures from peers.",
+	},
+		[]string{"From"},
+	)
+
+	snapshotSend = prometheus.NewCounterVec(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "snapshot_send_success",
+		Help:      "Total number of successful snapshot sends",
+	},
+		[]string{"To"},
+	)
+
+	snapshotSendInflights = prometheus.NewGaugeVec(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "snapshot_send_inflights_total",
+		Help:      "Total number of inflight snapshot sends",
+	},
+		[]string{"To"},
+	)
+
+	snapshotSendFailures = prometheus.NewCounterVec(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "snapshot_send_failures",
+		Help:      "Total number of snapshot send failures",
+	},
+		[]string{"To"},
+	)
+
+	snapshotSendSeconds = prometheus.NewHistogramVec(prometheus.HistogramOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "snapshot_send_total_duration_seconds",
+		Help:      "Total latency distributions of v3 snapshot sends",
+
+		// lowest bucket start of upper bound 0.1 sec (100 ms) with factor 2
+		// highest bucket start of 0.1 sec * 2^9 == 51.2 sec
+		Buckets: prometheus.ExponentialBuckets(0.1, 2, 10),
+	},
+		[]string{"To"},
+	)
+
+	snapshotReceive = prometheus.NewCounterVec(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "snapshot_receive_success",
+		Help:      "Total number of successful snapshot receives",
+	},
+		[]string{"From"},
+	)
+
+	snapshotReceiveInflights = prometheus.NewGaugeVec(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "snapshot_receive_inflights_total",
+		Help:      "Total number of inflight snapshot receives",
+	},
+		[]string{"From"},
+	)
+
+	snapshotReceiveFailures = prometheus.NewCounterVec(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "snapshot_receive_failures",
+		Help:      "Total number of snapshot receive failures",
+	},
+		[]string{"From"},
+	)
+
+	snapshotReceiveSeconds = prometheus.NewHistogramVec(prometheus.HistogramOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "snapshot_receive_total_duration_seconds",
+		Help:      "Total latency distributions of v3 snapshot receives",
+
+		// lowest bucket start of upper bound 0.1 sec (100 ms) with factor 2
+		// highest bucket start of 0.1 sec * 2^9 == 51.2 sec
+		Buckets: prometheus.ExponentialBuckets(0.1, 2, 10),
+	},
+		[]string{"From"},
+	)
+
+	rttSec = prometheus.NewHistogramVec(prometheus.HistogramOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "peer_round_trip_time_seconds",
+		Help:      "Round-Trip-Time histogram between peers",
+
+		// lowest bucket start of upper bound 0.0001 sec (0.1 ms) with factor 2
+		// highest bucket start of 0.0001 sec * 2^15 == 3.2768 sec
+		Buckets: prometheus.ExponentialBuckets(0.0001, 2, 16),
+	},
+		[]string{"To"},
+	)
+)
+
+func init() {
+	prometheus.MustRegister(activePeers)
+	prometheus.MustRegister(disconnectedPeers)
+	prometheus.MustRegister(sentBytes)
+	prometheus.MustRegister(receivedBytes)
+	prometheus.MustRegister(sentFailures)
+	prometheus.MustRegister(recvFailures)
+
+	prometheus.MustRegister(snapshotSend)
+	prometheus.MustRegister(snapshotSendInflights)
+	prometheus.MustRegister(snapshotSendFailures)
+	prometheus.MustRegister(snapshotSendSeconds)
+	prometheus.MustRegister(snapshotReceive)
+	prometheus.MustRegister(snapshotReceiveInflights)
+	prometheus.MustRegister(snapshotReceiveFailures)
+	prometheus.MustRegister(snapshotReceiveSeconds)
+
+	prometheus.MustRegister(rttSec)
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/msg_codec.go b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/msg_codec.go
new file mode 100644
index 0000000..2417d22
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/msg_codec.go
@@ -0,0 +1,68 @@
+// Copyright 2015 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 rafthttp
+
+import (
+	"encoding/binary"
+	"errors"
+	"io"
+
+	"go.etcd.io/etcd/pkg/pbutil"
+	"go.etcd.io/etcd/raft/raftpb"
+)
+
+// messageEncoder is a encoder that can encode all kinds of messages.
+// It MUST be used with a paired messageDecoder.
+type messageEncoder struct {
+	w io.Writer
+}
+
+func (enc *messageEncoder) encode(m *raftpb.Message) error {
+	if err := binary.Write(enc.w, binary.BigEndian, uint64(m.Size())); err != nil {
+		return err
+	}
+	_, err := enc.w.Write(pbutil.MustMarshal(m))
+	return err
+}
+
+// messageDecoder is a decoder that can decode all kinds of messages.
+type messageDecoder struct {
+	r io.Reader
+}
+
+var (
+	readBytesLimit     uint64 = 512 * 1024 * 1024 // 512 MB
+	ErrExceedSizeLimit        = errors.New("rafthttp: error limit exceeded")
+)
+
+func (dec *messageDecoder) decode() (raftpb.Message, error) {
+	return dec.decodeLimit(readBytesLimit)
+}
+
+func (dec *messageDecoder) decodeLimit(numBytes uint64) (raftpb.Message, error) {
+	var m raftpb.Message
+	var l uint64
+	if err := binary.Read(dec.r, binary.BigEndian, &l); err != nil {
+		return m, err
+	}
+	if l > numBytes {
+		return m, ErrExceedSizeLimit
+	}
+	buf := make([]byte, int(l))
+	if _, err := io.ReadFull(dec.r, buf); err != nil {
+		return m, err
+	}
+	return m, m.Unmarshal(buf)
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/msgappv2_codec.go b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/msgappv2_codec.go
new file mode 100644
index 0000000..1fa36de
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/msgappv2_codec.go
@@ -0,0 +1,248 @@
+// Copyright 2015 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 rafthttp
+
+import (
+	"encoding/binary"
+	"fmt"
+	"io"
+	"time"
+
+	stats "go.etcd.io/etcd/etcdserver/api/v2stats"
+	"go.etcd.io/etcd/pkg/pbutil"
+	"go.etcd.io/etcd/pkg/types"
+	"go.etcd.io/etcd/raft/raftpb"
+)
+
+const (
+	msgTypeLinkHeartbeat uint8 = 0
+	msgTypeAppEntries    uint8 = 1
+	msgTypeApp           uint8 = 2
+
+	msgAppV2BufSize = 1024 * 1024
+)
+
+// msgappv2 stream sends three types of message: linkHeartbeatMessage,
+// AppEntries and MsgApp. AppEntries is the MsgApp that is sent in
+// replicate state in raft, whose index and term are fully predictable.
+//
+// Data format of linkHeartbeatMessage:
+// | offset | bytes | description |
+// +--------+-------+-------------+
+// | 0      | 1     | \x00        |
+//
+// Data format of AppEntries:
+// | offset | bytes | description |
+// +--------+-------+-------------+
+// | 0      | 1     | \x01        |
+// | 1      | 8     | length of entries |
+// | 9      | 8     | length of first entry |
+// | 17     | n1    | first entry |
+// ...
+// | x      | 8     | length of k-th entry data |
+// | x+8    | nk    | k-th entry data |
+// | x+8+nk | 8     | commit index |
+//
+// Data format of MsgApp:
+// | offset | bytes | description |
+// +--------+-------+-------------+
+// | 0      | 1     | \x02        |
+// | 1      | 8     | length of encoded message |
+// | 9      | n     | encoded message |
+type msgAppV2Encoder struct {
+	w  io.Writer
+	fs *stats.FollowerStats
+
+	term      uint64
+	index     uint64
+	buf       []byte
+	uint64buf []byte
+	uint8buf  []byte
+}
+
+func newMsgAppV2Encoder(w io.Writer, fs *stats.FollowerStats) *msgAppV2Encoder {
+	return &msgAppV2Encoder{
+		w:         w,
+		fs:        fs,
+		buf:       make([]byte, msgAppV2BufSize),
+		uint64buf: make([]byte, 8),
+		uint8buf:  make([]byte, 1),
+	}
+}
+
+func (enc *msgAppV2Encoder) encode(m *raftpb.Message) error {
+	start := time.Now()
+	switch {
+	case isLinkHeartbeatMessage(m):
+		enc.uint8buf[0] = msgTypeLinkHeartbeat
+		if _, err := enc.w.Write(enc.uint8buf); err != nil {
+			return err
+		}
+	case enc.index == m.Index && enc.term == m.LogTerm && m.LogTerm == m.Term:
+		enc.uint8buf[0] = msgTypeAppEntries
+		if _, err := enc.w.Write(enc.uint8buf); err != nil {
+			return err
+		}
+		// write length of entries
+		binary.BigEndian.PutUint64(enc.uint64buf, uint64(len(m.Entries)))
+		if _, err := enc.w.Write(enc.uint64buf); err != nil {
+			return err
+		}
+		for i := 0; i < len(m.Entries); i++ {
+			// write length of entry
+			binary.BigEndian.PutUint64(enc.uint64buf, uint64(m.Entries[i].Size()))
+			if _, err := enc.w.Write(enc.uint64buf); err != nil {
+				return err
+			}
+			if n := m.Entries[i].Size(); n < msgAppV2BufSize {
+				if _, err := m.Entries[i].MarshalTo(enc.buf); err != nil {
+					return err
+				}
+				if _, err := enc.w.Write(enc.buf[:n]); err != nil {
+					return err
+				}
+			} else {
+				if _, err := enc.w.Write(pbutil.MustMarshal(&m.Entries[i])); err != nil {
+					return err
+				}
+			}
+			enc.index++
+		}
+		// write commit index
+		binary.BigEndian.PutUint64(enc.uint64buf, m.Commit)
+		if _, err := enc.w.Write(enc.uint64buf); err != nil {
+			return err
+		}
+		enc.fs.Succ(time.Since(start))
+	default:
+		if err := binary.Write(enc.w, binary.BigEndian, msgTypeApp); err != nil {
+			return err
+		}
+		// write size of message
+		if err := binary.Write(enc.w, binary.BigEndian, uint64(m.Size())); err != nil {
+			return err
+		}
+		// write message
+		if _, err := enc.w.Write(pbutil.MustMarshal(m)); err != nil {
+			return err
+		}
+
+		enc.term = m.Term
+		enc.index = m.Index
+		if l := len(m.Entries); l > 0 {
+			enc.index = m.Entries[l-1].Index
+		}
+		enc.fs.Succ(time.Since(start))
+	}
+	return nil
+}
+
+type msgAppV2Decoder struct {
+	r             io.Reader
+	local, remote types.ID
+
+	term      uint64
+	index     uint64
+	buf       []byte
+	uint64buf []byte
+	uint8buf  []byte
+}
+
+func newMsgAppV2Decoder(r io.Reader, local, remote types.ID) *msgAppV2Decoder {
+	return &msgAppV2Decoder{
+		r:         r,
+		local:     local,
+		remote:    remote,
+		buf:       make([]byte, msgAppV2BufSize),
+		uint64buf: make([]byte, 8),
+		uint8buf:  make([]byte, 1),
+	}
+}
+
+func (dec *msgAppV2Decoder) decode() (raftpb.Message, error) {
+	var (
+		m   raftpb.Message
+		typ uint8
+	)
+	if _, err := io.ReadFull(dec.r, dec.uint8buf); err != nil {
+		return m, err
+	}
+	typ = dec.uint8buf[0]
+	switch typ {
+	case msgTypeLinkHeartbeat:
+		return linkHeartbeatMessage, nil
+	case msgTypeAppEntries:
+		m = raftpb.Message{
+			Type:    raftpb.MsgApp,
+			From:    uint64(dec.remote),
+			To:      uint64(dec.local),
+			Term:    dec.term,
+			LogTerm: dec.term,
+			Index:   dec.index,
+		}
+
+		// decode entries
+		if _, err := io.ReadFull(dec.r, dec.uint64buf); err != nil {
+			return m, err
+		}
+		l := binary.BigEndian.Uint64(dec.uint64buf)
+		m.Entries = make([]raftpb.Entry, int(l))
+		for i := 0; i < int(l); i++ {
+			if _, err := io.ReadFull(dec.r, dec.uint64buf); err != nil {
+				return m, err
+			}
+			size := binary.BigEndian.Uint64(dec.uint64buf)
+			var buf []byte
+			if size < msgAppV2BufSize {
+				buf = dec.buf[:size]
+				if _, err := io.ReadFull(dec.r, buf); err != nil {
+					return m, err
+				}
+			} else {
+				buf = make([]byte, int(size))
+				if _, err := io.ReadFull(dec.r, buf); err != nil {
+					return m, err
+				}
+			}
+			dec.index++
+			// 1 alloc
+			pbutil.MustUnmarshal(&m.Entries[i], buf)
+		}
+		// decode commit index
+		if _, err := io.ReadFull(dec.r, dec.uint64buf); err != nil {
+			return m, err
+		}
+		m.Commit = binary.BigEndian.Uint64(dec.uint64buf)
+	case msgTypeApp:
+		var size uint64
+		if err := binary.Read(dec.r, binary.BigEndian, &size); err != nil {
+			return m, err
+		}
+		buf := make([]byte, int(size))
+		if _, err := io.ReadFull(dec.r, buf); err != nil {
+			return m, err
+		}
+		pbutil.MustUnmarshal(&m, buf)
+
+		dec.term = m.Term
+		dec.index = m.Index
+		if l := len(m.Entries); l > 0 {
+			dec.index = m.Entries[l-1].Index
+		}
+	default:
+		return m, fmt.Errorf("failed to parse type %d in msgappv2 stream", typ)
+	}
+	return m, nil
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/peer.go b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/peer.go
new file mode 100644
index 0000000..8130c4a
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/peer.go
@@ -0,0 +1,374 @@
+// Copyright 2015 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 rafthttp
+
+import (
+	"context"
+	"sync"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver/api/snap"
+	stats "go.etcd.io/etcd/etcdserver/api/v2stats"
+	"go.etcd.io/etcd/pkg/types"
+	"go.etcd.io/etcd/raft"
+	"go.etcd.io/etcd/raft/raftpb"
+
+	"go.uber.org/zap"
+	"golang.org/x/time/rate"
+)
+
+const (
+	// ConnReadTimeout and ConnWriteTimeout are the i/o timeout set on each connection rafthttp pkg creates.
+	// A 5 seconds timeout is good enough for recycling bad connections. Or we have to wait for
+	// tcp keepalive failing to detect a bad connection, which is at minutes level.
+	// For long term streaming connections, rafthttp pkg sends application level linkHeartbeatMessage
+	// to keep the connection alive.
+	// For short term pipeline connections, the connection MUST be killed to avoid it being
+	// put back to http pkg connection pool.
+	ConnReadTimeout  = 5 * time.Second
+	ConnWriteTimeout = 5 * time.Second
+
+	recvBufSize = 4096
+	// maxPendingProposals holds the proposals during one leader election process.
+	// Generally one leader election takes at most 1 sec. It should have
+	// 0-2 election conflicts, and each one takes 0.5 sec.
+	// We assume the number of concurrent proposers is smaller than 4096.
+	// One client blocks on its proposal for at least 1 sec, so 4096 is enough
+	// to hold all proposals.
+	maxPendingProposals = 4096
+
+	streamAppV2 = "streamMsgAppV2"
+	streamMsg   = "streamMsg"
+	pipelineMsg = "pipeline"
+	sendSnap    = "sendMsgSnap"
+)
+
+type Peer interface {
+	// send sends the message to the remote peer. The function is non-blocking
+	// and has no promise that the message will be received by the remote.
+	// When it fails to send message out, it will report the status to underlying
+	// raft.
+	send(m raftpb.Message)
+
+	// sendSnap sends the merged snapshot message to the remote peer. Its behavior
+	// is similar to send.
+	sendSnap(m snap.Message)
+
+	// update updates the urls of remote peer.
+	update(urls types.URLs)
+
+	// attachOutgoingConn attaches the outgoing connection to the peer for
+	// stream usage. After the call, the ownership of the outgoing
+	// connection hands over to the peer. The peer will close the connection
+	// when it is no longer used.
+	attachOutgoingConn(conn *outgoingConn)
+	// activeSince returns the time that the connection with the
+	// peer becomes active.
+	activeSince() time.Time
+	// stop performs any necessary finalization and terminates the peer
+	// elegantly.
+	stop()
+}
+
+// peer is the representative of a remote raft node. Local raft node sends
+// messages to the remote through peer.
+// Each peer has two underlying mechanisms to send out a message: stream and
+// pipeline.
+// A stream is a receiver initialized long-polling connection, which
+// is always open to transfer messages. Besides general stream, peer also has
+// a optimized stream for sending msgApp since msgApp accounts for large part
+// of all messages. Only raft leader uses the optimized stream to send msgApp
+// to the remote follower node.
+// A pipeline is a series of http clients that send http requests to the remote.
+// It is only used when the stream has not been established.
+type peer struct {
+	lg *zap.Logger
+
+	localID types.ID
+	// id of the remote raft peer node
+	id types.ID
+
+	r Raft
+
+	status *peerStatus
+
+	picker *urlPicker
+
+	msgAppV2Writer *streamWriter
+	writer         *streamWriter
+	pipeline       *pipeline
+	snapSender     *snapshotSender // snapshot sender to send v3 snapshot messages
+	msgAppV2Reader *streamReader
+	msgAppReader   *streamReader
+
+	recvc chan raftpb.Message
+	propc chan raftpb.Message
+
+	mu     sync.Mutex
+	paused bool
+
+	cancel context.CancelFunc // cancel pending works in go routine created by peer.
+	stopc  chan struct{}
+}
+
+func startPeer(t *Transport, urls types.URLs, peerID types.ID, fs *stats.FollowerStats) *peer {
+	if t.Logger != nil {
+		t.Logger.Info("starting remote peer", zap.String("remote-peer-id", peerID.String()))
+	} else {
+		plog.Infof("starting peer %s...", peerID)
+	}
+	defer func() {
+		if t.Logger != nil {
+			t.Logger.Info("started remote peer", zap.String("remote-peer-id", peerID.String()))
+		} else {
+			plog.Infof("started peer %s", peerID)
+		}
+	}()
+
+	status := newPeerStatus(t.Logger, t.ID, peerID)
+	picker := newURLPicker(urls)
+	errorc := t.ErrorC
+	r := t.Raft
+	pipeline := &pipeline{
+		peerID:        peerID,
+		tr:            t,
+		picker:        picker,
+		status:        status,
+		followerStats: fs,
+		raft:          r,
+		errorc:        errorc,
+	}
+	pipeline.start()
+
+	p := &peer{
+		lg:             t.Logger,
+		localID:        t.ID,
+		id:             peerID,
+		r:              r,
+		status:         status,
+		picker:         picker,
+		msgAppV2Writer: startStreamWriter(t.Logger, t.ID, peerID, status, fs, r),
+		writer:         startStreamWriter(t.Logger, t.ID, peerID, status, fs, r),
+		pipeline:       pipeline,
+		snapSender:     newSnapshotSender(t, picker, peerID, status),
+		recvc:          make(chan raftpb.Message, recvBufSize),
+		propc:          make(chan raftpb.Message, maxPendingProposals),
+		stopc:          make(chan struct{}),
+	}
+
+	ctx, cancel := context.WithCancel(context.Background())
+	p.cancel = cancel
+	go func() {
+		for {
+			select {
+			case mm := <-p.recvc:
+				if err := r.Process(ctx, mm); err != nil {
+					if t.Logger != nil {
+						t.Logger.Warn("failed to process Raft message", zap.Error(err))
+					} else {
+						plog.Warningf("failed to process raft message (%v)", err)
+					}
+				}
+			case <-p.stopc:
+				return
+			}
+		}
+	}()
+
+	// r.Process might block for processing proposal when there is no leader.
+	// Thus propc must be put into a separate routine with recvc to avoid blocking
+	// processing other raft messages.
+	go func() {
+		for {
+			select {
+			case mm := <-p.propc:
+				if err := r.Process(ctx, mm); err != nil {
+					plog.Warningf("failed to process raft message (%v)", err)
+				}
+			case <-p.stopc:
+				return
+			}
+		}
+	}()
+
+	p.msgAppV2Reader = &streamReader{
+		lg:     t.Logger,
+		peerID: peerID,
+		typ:    streamTypeMsgAppV2,
+		tr:     t,
+		picker: picker,
+		status: status,
+		recvc:  p.recvc,
+		propc:  p.propc,
+		rl:     rate.NewLimiter(t.DialRetryFrequency, 1),
+	}
+	p.msgAppReader = &streamReader{
+		lg:     t.Logger,
+		peerID: peerID,
+		typ:    streamTypeMessage,
+		tr:     t,
+		picker: picker,
+		status: status,
+		recvc:  p.recvc,
+		propc:  p.propc,
+		rl:     rate.NewLimiter(t.DialRetryFrequency, 1),
+	}
+
+	p.msgAppV2Reader.start()
+	p.msgAppReader.start()
+
+	return p
+}
+
+func (p *peer) send(m raftpb.Message) {
+	p.mu.Lock()
+	paused := p.paused
+	p.mu.Unlock()
+
+	if paused {
+		return
+	}
+
+	writec, name := p.pick(m)
+	select {
+	case writec <- m:
+	default:
+		p.r.ReportUnreachable(m.To)
+		if isMsgSnap(m) {
+			p.r.ReportSnapshot(m.To, raft.SnapshotFailure)
+		}
+		if p.status.isActive() {
+			if p.lg != nil {
+				p.lg.Warn(
+					"dropped internal Raft message since sending buffer is full (overloaded network)",
+					zap.String("message-type", m.Type.String()),
+					zap.String("local-member-id", p.localID.String()),
+					zap.String("from", types.ID(m.From).String()),
+					zap.String("remote-peer-id", p.id.String()),
+					zap.Bool("remote-peer-active", p.status.isActive()),
+				)
+			} else {
+				plog.MergeWarningf("dropped internal raft message to %s since %s's sending buffer is full (bad/overloaded network)", p.id, name)
+			}
+		} else {
+			if p.lg != nil {
+				p.lg.Warn(
+					"dropped internal Raft message since sending buffer is full (overloaded network)",
+					zap.String("message-type", m.Type.String()),
+					zap.String("local-member-id", p.localID.String()),
+					zap.String("from", types.ID(m.From).String()),
+					zap.String("remote-peer-id", p.id.String()),
+					zap.Bool("remote-peer-active", p.status.isActive()),
+				)
+			} else {
+				plog.Debugf("dropped %s to %s since %s's sending buffer is full", m.Type, p.id, name)
+			}
+		}
+		sentFailures.WithLabelValues(types.ID(m.To).String()).Inc()
+	}
+}
+
+func (p *peer) sendSnap(m snap.Message) {
+	go p.snapSender.send(m)
+}
+
+func (p *peer) update(urls types.URLs) {
+	p.picker.update(urls)
+}
+
+func (p *peer) attachOutgoingConn(conn *outgoingConn) {
+	var ok bool
+	switch conn.t {
+	case streamTypeMsgAppV2:
+		ok = p.msgAppV2Writer.attach(conn)
+	case streamTypeMessage:
+		ok = p.writer.attach(conn)
+	default:
+		if p.lg != nil {
+			p.lg.Panic("unknown stream type", zap.String("type", conn.t.String()))
+		} else {
+			plog.Panicf("unhandled stream type %s", conn.t)
+		}
+	}
+	if !ok {
+		conn.Close()
+	}
+}
+
+func (p *peer) activeSince() time.Time { return p.status.activeSince() }
+
+// Pause pauses the peer. The peer will simply drops all incoming
+// messages without returning an error.
+func (p *peer) Pause() {
+	p.mu.Lock()
+	defer p.mu.Unlock()
+	p.paused = true
+	p.msgAppReader.pause()
+	p.msgAppV2Reader.pause()
+}
+
+// Resume resumes a paused peer.
+func (p *peer) Resume() {
+	p.mu.Lock()
+	defer p.mu.Unlock()
+	p.paused = false
+	p.msgAppReader.resume()
+	p.msgAppV2Reader.resume()
+}
+
+func (p *peer) stop() {
+	if p.lg != nil {
+		p.lg.Info("stopping remote peer", zap.String("remote-peer-id", p.id.String()))
+	} else {
+		plog.Infof("stopping peer %s...", p.id)
+	}
+
+	defer func() {
+		if p.lg != nil {
+			p.lg.Info("stopped remote peer", zap.String("remote-peer-id", p.id.String()))
+		} else {
+			plog.Infof("stopped peer %s", p.id)
+		}
+	}()
+
+	close(p.stopc)
+	p.cancel()
+	p.msgAppV2Writer.stop()
+	p.writer.stop()
+	p.pipeline.stop()
+	p.snapSender.stop()
+	p.msgAppV2Reader.stop()
+	p.msgAppReader.stop()
+}
+
+// pick picks a chan for sending the given message. The picked chan and the picked chan
+// string name are returned.
+func (p *peer) pick(m raftpb.Message) (writec chan<- raftpb.Message, picked string) {
+	var ok bool
+	// Considering MsgSnap may have a big size, e.g., 1G, and will block
+	// stream for a long time, only use one of the N pipelines to send MsgSnap.
+	if isMsgSnap(m) {
+		return p.pipeline.msgc, pipelineMsg
+	} else if writec, ok = p.msgAppV2Writer.writec(); ok && isMsgApp(m) {
+		return writec, streamAppV2
+	} else if writec, ok = p.writer.writec(); ok {
+		return writec, streamMsg
+	}
+	return p.pipeline.msgc, pipelineMsg
+}
+
+func isMsgApp(m raftpb.Message) bool { return m.Type == raftpb.MsgApp }
+
+func isMsgSnap(m raftpb.Message) bool { return m.Type == raftpb.MsgSnap }
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/peer_status.go b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/peer_status.go
new file mode 100644
index 0000000..66149ff
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/peer_status.go
@@ -0,0 +1,96 @@
+// Copyright 2015 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 rafthttp
+
+import (
+	"errors"
+	"fmt"
+	"sync"
+	"time"
+
+	"go.etcd.io/etcd/pkg/types"
+
+	"go.uber.org/zap"
+)
+
+type failureType struct {
+	source string
+	action string
+}
+
+type peerStatus struct {
+	lg     *zap.Logger
+	local  types.ID
+	id     types.ID
+	mu     sync.Mutex // protect variables below
+	active bool
+	since  time.Time
+}
+
+func newPeerStatus(lg *zap.Logger, local, id types.ID) *peerStatus {
+	return &peerStatus{lg: lg, local: local, id: id}
+}
+
+func (s *peerStatus) activate() {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+	if !s.active {
+		if s.lg != nil {
+			s.lg.Info("peer became active", zap.String("peer-id", s.id.String()))
+		} else {
+			plog.Infof("peer %s became active", s.id)
+		}
+		s.active = true
+		s.since = time.Now()
+
+		activePeers.WithLabelValues(s.local.String(), s.id.String()).Inc()
+	}
+}
+
+func (s *peerStatus) deactivate(failure failureType, reason string) {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+	msg := fmt.Sprintf("failed to %s %s on %s (%s)", failure.action, s.id, failure.source, reason)
+	if s.active {
+		if s.lg != nil {
+			s.lg.Warn("peer became inactive (message send to peer failed)", zap.String("peer-id", s.id.String()), zap.Error(errors.New(msg)))
+		} else {
+			plog.Errorf(msg)
+			plog.Infof("peer %s became inactive (message send to peer failed)", s.id)
+		}
+		s.active = false
+		s.since = time.Time{}
+
+		activePeers.WithLabelValues(s.local.String(), s.id.String()).Dec()
+		disconnectedPeers.WithLabelValues(s.local.String(), s.id.String()).Inc()
+		return
+	}
+
+	if s.lg != nil {
+		s.lg.Debug("peer deactivated again", zap.String("peer-id", s.id.String()), zap.Error(errors.New(msg)))
+	}
+}
+
+func (s *peerStatus) isActive() bool {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+	return s.active
+}
+
+func (s *peerStatus) activeSince() time.Time {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+	return s.since
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/pipeline.go b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/pipeline.go
new file mode 100644
index 0000000..70f9257
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/pipeline.go
@@ -0,0 +1,180 @@
+// Copyright 2015 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 rafthttp
+
+import (
+	"bytes"
+	"context"
+	"errors"
+	"io/ioutil"
+	"sync"
+	"time"
+
+	stats "go.etcd.io/etcd/etcdserver/api/v2stats"
+	"go.etcd.io/etcd/pkg/pbutil"
+	"go.etcd.io/etcd/pkg/types"
+	"go.etcd.io/etcd/raft"
+	"go.etcd.io/etcd/raft/raftpb"
+
+	"go.uber.org/zap"
+)
+
+const (
+	connPerPipeline = 4
+	// pipelineBufSize is the size of pipeline buffer, which helps hold the
+	// temporary network latency.
+	// The size ensures that pipeline does not drop messages when the network
+	// is out of work for less than 1 second in good path.
+	pipelineBufSize = 64
+)
+
+var errStopped = errors.New("stopped")
+
+type pipeline struct {
+	peerID types.ID
+
+	tr     *Transport
+	picker *urlPicker
+	status *peerStatus
+	raft   Raft
+	errorc chan error
+	// deprecate when we depercate v2 API
+	followerStats *stats.FollowerStats
+
+	msgc chan raftpb.Message
+	// wait for the handling routines
+	wg    sync.WaitGroup
+	stopc chan struct{}
+}
+
+func (p *pipeline) start() {
+	p.stopc = make(chan struct{})
+	p.msgc = make(chan raftpb.Message, pipelineBufSize)
+	p.wg.Add(connPerPipeline)
+	for i := 0; i < connPerPipeline; i++ {
+		go p.handle()
+	}
+
+	if p.tr != nil && p.tr.Logger != nil {
+		p.tr.Logger.Info(
+			"started HTTP pipelining with remote peer",
+			zap.String("local-member-id", p.tr.ID.String()),
+			zap.String("remote-peer-id", p.peerID.String()),
+		)
+	} else {
+		plog.Infof("started HTTP pipelining with peer %s", p.peerID)
+	}
+}
+
+func (p *pipeline) stop() {
+	close(p.stopc)
+	p.wg.Wait()
+
+	if p.tr != nil && p.tr.Logger != nil {
+		p.tr.Logger.Info(
+			"stopped HTTP pipelining with remote peer",
+			zap.String("local-member-id", p.tr.ID.String()),
+			zap.String("remote-peer-id", p.peerID.String()),
+		)
+	} else {
+		plog.Infof("stopped HTTP pipelining with peer %s", p.peerID)
+	}
+}
+
+func (p *pipeline) handle() {
+	defer p.wg.Done()
+
+	for {
+		select {
+		case m := <-p.msgc:
+			start := time.Now()
+			err := p.post(pbutil.MustMarshal(&m))
+			end := time.Now()
+
+			if err != nil {
+				p.status.deactivate(failureType{source: pipelineMsg, action: "write"}, err.Error())
+
+				if m.Type == raftpb.MsgApp && p.followerStats != nil {
+					p.followerStats.Fail()
+				}
+				p.raft.ReportUnreachable(m.To)
+				if isMsgSnap(m) {
+					p.raft.ReportSnapshot(m.To, raft.SnapshotFailure)
+				}
+				sentFailures.WithLabelValues(types.ID(m.To).String()).Inc()
+				continue
+			}
+
+			p.status.activate()
+			if m.Type == raftpb.MsgApp && p.followerStats != nil {
+				p.followerStats.Succ(end.Sub(start))
+			}
+			if isMsgSnap(m) {
+				p.raft.ReportSnapshot(m.To, raft.SnapshotFinish)
+			}
+			sentBytes.WithLabelValues(types.ID(m.To).String()).Add(float64(m.Size()))
+		case <-p.stopc:
+			return
+		}
+	}
+}
+
+// post POSTs a data payload to a url. Returns nil if the POST succeeds,
+// error on any failure.
+func (p *pipeline) post(data []byte) (err error) {
+	u := p.picker.pick()
+	req := createPostRequest(u, RaftPrefix, bytes.NewBuffer(data), "application/protobuf", p.tr.URLs, p.tr.ID, p.tr.ClusterID)
+
+	done := make(chan struct{}, 1)
+	ctx, cancel := context.WithCancel(context.Background())
+	req = req.WithContext(ctx)
+	go func() {
+		select {
+		case <-done:
+		case <-p.stopc:
+			waitSchedule()
+			cancel()
+		}
+	}()
+
+	resp, err := p.tr.pipelineRt.RoundTrip(req)
+	done <- struct{}{}
+	if err != nil {
+		p.picker.unreachable(u)
+		return err
+	}
+	defer resp.Body.Close()
+	b, err := ioutil.ReadAll(resp.Body)
+	if err != nil {
+		p.picker.unreachable(u)
+		return err
+	}
+
+	err = checkPostResponse(resp, b, req, p.peerID)
+	if err != nil {
+		p.picker.unreachable(u)
+		// errMemberRemoved is a critical error since a removed member should
+		// always be stopped. So we use reportCriticalError to report it to errorc.
+		if err == errMemberRemoved {
+			reportCriticalError(err, p.errorc)
+		}
+		return err
+	}
+
+	return nil
+}
+
+// waitSchedule waits other goroutines to be scheduled for a while
+func waitSchedule() { time.Sleep(time.Millisecond) }
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/probing_status.go b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/probing_status.go
new file mode 100644
index 0000000..474d9a0
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/probing_status.go
@@ -0,0 +1,104 @@
+// Copyright 2015 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 rafthttp
+
+import (
+	"time"
+
+	"github.com/prometheus/client_golang/prometheus"
+	"github.com/xiang90/probing"
+	"go.uber.org/zap"
+)
+
+const (
+	// RoundTripperNameRaftMessage is the name of round-tripper that sends
+	// all other Raft messages, other than "snap.Message".
+	RoundTripperNameRaftMessage = "ROUND_TRIPPER_RAFT_MESSAGE"
+	// RoundTripperNameSnapshot is the name of round-tripper that sends merged snapshot message.
+	RoundTripperNameSnapshot = "ROUND_TRIPPER_SNAPSHOT"
+)
+
+var (
+	// proberInterval must be shorter than read timeout.
+	// Or the connection will time-out.
+	proberInterval           = ConnReadTimeout - time.Second
+	statusMonitoringInterval = 30 * time.Second
+	statusErrorInterval      = 5 * time.Second
+)
+
+func addPeerToProber(lg *zap.Logger, p probing.Prober, id string, us []string, roundTripperName string, rttSecProm *prometheus.HistogramVec) {
+	hus := make([]string, len(us))
+	for i := range us {
+		hus[i] = us[i] + ProbingPrefix
+	}
+
+	p.AddHTTP(id, proberInterval, hus)
+
+	s, err := p.Status(id)
+	if err != nil {
+		if lg != nil {
+			lg.Warn("failed to add peer into prober", zap.String("remote-peer-id", id))
+		} else {
+			plog.Errorf("failed to add peer %s into prober", id)
+		}
+		return
+	}
+
+	go monitorProbingStatus(lg, s, id, roundTripperName, rttSecProm)
+}
+
+func monitorProbingStatus(lg *zap.Logger, s probing.Status, id string, roundTripperName string, rttSecProm *prometheus.HistogramVec) {
+	// set the first interval short to log error early.
+	interval := statusErrorInterval
+	for {
+		select {
+		case <-time.After(interval):
+			if !s.Health() {
+				if lg != nil {
+					lg.Warn(
+						"prober detected unhealthy status",
+						zap.String("round-tripper-name", roundTripperName),
+						zap.String("remote-peer-id", id),
+						zap.Duration("rtt", s.SRTT()),
+						zap.Error(s.Err()),
+					)
+				} else {
+					plog.Warningf("health check for peer %s could not connect: %v", id, s.Err())
+				}
+				interval = statusErrorInterval
+			} else {
+				interval = statusMonitoringInterval
+			}
+			if s.ClockDiff() > time.Second {
+				if lg != nil {
+					lg.Warn(
+						"prober found high clock drift",
+						zap.String("round-tripper-name", roundTripperName),
+						zap.String("remote-peer-id", id),
+						zap.Duration("clock-drift", s.ClockDiff()),
+						zap.Duration("rtt", s.SRTT()),
+						zap.Error(s.Err()),
+					)
+				} else {
+					plog.Warningf("the clock difference against peer %s is too high [%v > %v]", id, s.ClockDiff(), time.Second)
+				}
+			}
+			rttSecProm.WithLabelValues(id).Observe(s.SRTT().Seconds())
+
+		case <-s.StopNotify():
+			return
+		}
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/remote.go b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/remote.go
new file mode 100644
index 0000000..1ef2493
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/remote.go
@@ -0,0 +1,99 @@
+// Copyright 2015 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 rafthttp
+
+import (
+	"go.etcd.io/etcd/pkg/types"
+	"go.etcd.io/etcd/raft/raftpb"
+
+	"go.uber.org/zap"
+)
+
+type remote struct {
+	lg       *zap.Logger
+	localID  types.ID
+	id       types.ID
+	status   *peerStatus
+	pipeline *pipeline
+}
+
+func startRemote(tr *Transport, urls types.URLs, id types.ID) *remote {
+	picker := newURLPicker(urls)
+	status := newPeerStatus(tr.Logger, tr.ID, id)
+	pipeline := &pipeline{
+		peerID: id,
+		tr:     tr,
+		picker: picker,
+		status: status,
+		raft:   tr.Raft,
+		errorc: tr.ErrorC,
+	}
+	pipeline.start()
+
+	return &remote{
+		lg:       tr.Logger,
+		localID:  tr.ID,
+		id:       id,
+		status:   status,
+		pipeline: pipeline,
+	}
+}
+
+func (g *remote) send(m raftpb.Message) {
+	select {
+	case g.pipeline.msgc <- m:
+	default:
+		if g.status.isActive() {
+			if g.lg != nil {
+				g.lg.Warn(
+					"dropped internal Raft message since sending buffer is full (overloaded network)",
+					zap.String("message-type", m.Type.String()),
+					zap.String("local-member-id", g.localID.String()),
+					zap.String("from", types.ID(m.From).String()),
+					zap.String("remote-peer-id", g.id.String()),
+					zap.Bool("remote-peer-active", g.status.isActive()),
+				)
+			} else {
+				plog.MergeWarningf("dropped internal raft message to %s since sending buffer is full (bad/overloaded network)", g.id)
+			}
+		} else {
+			if g.lg != nil {
+				g.lg.Warn(
+					"dropped Raft message since sending buffer is full (overloaded network)",
+					zap.String("message-type", m.Type.String()),
+					zap.String("local-member-id", g.localID.String()),
+					zap.String("from", types.ID(m.From).String()),
+					zap.String("remote-peer-id", g.id.String()),
+					zap.Bool("remote-peer-active", g.status.isActive()),
+				)
+			} else {
+				plog.Debugf("dropped %s to %s since sending buffer is full", m.Type, g.id)
+			}
+		}
+		sentFailures.WithLabelValues(types.ID(m.To).String()).Inc()
+	}
+}
+
+func (g *remote) stop() {
+	g.pipeline.stop()
+}
+
+func (g *remote) Pause() {
+	g.stop()
+}
+
+func (g *remote) Resume() {
+	g.pipeline.start()
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/snapshot_sender.go b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/snapshot_sender.go
new file mode 100644
index 0000000..62efb0c
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/snapshot_sender.go
@@ -0,0 +1,207 @@
+// Copyright 2015 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 rafthttp
+
+import (
+	"bytes"
+	"context"
+	"io"
+	"io/ioutil"
+	"net/http"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver/api/snap"
+	"go.etcd.io/etcd/pkg/httputil"
+	pioutil "go.etcd.io/etcd/pkg/ioutil"
+	"go.etcd.io/etcd/pkg/types"
+	"go.etcd.io/etcd/raft"
+
+	"github.com/dustin/go-humanize"
+	"go.uber.org/zap"
+)
+
+var (
+	// timeout for reading snapshot response body
+	snapResponseReadTimeout = 5 * time.Second
+)
+
+type snapshotSender struct {
+	from, to types.ID
+	cid      types.ID
+
+	tr     *Transport
+	picker *urlPicker
+	status *peerStatus
+	r      Raft
+	errorc chan error
+
+	stopc chan struct{}
+}
+
+func newSnapshotSender(tr *Transport, picker *urlPicker, to types.ID, status *peerStatus) *snapshotSender {
+	return &snapshotSender{
+		from:   tr.ID,
+		to:     to,
+		cid:    tr.ClusterID,
+		tr:     tr,
+		picker: picker,
+		status: status,
+		r:      tr.Raft,
+		errorc: tr.ErrorC,
+		stopc:  make(chan struct{}),
+	}
+}
+
+func (s *snapshotSender) stop() { close(s.stopc) }
+
+func (s *snapshotSender) send(merged snap.Message) {
+	start := time.Now()
+
+	m := merged.Message
+	to := types.ID(m.To).String()
+
+	body := createSnapBody(s.tr.Logger, merged)
+	defer body.Close()
+
+	u := s.picker.pick()
+	req := createPostRequest(u, RaftSnapshotPrefix, body, "application/octet-stream", s.tr.URLs, s.from, s.cid)
+
+	if s.tr.Logger != nil {
+		s.tr.Logger.Info(
+			"sending database snapshot",
+			zap.Uint64("snapshot-index", m.Snapshot.Metadata.Index),
+			zap.String("remote-peer-id", to),
+			zap.Int64("bytes", merged.TotalSize),
+			zap.String("size", humanize.Bytes(uint64(merged.TotalSize))),
+		)
+	} else {
+		plog.Infof("start to send database snapshot [index: %d, to %s]...", m.Snapshot.Metadata.Index, types.ID(m.To))
+	}
+
+	snapshotSendInflights.WithLabelValues(to).Inc()
+	defer func() {
+		snapshotSendInflights.WithLabelValues(to).Dec()
+	}()
+
+	err := s.post(req)
+	defer merged.CloseWithError(err)
+	if err != nil {
+		if s.tr.Logger != nil {
+			s.tr.Logger.Warn(
+				"failed to send database snapshot",
+				zap.Uint64("snapshot-index", m.Snapshot.Metadata.Index),
+				zap.String("remote-peer-id", to),
+				zap.Int64("bytes", merged.TotalSize),
+				zap.String("size", humanize.Bytes(uint64(merged.TotalSize))),
+				zap.Error(err),
+			)
+		} else {
+			plog.Warningf("database snapshot [index: %d, to: %s] failed to be sent out (%v)", m.Snapshot.Metadata.Index, types.ID(m.To), err)
+		}
+
+		// errMemberRemoved is a critical error since a removed member should
+		// always be stopped. So we use reportCriticalError to report it to errorc.
+		if err == errMemberRemoved {
+			reportCriticalError(err, s.errorc)
+		}
+
+		s.picker.unreachable(u)
+		s.status.deactivate(failureType{source: sendSnap, action: "post"}, err.Error())
+		s.r.ReportUnreachable(m.To)
+		// report SnapshotFailure to raft state machine. After raft state
+		// machine knows about it, it would pause a while and retry sending
+		// new snapshot message.
+		s.r.ReportSnapshot(m.To, raft.SnapshotFailure)
+		sentFailures.WithLabelValues(to).Inc()
+		snapshotSendFailures.WithLabelValues(to).Inc()
+		return
+	}
+	s.status.activate()
+	s.r.ReportSnapshot(m.To, raft.SnapshotFinish)
+
+	if s.tr.Logger != nil {
+		s.tr.Logger.Info(
+			"sent database snapshot",
+			zap.Uint64("snapshot-index", m.Snapshot.Metadata.Index),
+			zap.String("remote-peer-id", to),
+			zap.Int64("bytes", merged.TotalSize),
+			zap.String("size", humanize.Bytes(uint64(merged.TotalSize))),
+		)
+	} else {
+		plog.Infof("database snapshot [index: %d, to: %s] sent out successfully", m.Snapshot.Metadata.Index, types.ID(m.To))
+	}
+
+	sentBytes.WithLabelValues(to).Add(float64(merged.TotalSize))
+	snapshotSend.WithLabelValues(to).Inc()
+	snapshotSendSeconds.WithLabelValues(to).Observe(time.Since(start).Seconds())
+}
+
+// post posts the given request.
+// It returns nil when request is sent out and processed successfully.
+func (s *snapshotSender) post(req *http.Request) (err error) {
+	ctx, cancel := context.WithCancel(context.Background())
+	req = req.WithContext(ctx)
+	defer cancel()
+
+	type responseAndError struct {
+		resp *http.Response
+		body []byte
+		err  error
+	}
+	result := make(chan responseAndError, 1)
+
+	go func() {
+		resp, err := s.tr.pipelineRt.RoundTrip(req)
+		if err != nil {
+			result <- responseAndError{resp, nil, err}
+			return
+		}
+
+		// close the response body when timeouts.
+		// prevents from reading the body forever when the other side dies right after
+		// successfully receives the request body.
+		time.AfterFunc(snapResponseReadTimeout, func() { httputil.GracefulClose(resp) })
+		body, err := ioutil.ReadAll(resp.Body)
+		result <- responseAndError{resp, body, err}
+	}()
+
+	select {
+	case <-s.stopc:
+		return errStopped
+	case r := <-result:
+		if r.err != nil {
+			return r.err
+		}
+		return checkPostResponse(r.resp, r.body, req, s.to)
+	}
+}
+
+func createSnapBody(lg *zap.Logger, merged snap.Message) io.ReadCloser {
+	buf := new(bytes.Buffer)
+	enc := &messageEncoder{w: buf}
+	// encode raft message
+	if err := enc.encode(&merged.Message); err != nil {
+		if lg != nil {
+			lg.Panic("failed to encode message", zap.Error(err))
+		} else {
+			plog.Panicf("encode message error (%v)", err)
+		}
+	}
+
+	return &pioutil.ReaderAndCloser{
+		Reader: io.MultiReader(buf, merged.ReadCloser),
+		Closer: merged.ReadCloser,
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/stream.go b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/stream.go
new file mode 100644
index 0000000..dcb2223
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/stream.go
@@ -0,0 +1,746 @@
+// Copyright 2015 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 rafthttp
+
+import (
+	"context"
+	"fmt"
+	"io"
+	"io/ioutil"
+	"net/http"
+	"path"
+	"strings"
+	"sync"
+	"time"
+
+	stats "go.etcd.io/etcd/etcdserver/api/v2stats"
+	"go.etcd.io/etcd/pkg/httputil"
+	"go.etcd.io/etcd/pkg/transport"
+	"go.etcd.io/etcd/pkg/types"
+	"go.etcd.io/etcd/raft/raftpb"
+	"go.etcd.io/etcd/version"
+
+	"github.com/coreos/go-semver/semver"
+	"go.uber.org/zap"
+	"golang.org/x/time/rate"
+)
+
+const (
+	streamTypeMessage  streamType = "message"
+	streamTypeMsgAppV2 streamType = "msgappv2"
+
+	streamBufSize = 4096
+)
+
+var (
+	errUnsupportedStreamType = fmt.Errorf("unsupported stream type")
+
+	// the key is in string format "major.minor.patch"
+	supportedStream = map[string][]streamType{
+		"2.0.0": {},
+		"2.1.0": {streamTypeMsgAppV2, streamTypeMessage},
+		"2.2.0": {streamTypeMsgAppV2, streamTypeMessage},
+		"2.3.0": {streamTypeMsgAppV2, streamTypeMessage},
+		"3.0.0": {streamTypeMsgAppV2, streamTypeMessage},
+		"3.1.0": {streamTypeMsgAppV2, streamTypeMessage},
+		"3.2.0": {streamTypeMsgAppV2, streamTypeMessage},
+		"3.3.0": {streamTypeMsgAppV2, streamTypeMessage},
+	}
+)
+
+type streamType string
+
+func (t streamType) endpoint() string {
+	switch t {
+	case streamTypeMsgAppV2:
+		return path.Join(RaftStreamPrefix, "msgapp")
+	case streamTypeMessage:
+		return path.Join(RaftStreamPrefix, "message")
+	default:
+		plog.Panicf("unhandled stream type %v", t)
+		return ""
+	}
+}
+
+func (t streamType) String() string {
+	switch t {
+	case streamTypeMsgAppV2:
+		return "stream MsgApp v2"
+	case streamTypeMessage:
+		return "stream Message"
+	default:
+		return "unknown stream"
+	}
+}
+
+var (
+	// linkHeartbeatMessage is a special message used as heartbeat message in
+	// link layer. It never conflicts with messages from raft because raft
+	// doesn't send out messages without From and To fields.
+	linkHeartbeatMessage = raftpb.Message{Type: raftpb.MsgHeartbeat}
+)
+
+func isLinkHeartbeatMessage(m *raftpb.Message) bool {
+	return m.Type == raftpb.MsgHeartbeat && m.From == 0 && m.To == 0
+}
+
+type outgoingConn struct {
+	t streamType
+	io.Writer
+	http.Flusher
+	io.Closer
+
+	localID types.ID
+	peerID  types.ID
+}
+
+// streamWriter writes messages to the attached outgoingConn.
+type streamWriter struct {
+	lg *zap.Logger
+
+	localID types.ID
+	peerID  types.ID
+
+	status *peerStatus
+	fs     *stats.FollowerStats
+	r      Raft
+
+	mu      sync.Mutex // guard field working and closer
+	closer  io.Closer
+	working bool
+
+	msgc  chan raftpb.Message
+	connc chan *outgoingConn
+	stopc chan struct{}
+	done  chan struct{}
+}
+
+// startStreamWriter creates a streamWrite and starts a long running go-routine that accepts
+// messages and writes to the attached outgoing connection.
+func startStreamWriter(lg *zap.Logger, local, id types.ID, status *peerStatus, fs *stats.FollowerStats, r Raft) *streamWriter {
+	w := &streamWriter{
+		lg: lg,
+
+		localID: local,
+		peerID:  id,
+
+		status: status,
+		fs:     fs,
+		r:      r,
+		msgc:   make(chan raftpb.Message, streamBufSize),
+		connc:  make(chan *outgoingConn),
+		stopc:  make(chan struct{}),
+		done:   make(chan struct{}),
+	}
+	go w.run()
+	return w
+}
+
+func (cw *streamWriter) run() {
+	var (
+		msgc       chan raftpb.Message
+		heartbeatc <-chan time.Time
+		t          streamType
+		enc        encoder
+		flusher    http.Flusher
+		batched    int
+	)
+	tickc := time.NewTicker(ConnReadTimeout / 3)
+	defer tickc.Stop()
+	unflushed := 0
+
+	if cw.lg != nil {
+		cw.lg.Info(
+			"started stream writer with remote peer",
+			zap.String("local-member-id", cw.localID.String()),
+			zap.String("remote-peer-id", cw.peerID.String()),
+		)
+	} else {
+		plog.Infof("started streaming with peer %s (writer)", cw.peerID)
+	}
+
+	for {
+		select {
+		case <-heartbeatc:
+			err := enc.encode(&linkHeartbeatMessage)
+			unflushed += linkHeartbeatMessage.Size()
+			if err == nil {
+				flusher.Flush()
+				batched = 0
+				sentBytes.WithLabelValues(cw.peerID.String()).Add(float64(unflushed))
+				unflushed = 0
+				continue
+			}
+
+			cw.status.deactivate(failureType{source: t.String(), action: "heartbeat"}, err.Error())
+
+			sentFailures.WithLabelValues(cw.peerID.String()).Inc()
+			cw.close()
+			if cw.lg != nil {
+				cw.lg.Warn(
+					"lost TCP streaming connection with remote peer",
+					zap.String("stream-writer-type", t.String()),
+					zap.String("local-member-id", cw.localID.String()),
+					zap.String("remote-peer-id", cw.peerID.String()),
+				)
+			} else {
+				plog.Warningf("lost the TCP streaming connection with peer %s (%s writer)", cw.peerID, t)
+			}
+			heartbeatc, msgc = nil, nil
+
+		case m := <-msgc:
+			err := enc.encode(&m)
+			if err == nil {
+				unflushed += m.Size()
+
+				if len(msgc) == 0 || batched > streamBufSize/2 {
+					flusher.Flush()
+					sentBytes.WithLabelValues(cw.peerID.String()).Add(float64(unflushed))
+					unflushed = 0
+					batched = 0
+				} else {
+					batched++
+				}
+
+				continue
+			}
+
+			cw.status.deactivate(failureType{source: t.String(), action: "write"}, err.Error())
+			cw.close()
+			if cw.lg != nil {
+				cw.lg.Warn(
+					"lost TCP streaming connection with remote peer",
+					zap.String("stream-writer-type", t.String()),
+					zap.String("local-member-id", cw.localID.String()),
+					zap.String("remote-peer-id", cw.peerID.String()),
+				)
+			} else {
+				plog.Warningf("lost the TCP streaming connection with peer %s (%s writer)", cw.peerID, t)
+			}
+			heartbeatc, msgc = nil, nil
+			cw.r.ReportUnreachable(m.To)
+			sentFailures.WithLabelValues(cw.peerID.String()).Inc()
+
+		case conn := <-cw.connc:
+			cw.mu.Lock()
+			closed := cw.closeUnlocked()
+			t = conn.t
+			switch conn.t {
+			case streamTypeMsgAppV2:
+				enc = newMsgAppV2Encoder(conn.Writer, cw.fs)
+			case streamTypeMessage:
+				enc = &messageEncoder{w: conn.Writer}
+			default:
+				plog.Panicf("unhandled stream type %s", conn.t)
+			}
+			if cw.lg != nil {
+				cw.lg.Info(
+					"set message encoder",
+					zap.String("from", conn.localID.String()),
+					zap.String("to", conn.peerID.String()),
+					zap.String("stream-type", t.String()),
+				)
+			}
+			flusher = conn.Flusher
+			unflushed = 0
+			cw.status.activate()
+			cw.closer = conn.Closer
+			cw.working = true
+			cw.mu.Unlock()
+
+			if closed {
+				if cw.lg != nil {
+					cw.lg.Warn(
+						"closed TCP streaming connection with remote peer",
+						zap.String("stream-writer-type", t.String()),
+						zap.String("local-member-id", cw.localID.String()),
+						zap.String("remote-peer-id", cw.peerID.String()),
+					)
+				} else {
+					plog.Warningf("closed an existing TCP streaming connection with peer %s (%s writer)", cw.peerID, t)
+				}
+			}
+			if cw.lg != nil {
+				cw.lg.Warn(
+					"established TCP streaming connection with remote peer",
+					zap.String("stream-writer-type", t.String()),
+					zap.String("local-member-id", cw.localID.String()),
+					zap.String("remote-peer-id", cw.peerID.String()),
+				)
+			} else {
+				plog.Infof("established a TCP streaming connection with peer %s (%s writer)", cw.peerID, t)
+			}
+			heartbeatc, msgc = tickc.C, cw.msgc
+
+		case <-cw.stopc:
+			if cw.close() {
+				if cw.lg != nil {
+					cw.lg.Warn(
+						"closed TCP streaming connection with remote peer",
+						zap.String("stream-writer-type", t.String()),
+						zap.String("remote-peer-id", cw.peerID.String()),
+					)
+				} else {
+					plog.Infof("closed the TCP streaming connection with peer %s (%s writer)", cw.peerID, t)
+				}
+			}
+			if cw.lg != nil {
+				cw.lg.Warn(
+					"stopped TCP streaming connection with remote peer",
+					zap.String("stream-writer-type", t.String()),
+					zap.String("remote-peer-id", cw.peerID.String()),
+				)
+			} else {
+				plog.Infof("stopped streaming with peer %s (writer)", cw.peerID)
+			}
+			close(cw.done)
+			return
+		}
+	}
+}
+
+func (cw *streamWriter) writec() (chan<- raftpb.Message, bool) {
+	cw.mu.Lock()
+	defer cw.mu.Unlock()
+	return cw.msgc, cw.working
+}
+
+func (cw *streamWriter) close() bool {
+	cw.mu.Lock()
+	defer cw.mu.Unlock()
+	return cw.closeUnlocked()
+}
+
+func (cw *streamWriter) closeUnlocked() bool {
+	if !cw.working {
+		return false
+	}
+	if err := cw.closer.Close(); err != nil {
+		if cw.lg != nil {
+			cw.lg.Warn(
+				"failed to close connection with remote peer",
+				zap.String("remote-peer-id", cw.peerID.String()),
+				zap.Error(err),
+			)
+		} else {
+			plog.Errorf("peer %s (writer) connection close error: %v", cw.peerID, err)
+		}
+	}
+	if len(cw.msgc) > 0 {
+		cw.r.ReportUnreachable(uint64(cw.peerID))
+	}
+	cw.msgc = make(chan raftpb.Message, streamBufSize)
+	cw.working = false
+	return true
+}
+
+func (cw *streamWriter) attach(conn *outgoingConn) bool {
+	select {
+	case cw.connc <- conn:
+		return true
+	case <-cw.done:
+		return false
+	}
+}
+
+func (cw *streamWriter) stop() {
+	close(cw.stopc)
+	<-cw.done
+}
+
+// streamReader is a long-running go-routine that dials to the remote stream
+// endpoint and reads messages from the response body returned.
+type streamReader struct {
+	lg *zap.Logger
+
+	peerID types.ID
+	typ    streamType
+
+	tr     *Transport
+	picker *urlPicker
+	status *peerStatus
+	recvc  chan<- raftpb.Message
+	propc  chan<- raftpb.Message
+
+	rl *rate.Limiter // alters the frequency of dial retrial attempts
+
+	errorc chan<- error
+
+	mu     sync.Mutex
+	paused bool
+	closer io.Closer
+
+	ctx    context.Context
+	cancel context.CancelFunc
+	done   chan struct{}
+}
+
+func (cr *streamReader) start() {
+	cr.done = make(chan struct{})
+	if cr.errorc == nil {
+		cr.errorc = cr.tr.ErrorC
+	}
+	if cr.ctx == nil {
+		cr.ctx, cr.cancel = context.WithCancel(context.Background())
+	}
+	go cr.run()
+}
+
+func (cr *streamReader) run() {
+	t := cr.typ
+
+	if cr.lg != nil {
+		cr.lg.Info(
+			"started stream reader with remote peer",
+			zap.String("stream-reader-type", t.String()),
+			zap.String("local-member-id", cr.tr.ID.String()),
+			zap.String("remote-peer-id", cr.peerID.String()),
+		)
+	} else {
+		plog.Infof("started streaming with peer %s (%s reader)", cr.peerID, t)
+	}
+
+	for {
+		rc, err := cr.dial(t)
+		if err != nil {
+			if err != errUnsupportedStreamType {
+				cr.status.deactivate(failureType{source: t.String(), action: "dial"}, err.Error())
+			}
+		} else {
+			cr.status.activate()
+			if cr.lg != nil {
+				cr.lg.Info(
+					"established TCP streaming connection with remote peer",
+					zap.String("stream-reader-type", cr.typ.String()),
+					zap.String("local-member-id", cr.tr.ID.String()),
+					zap.String("remote-peer-id", cr.peerID.String()),
+				)
+			} else {
+				plog.Infof("established a TCP streaming connection with peer %s (%s reader)", cr.peerID, cr.typ)
+			}
+			err = cr.decodeLoop(rc, t)
+			if cr.lg != nil {
+				cr.lg.Warn(
+					"lost TCP streaming connection with remote peer",
+					zap.String("stream-reader-type", cr.typ.String()),
+					zap.String("local-member-id", cr.tr.ID.String()),
+					zap.String("remote-peer-id", cr.peerID.String()),
+					zap.Error(err),
+				)
+			} else {
+				plog.Warningf("lost the TCP streaming connection with peer %s (%s reader)", cr.peerID, cr.typ)
+			}
+			switch {
+			// all data is read out
+			case err == io.EOF:
+			// connection is closed by the remote
+			case transport.IsClosedConnError(err):
+			default:
+				cr.status.deactivate(failureType{source: t.String(), action: "read"}, err.Error())
+			}
+		}
+		// Wait for a while before new dial attempt
+		err = cr.rl.Wait(cr.ctx)
+		if cr.ctx.Err() != nil {
+			if cr.lg != nil {
+				cr.lg.Info(
+					"stopped stream reader with remote peer",
+					zap.String("stream-reader-type", t.String()),
+					zap.String("local-member-id", cr.tr.ID.String()),
+					zap.String("remote-peer-id", cr.peerID.String()),
+				)
+			} else {
+				plog.Infof("stopped streaming with peer %s (%s reader)", cr.peerID, t)
+			}
+			close(cr.done)
+			return
+		}
+		if err != nil {
+			if cr.lg != nil {
+				cr.lg.Warn(
+					"rate limit on stream reader with remote peer",
+					zap.String("stream-reader-type", t.String()),
+					zap.String("local-member-id", cr.tr.ID.String()),
+					zap.String("remote-peer-id", cr.peerID.String()),
+					zap.Error(err),
+				)
+			} else {
+				plog.Errorf("streaming with peer %s (%s reader) rate limiter error: %v", cr.peerID, t, err)
+			}
+		}
+	}
+}
+
+func (cr *streamReader) decodeLoop(rc io.ReadCloser, t streamType) error {
+	var dec decoder
+	cr.mu.Lock()
+	switch t {
+	case streamTypeMsgAppV2:
+		dec = newMsgAppV2Decoder(rc, cr.tr.ID, cr.peerID)
+	case streamTypeMessage:
+		dec = &messageDecoder{r: rc}
+	default:
+		if cr.lg != nil {
+			cr.lg.Panic("unknown stream type", zap.String("type", t.String()))
+		} else {
+			plog.Panicf("unhandled stream type %s", t)
+		}
+	}
+	select {
+	case <-cr.ctx.Done():
+		cr.mu.Unlock()
+		if err := rc.Close(); err != nil {
+			return err
+		}
+		return io.EOF
+	default:
+		cr.closer = rc
+	}
+	cr.mu.Unlock()
+
+	// gofail: labelRaftDropHeartbeat:
+	for {
+		m, err := dec.decode()
+		if err != nil {
+			cr.mu.Lock()
+			cr.close()
+			cr.mu.Unlock()
+			return err
+		}
+
+		// gofail-go: var raftDropHeartbeat struct{}
+		// continue labelRaftDropHeartbeat
+		receivedBytes.WithLabelValues(types.ID(m.From).String()).Add(float64(m.Size()))
+
+		cr.mu.Lock()
+		paused := cr.paused
+		cr.mu.Unlock()
+
+		if paused {
+			continue
+		}
+
+		if isLinkHeartbeatMessage(&m) {
+			// raft is not interested in link layer
+			// heartbeat message, so we should ignore
+			// it.
+			continue
+		}
+
+		recvc := cr.recvc
+		if m.Type == raftpb.MsgProp {
+			recvc = cr.propc
+		}
+
+		select {
+		case recvc <- m:
+		default:
+			if cr.status.isActive() {
+				if cr.lg != nil {
+					cr.lg.Warn(
+						"dropped internal Raft message since receiving buffer is full (overloaded network)",
+						zap.String("message-type", m.Type.String()),
+						zap.String("local-member-id", cr.tr.ID.String()),
+						zap.String("from", types.ID(m.From).String()),
+						zap.String("remote-peer-id", types.ID(m.To).String()),
+						zap.Bool("remote-peer-active", cr.status.isActive()),
+					)
+				} else {
+					plog.MergeWarningf("dropped internal raft message from %s since receiving buffer is full (overloaded network)", types.ID(m.From))
+				}
+			} else {
+				if cr.lg != nil {
+					cr.lg.Warn(
+						"dropped Raft message since receiving buffer is full (overloaded network)",
+						zap.String("message-type", m.Type.String()),
+						zap.String("local-member-id", cr.tr.ID.String()),
+						zap.String("from", types.ID(m.From).String()),
+						zap.String("remote-peer-id", types.ID(m.To).String()),
+						zap.Bool("remote-peer-active", cr.status.isActive()),
+					)
+				} else {
+					plog.Debugf("dropped %s from %s since receiving buffer is full", m.Type, types.ID(m.From))
+				}
+			}
+			recvFailures.WithLabelValues(types.ID(m.From).String()).Inc()
+		}
+	}
+}
+
+func (cr *streamReader) stop() {
+	cr.mu.Lock()
+	cr.cancel()
+	cr.close()
+	cr.mu.Unlock()
+	<-cr.done
+}
+
+func (cr *streamReader) dial(t streamType) (io.ReadCloser, error) {
+	u := cr.picker.pick()
+	uu := u
+	uu.Path = path.Join(t.endpoint(), cr.tr.ID.String())
+
+	if cr.lg != nil {
+		cr.lg.Debug(
+			"dial stream reader",
+			zap.String("from", cr.tr.ID.String()),
+			zap.String("to", cr.peerID.String()),
+			zap.String("address", uu.String()),
+		)
+	}
+	req, err := http.NewRequest("GET", uu.String(), nil)
+	if err != nil {
+		cr.picker.unreachable(u)
+		return nil, fmt.Errorf("failed to make http request to %v (%v)", u, err)
+	}
+	req.Header.Set("X-Server-From", cr.tr.ID.String())
+	req.Header.Set("X-Server-Version", version.Version)
+	req.Header.Set("X-Min-Cluster-Version", version.MinClusterVersion)
+	req.Header.Set("X-Etcd-Cluster-ID", cr.tr.ClusterID.String())
+	req.Header.Set("X-Raft-To", cr.peerID.String())
+
+	setPeerURLsHeader(req, cr.tr.URLs)
+
+	req = req.WithContext(cr.ctx)
+
+	cr.mu.Lock()
+	select {
+	case <-cr.ctx.Done():
+		cr.mu.Unlock()
+		return nil, fmt.Errorf("stream reader is stopped")
+	default:
+	}
+	cr.mu.Unlock()
+
+	resp, err := cr.tr.streamRt.RoundTrip(req)
+	if err != nil {
+		cr.picker.unreachable(u)
+		return nil, err
+	}
+
+	rv := serverVersion(resp.Header)
+	lv := semver.Must(semver.NewVersion(version.Version))
+	if compareMajorMinorVersion(rv, lv) == -1 && !checkStreamSupport(rv, t) {
+		httputil.GracefulClose(resp)
+		cr.picker.unreachable(u)
+		return nil, errUnsupportedStreamType
+	}
+
+	switch resp.StatusCode {
+	case http.StatusGone:
+		httputil.GracefulClose(resp)
+		cr.picker.unreachable(u)
+		reportCriticalError(errMemberRemoved, cr.errorc)
+		return nil, errMemberRemoved
+
+	case http.StatusOK:
+		return resp.Body, nil
+
+	case http.StatusNotFound:
+		httputil.GracefulClose(resp)
+		cr.picker.unreachable(u)
+		return nil, fmt.Errorf("peer %s failed to find local node %s", cr.peerID, cr.tr.ID)
+
+	case http.StatusPreconditionFailed:
+		b, err := ioutil.ReadAll(resp.Body)
+		if err != nil {
+			cr.picker.unreachable(u)
+			return nil, err
+		}
+		httputil.GracefulClose(resp)
+		cr.picker.unreachable(u)
+
+		switch strings.TrimSuffix(string(b), "\n") {
+		case errIncompatibleVersion.Error():
+			if cr.lg != nil {
+				cr.lg.Warn(
+					"request sent was ignored by remote peer due to server version incompatibility",
+					zap.String("local-member-id", cr.tr.ID.String()),
+					zap.String("remote-peer-id", cr.peerID.String()),
+					zap.Error(errIncompatibleVersion),
+				)
+			} else {
+				plog.Errorf("request sent was ignored by peer %s (server version incompatible)", cr.peerID)
+			}
+			return nil, errIncompatibleVersion
+
+		case errClusterIDMismatch.Error():
+			if cr.lg != nil {
+				cr.lg.Warn(
+					"request sent was ignored by remote peer due to cluster ID mismatch",
+					zap.String("remote-peer-id", cr.peerID.String()),
+					zap.String("remote-peer-cluster-id", resp.Header.Get("X-Etcd-Cluster-ID")),
+					zap.String("local-member-id", cr.tr.ID.String()),
+					zap.String("local-member-cluster-id", cr.tr.ClusterID.String()),
+					zap.Error(errClusterIDMismatch),
+				)
+			} else {
+				plog.Errorf("request sent was ignored (cluster ID mismatch: peer[%s]=%s, local=%s)",
+					cr.peerID, resp.Header.Get("X-Etcd-Cluster-ID"), cr.tr.ClusterID)
+			}
+			return nil, errClusterIDMismatch
+
+		default:
+			return nil, fmt.Errorf("unhandled error %q when precondition failed", string(b))
+		}
+
+	default:
+		httputil.GracefulClose(resp)
+		cr.picker.unreachable(u)
+		return nil, fmt.Errorf("unhandled http status %d", resp.StatusCode)
+	}
+}
+
+func (cr *streamReader) close() {
+	if cr.closer != nil {
+		if err := cr.closer.Close(); err != nil {
+			if cr.lg != nil {
+				cr.lg.Warn(
+					"failed to close remote peer connection",
+					zap.String("local-member-id", cr.tr.ID.String()),
+					zap.String("remote-peer-id", cr.peerID.String()),
+					zap.Error(err),
+				)
+			} else {
+				plog.Errorf("peer %s (reader) connection close error: %v", cr.peerID, err)
+			}
+		}
+	}
+	cr.closer = nil
+}
+
+func (cr *streamReader) pause() {
+	cr.mu.Lock()
+	defer cr.mu.Unlock()
+	cr.paused = true
+}
+
+func (cr *streamReader) resume() {
+	cr.mu.Lock()
+	defer cr.mu.Unlock()
+	cr.paused = false
+}
+
+// checkStreamSupport checks whether the stream type is supported in the
+// given version.
+func checkStreamSupport(v *semver.Version, t streamType) bool {
+	nv := &semver.Version{Major: v.Major, Minor: v.Minor}
+	for _, s := range supportedStream[nv.String()] {
+		if s == t {
+			return true
+		}
+	}
+	return false
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/transport.go b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/transport.go
new file mode 100644
index 0000000..7191c3d
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/transport.go
@@ -0,0 +1,467 @@
+// Copyright 2015 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 rafthttp
+
+import (
+	"context"
+	"net/http"
+	"sync"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver/api/snap"
+	stats "go.etcd.io/etcd/etcdserver/api/v2stats"
+	"go.etcd.io/etcd/pkg/logutil"
+	"go.etcd.io/etcd/pkg/transport"
+	"go.etcd.io/etcd/pkg/types"
+	"go.etcd.io/etcd/raft"
+	"go.etcd.io/etcd/raft/raftpb"
+
+	"github.com/coreos/pkg/capnslog"
+	"github.com/xiang90/probing"
+	"go.uber.org/zap"
+	"golang.org/x/time/rate"
+)
+
+var plog = logutil.NewMergeLogger(capnslog.NewPackageLogger("go.etcd.io/etcd", "rafthttp"))
+
+type Raft interface {
+	Process(ctx context.Context, m raftpb.Message) error
+	IsIDRemoved(id uint64) bool
+	ReportUnreachable(id uint64)
+	ReportSnapshot(id uint64, status raft.SnapshotStatus)
+}
+
+type Transporter interface {
+	// Start starts the given Transporter.
+	// Start MUST be called before calling other functions in the interface.
+	Start() error
+	// Handler returns the HTTP handler of the transporter.
+	// A transporter HTTP handler handles the HTTP requests
+	// from remote peers.
+	// The handler MUST be used to handle RaftPrefix(/raft)
+	// endpoint.
+	Handler() http.Handler
+	// Send sends out the given messages to the remote peers.
+	// Each message has a To field, which is an id that maps
+	// to an existing peer in the transport.
+	// If the id cannot be found in the transport, the message
+	// will be ignored.
+	Send(m []raftpb.Message)
+	// SendSnapshot sends out the given snapshot message to a remote peer.
+	// The behavior of SendSnapshot is similar to Send.
+	SendSnapshot(m snap.Message)
+	// AddRemote adds a remote with given peer urls into the transport.
+	// A remote helps newly joined member to catch up the progress of cluster,
+	// and will not be used after that.
+	// It is the caller's responsibility to ensure the urls are all valid,
+	// or it panics.
+	AddRemote(id types.ID, urls []string)
+	// AddPeer adds a peer with given peer urls into the transport.
+	// It is the caller's responsibility to ensure the urls are all valid,
+	// or it panics.
+	// Peer urls are used to connect to the remote peer.
+	AddPeer(id types.ID, urls []string)
+	// RemovePeer removes the peer with given id.
+	RemovePeer(id types.ID)
+	// RemoveAllPeers removes all the existing peers in the transport.
+	RemoveAllPeers()
+	// UpdatePeer updates the peer urls of the peer with the given id.
+	// It is the caller's responsibility to ensure the urls are all valid,
+	// or it panics.
+	UpdatePeer(id types.ID, urls []string)
+	// ActiveSince returns the time that the connection with the peer
+	// of the given id becomes active.
+	// If the connection is active since peer was added, it returns the adding time.
+	// If the connection is currently inactive, it returns zero time.
+	ActiveSince(id types.ID) time.Time
+	// ActivePeers returns the number of active peers.
+	ActivePeers() int
+	// Stop closes the connections and stops the transporter.
+	Stop()
+}
+
+// Transport implements Transporter interface. It provides the functionality
+// to send raft messages to peers, and receive raft messages from peers.
+// User should call Handler method to get a handler to serve requests
+// received from peerURLs.
+// User needs to call Start before calling other functions, and call
+// Stop when the Transport is no longer used.
+type Transport struct {
+	Logger *zap.Logger
+
+	DialTimeout time.Duration // maximum duration before timing out dial of the request
+	// DialRetryFrequency defines the frequency of streamReader dial retrial attempts;
+	// a distinct rate limiter is created per every peer (default value: 10 events/sec)
+	DialRetryFrequency rate.Limit
+
+	TLSInfo transport.TLSInfo // TLS information used when creating connection
+
+	ID          types.ID   // local member ID
+	URLs        types.URLs // local peer URLs
+	ClusterID   types.ID   // raft cluster ID for request validation
+	Raft        Raft       // raft state machine, to which the Transport forwards received messages and reports status
+	Snapshotter *snap.Snapshotter
+	ServerStats *stats.ServerStats // used to record general transportation statistics
+	// used to record transportation statistics with followers when
+	// performing as leader in raft protocol
+	LeaderStats *stats.LeaderStats
+	// ErrorC is used to report detected critical errors, e.g.,
+	// the member has been permanently removed from the cluster
+	// When an error is received from ErrorC, user should stop raft state
+	// machine and thus stop the Transport.
+	ErrorC chan error
+
+	streamRt   http.RoundTripper // roundTripper used by streams
+	pipelineRt http.RoundTripper // roundTripper used by pipelines
+
+	mu      sync.RWMutex         // protect the remote and peer map
+	remotes map[types.ID]*remote // remotes map that helps newly joined member to catch up
+	peers   map[types.ID]Peer    // peers map
+
+	pipelineProber probing.Prober
+	streamProber   probing.Prober
+}
+
+func (t *Transport) Start() error {
+	var err error
+	t.streamRt, err = newStreamRoundTripper(t.TLSInfo, t.DialTimeout)
+	if err != nil {
+		return err
+	}
+	t.pipelineRt, err = NewRoundTripper(t.TLSInfo, t.DialTimeout)
+	if err != nil {
+		return err
+	}
+	t.remotes = make(map[types.ID]*remote)
+	t.peers = make(map[types.ID]Peer)
+	t.pipelineProber = probing.NewProber(t.pipelineRt)
+	t.streamProber = probing.NewProber(t.streamRt)
+
+	// If client didn't provide dial retry frequency, use the default
+	// (100ms backoff between attempts to create a new stream),
+	// so it doesn't bring too much overhead when retry.
+	if t.DialRetryFrequency == 0 {
+		t.DialRetryFrequency = rate.Every(100 * time.Millisecond)
+	}
+	return nil
+}
+
+func (t *Transport) Handler() http.Handler {
+	pipelineHandler := newPipelineHandler(t, t.Raft, t.ClusterID)
+	streamHandler := newStreamHandler(t, t, t.Raft, t.ID, t.ClusterID)
+	snapHandler := newSnapshotHandler(t, t.Raft, t.Snapshotter, t.ClusterID)
+	mux := http.NewServeMux()
+	mux.Handle(RaftPrefix, pipelineHandler)
+	mux.Handle(RaftStreamPrefix+"/", streamHandler)
+	mux.Handle(RaftSnapshotPrefix, snapHandler)
+	mux.Handle(ProbingPrefix, probing.NewHandler())
+	return mux
+}
+
+func (t *Transport) Get(id types.ID) Peer {
+	t.mu.RLock()
+	defer t.mu.RUnlock()
+	return t.peers[id]
+}
+
+func (t *Transport) Send(msgs []raftpb.Message) {
+	for _, m := range msgs {
+		if m.To == 0 {
+			// ignore intentionally dropped message
+			continue
+		}
+		to := types.ID(m.To)
+
+		t.mu.RLock()
+		p, pok := t.peers[to]
+		g, rok := t.remotes[to]
+		t.mu.RUnlock()
+
+		if pok {
+			if m.Type == raftpb.MsgApp {
+				t.ServerStats.SendAppendReq(m.Size())
+			}
+			p.send(m)
+			continue
+		}
+
+		if rok {
+			g.send(m)
+			continue
+		}
+
+		if t.Logger != nil {
+			t.Logger.Debug(
+				"ignored message send request; unknown remote peer target",
+				zap.String("type", m.Type.String()),
+				zap.String("unknown-target-peer-id", to.String()),
+			)
+		} else {
+			plog.Debugf("ignored message %s (sent to unknown peer %s)", m.Type, to)
+		}
+	}
+}
+
+func (t *Transport) Stop() {
+	t.mu.Lock()
+	defer t.mu.Unlock()
+	for _, r := range t.remotes {
+		r.stop()
+	}
+	for _, p := range t.peers {
+		p.stop()
+	}
+	t.pipelineProber.RemoveAll()
+	t.streamProber.RemoveAll()
+	if tr, ok := t.streamRt.(*http.Transport); ok {
+		tr.CloseIdleConnections()
+	}
+	if tr, ok := t.pipelineRt.(*http.Transport); ok {
+		tr.CloseIdleConnections()
+	}
+	t.peers = nil
+	t.remotes = nil
+}
+
+// CutPeer drops messages to the specified peer.
+func (t *Transport) CutPeer(id types.ID) {
+	t.mu.RLock()
+	p, pok := t.peers[id]
+	g, gok := t.remotes[id]
+	t.mu.RUnlock()
+
+	if pok {
+		p.(Pausable).Pause()
+	}
+	if gok {
+		g.Pause()
+	}
+}
+
+// MendPeer recovers the message dropping behavior of the given peer.
+func (t *Transport) MendPeer(id types.ID) {
+	t.mu.RLock()
+	p, pok := t.peers[id]
+	g, gok := t.remotes[id]
+	t.mu.RUnlock()
+
+	if pok {
+		p.(Pausable).Resume()
+	}
+	if gok {
+		g.Resume()
+	}
+}
+
+func (t *Transport) AddRemote(id types.ID, us []string) {
+	t.mu.Lock()
+	defer t.mu.Unlock()
+	if t.remotes == nil {
+		// there's no clean way to shutdown the golang http server
+		// (see: https://github.com/golang/go/issues/4674) before
+		// stopping the transport; ignore any new connections.
+		return
+	}
+	if _, ok := t.peers[id]; ok {
+		return
+	}
+	if _, ok := t.remotes[id]; ok {
+		return
+	}
+	urls, err := types.NewURLs(us)
+	if err != nil {
+		if t.Logger != nil {
+			t.Logger.Panic("failed NewURLs", zap.Strings("urls", us), zap.Error(err))
+		} else {
+			plog.Panicf("newURLs %+v should never fail: %+v", us, err)
+		}
+	}
+	t.remotes[id] = startRemote(t, urls, id)
+
+	if t.Logger != nil {
+		t.Logger.Info(
+			"added new remote peer",
+			zap.String("local-member-id", t.ID.String()),
+			zap.String("remote-peer-id", id.String()),
+			zap.Strings("remote-peer-urls", us),
+		)
+	}
+}
+
+func (t *Transport) AddPeer(id types.ID, us []string) {
+	t.mu.Lock()
+	defer t.mu.Unlock()
+
+	if t.peers == nil {
+		panic("transport stopped")
+	}
+	if _, ok := t.peers[id]; ok {
+		return
+	}
+	urls, err := types.NewURLs(us)
+	if err != nil {
+		if t.Logger != nil {
+			t.Logger.Panic("failed NewURLs", zap.Strings("urls", us), zap.Error(err))
+		} else {
+			plog.Panicf("newURLs %+v should never fail: %+v", us, err)
+		}
+	}
+	fs := t.LeaderStats.Follower(id.String())
+	t.peers[id] = startPeer(t, urls, id, fs)
+	addPeerToProber(t.Logger, t.pipelineProber, id.String(), us, RoundTripperNameSnapshot, rttSec)
+	addPeerToProber(t.Logger, t.streamProber, id.String(), us, RoundTripperNameRaftMessage, rttSec)
+
+	if t.Logger != nil {
+		t.Logger.Info(
+			"added remote peer",
+			zap.String("local-member-id", t.ID.String()),
+			zap.String("remote-peer-id", id.String()),
+			zap.Strings("remote-peer-urls", us),
+		)
+	} else {
+		plog.Infof("added peer %s", id)
+	}
+}
+
+func (t *Transport) RemovePeer(id types.ID) {
+	t.mu.Lock()
+	defer t.mu.Unlock()
+	t.removePeer(id)
+}
+
+func (t *Transport) RemoveAllPeers() {
+	t.mu.Lock()
+	defer t.mu.Unlock()
+	for id := range t.peers {
+		t.removePeer(id)
+	}
+}
+
+// the caller of this function must have the peers mutex.
+func (t *Transport) removePeer(id types.ID) {
+	if peer, ok := t.peers[id]; ok {
+		peer.stop()
+	} else {
+		if t.Logger != nil {
+			t.Logger.Panic("unexpected removal of unknown remote peer", zap.String("remote-peer-id", id.String()))
+		} else {
+			plog.Panicf("unexpected removal of unknown peer '%d'", id)
+		}
+	}
+	delete(t.peers, id)
+	delete(t.LeaderStats.Followers, id.String())
+	t.pipelineProber.Remove(id.String())
+	t.streamProber.Remove(id.String())
+
+	if t.Logger != nil {
+		t.Logger.Info(
+			"removed remote peer",
+			zap.String("local-member-id", t.ID.String()),
+			zap.String("removed-remote-peer-id", id.String()),
+		)
+	} else {
+		plog.Infof("removed peer %s", id)
+	}
+}
+
+func (t *Transport) UpdatePeer(id types.ID, us []string) {
+	t.mu.Lock()
+	defer t.mu.Unlock()
+	// TODO: return error or just panic?
+	if _, ok := t.peers[id]; !ok {
+		return
+	}
+	urls, err := types.NewURLs(us)
+	if err != nil {
+		if t.Logger != nil {
+			t.Logger.Panic("failed NewURLs", zap.Strings("urls", us), zap.Error(err))
+		} else {
+			plog.Panicf("newURLs %+v should never fail: %+v", us, err)
+		}
+	}
+	t.peers[id].update(urls)
+
+	t.pipelineProber.Remove(id.String())
+	addPeerToProber(t.Logger, t.pipelineProber, id.String(), us, RoundTripperNameSnapshot, rttSec)
+	t.streamProber.Remove(id.String())
+	addPeerToProber(t.Logger, t.streamProber, id.String(), us, RoundTripperNameRaftMessage, rttSec)
+
+	if t.Logger != nil {
+		t.Logger.Info(
+			"updated remote peer",
+			zap.String("local-member-id", t.ID.String()),
+			zap.String("updated-remote-peer-id", id.String()),
+			zap.Strings("updated-remote-peer-urls", us),
+		)
+	} else {
+		plog.Infof("updated peer %s", id)
+	}
+}
+
+func (t *Transport) ActiveSince(id types.ID) time.Time {
+	t.mu.RLock()
+	defer t.mu.RUnlock()
+	if p, ok := t.peers[id]; ok {
+		return p.activeSince()
+	}
+	return time.Time{}
+}
+
+func (t *Transport) SendSnapshot(m snap.Message) {
+	t.mu.Lock()
+	defer t.mu.Unlock()
+	p := t.peers[types.ID(m.To)]
+	if p == nil {
+		m.CloseWithError(errMemberNotFound)
+		return
+	}
+	p.sendSnap(m)
+}
+
+// Pausable is a testing interface for pausing transport traffic.
+type Pausable interface {
+	Pause()
+	Resume()
+}
+
+func (t *Transport) Pause() {
+	t.mu.RLock()
+	defer t.mu.RUnlock()
+	for _, p := range t.peers {
+		p.(Pausable).Pause()
+	}
+}
+
+func (t *Transport) Resume() {
+	t.mu.RLock()
+	defer t.mu.RUnlock()
+	for _, p := range t.peers {
+		p.(Pausable).Resume()
+	}
+}
+
+// ActivePeers returns a channel that closes when an initial
+// peer connection has been established. Use this to wait until the
+// first peer connection becomes active.
+func (t *Transport) ActivePeers() (cnt int) {
+	t.mu.RLock()
+	defer t.mu.RUnlock()
+	for _, p := range t.peers {
+		if !p.activeSince().IsZero() {
+			cnt++
+		}
+	}
+	return cnt
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/urlpick.go b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/urlpick.go
new file mode 100644
index 0000000..61ef468
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/urlpick.go
@@ -0,0 +1,57 @@
+// Copyright 2015 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 rafthttp
+
+import (
+	"net/url"
+	"sync"
+
+	"go.etcd.io/etcd/pkg/types"
+)
+
+type urlPicker struct {
+	mu     sync.Mutex // guards urls and picked
+	urls   types.URLs
+	picked int
+}
+
+func newURLPicker(urls types.URLs) *urlPicker {
+	return &urlPicker{
+		urls: urls,
+	}
+}
+
+func (p *urlPicker) update(urls types.URLs) {
+	p.mu.Lock()
+	defer p.mu.Unlock()
+	p.urls = urls
+	p.picked = 0
+}
+
+func (p *urlPicker) pick() url.URL {
+	p.mu.Lock()
+	defer p.mu.Unlock()
+	return p.urls[p.picked]
+}
+
+// unreachable notices the picker that the given url is unreachable,
+// and it should use other possible urls.
+func (p *urlPicker) unreachable(u url.URL) {
+	p.mu.Lock()
+	defer p.mu.Unlock()
+	if u == p.urls[p.picked] {
+		p.picked = (p.picked + 1) % len(p.urls)
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/util.go b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/util.go
new file mode 100644
index 0000000..2093864
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/rafthttp/util.go
@@ -0,0 +1,190 @@
+// Copyright 2015 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 rafthttp
+
+import (
+	"fmt"
+	"io"
+	"net"
+	"net/http"
+	"net/url"
+	"strings"
+	"time"
+
+	"go.etcd.io/etcd/pkg/transport"
+	"go.etcd.io/etcd/pkg/types"
+	"go.etcd.io/etcd/version"
+
+	"github.com/coreos/go-semver/semver"
+)
+
+var (
+	errMemberRemoved  = fmt.Errorf("the member has been permanently removed from the cluster")
+	errMemberNotFound = fmt.Errorf("member not found")
+)
+
+// NewListener returns a listener for raft message transfer between peers.
+// It uses timeout listener to identify broken streams promptly.
+func NewListener(u url.URL, tlsinfo *transport.TLSInfo) (net.Listener, error) {
+	return transport.NewTimeoutListener(u.Host, u.Scheme, tlsinfo, ConnReadTimeout, ConnWriteTimeout)
+}
+
+// NewRoundTripper returns a roundTripper used to send requests
+// to rafthttp listener of remote peers.
+func NewRoundTripper(tlsInfo transport.TLSInfo, dialTimeout time.Duration) (http.RoundTripper, error) {
+	// It uses timeout transport to pair with remote timeout listeners.
+	// It sets no read/write timeout, because message in requests may
+	// take long time to write out before reading out the response.
+	return transport.NewTimeoutTransport(tlsInfo, dialTimeout, 0, 0)
+}
+
+// newStreamRoundTripper returns a roundTripper used to send stream requests
+// to rafthttp listener of remote peers.
+// Read/write timeout is set for stream roundTripper to promptly
+// find out broken status, which minimizes the number of messages
+// sent on broken connection.
+func newStreamRoundTripper(tlsInfo transport.TLSInfo, dialTimeout time.Duration) (http.RoundTripper, error) {
+	return transport.NewTimeoutTransport(tlsInfo, dialTimeout, ConnReadTimeout, ConnWriteTimeout)
+}
+
+// createPostRequest creates a HTTP POST request that sends raft message.
+func createPostRequest(u url.URL, path string, body io.Reader, ct string, urls types.URLs, from, cid types.ID) *http.Request {
+	uu := u
+	uu.Path = path
+	req, err := http.NewRequest("POST", uu.String(), body)
+	if err != nil {
+		plog.Panicf("unexpected new request error (%v)", err)
+	}
+	req.Header.Set("Content-Type", ct)
+	req.Header.Set("X-Server-From", from.String())
+	req.Header.Set("X-Server-Version", version.Version)
+	req.Header.Set("X-Min-Cluster-Version", version.MinClusterVersion)
+	req.Header.Set("X-Etcd-Cluster-ID", cid.String())
+	setPeerURLsHeader(req, urls)
+
+	return req
+}
+
+// checkPostResponse checks the response of the HTTP POST request that sends
+// raft message.
+func checkPostResponse(resp *http.Response, body []byte, req *http.Request, to types.ID) error {
+	switch resp.StatusCode {
+	case http.StatusPreconditionFailed:
+		switch strings.TrimSuffix(string(body), "\n") {
+		case errIncompatibleVersion.Error():
+			plog.Errorf("request sent was ignored by peer %s (server version incompatible)", to)
+			return errIncompatibleVersion
+		case errClusterIDMismatch.Error():
+			plog.Errorf("request sent was ignored (cluster ID mismatch: remote[%s]=%s, local=%s)",
+				to, resp.Header.Get("X-Etcd-Cluster-ID"), req.Header.Get("X-Etcd-Cluster-ID"))
+			return errClusterIDMismatch
+		default:
+			return fmt.Errorf("unhandled error %q when precondition failed", string(body))
+		}
+	case http.StatusForbidden:
+		return errMemberRemoved
+	case http.StatusNoContent:
+		return nil
+	default:
+		return fmt.Errorf("unexpected http status %s while posting to %q", http.StatusText(resp.StatusCode), req.URL.String())
+	}
+}
+
+// reportCriticalError reports the given error through sending it into
+// the given error channel.
+// If the error channel is filled up when sending error, it drops the error
+// because the fact that error has happened is reported, which is
+// good enough.
+func reportCriticalError(err error, errc chan<- error) {
+	select {
+	case errc <- err:
+	default:
+	}
+}
+
+// compareMajorMinorVersion returns an integer comparing two versions based on
+// their major and minor version. The result will be 0 if a==b, -1 if a < b,
+// and 1 if a > b.
+func compareMajorMinorVersion(a, b *semver.Version) int {
+	na := &semver.Version{Major: a.Major, Minor: a.Minor}
+	nb := &semver.Version{Major: b.Major, Minor: b.Minor}
+	switch {
+	case na.LessThan(*nb):
+		return -1
+	case nb.LessThan(*na):
+		return 1
+	default:
+		return 0
+	}
+}
+
+// serverVersion returns the server version from the given header.
+func serverVersion(h http.Header) *semver.Version {
+	verStr := h.Get("X-Server-Version")
+	// backward compatibility with etcd 2.0
+	if verStr == "" {
+		verStr = "2.0.0"
+	}
+	return semver.Must(semver.NewVersion(verStr))
+}
+
+// serverVersion returns the min cluster version from the given header.
+func minClusterVersion(h http.Header) *semver.Version {
+	verStr := h.Get("X-Min-Cluster-Version")
+	// backward compatibility with etcd 2.0
+	if verStr == "" {
+		verStr = "2.0.0"
+	}
+	return semver.Must(semver.NewVersion(verStr))
+}
+
+// checkVersionCompatibility checks whether the given version is compatible
+// with the local version.
+func checkVersionCompatibility(name string, server, minCluster *semver.Version) (
+	localServer *semver.Version,
+	localMinCluster *semver.Version,
+	err error) {
+	localServer = semver.Must(semver.NewVersion(version.Version))
+	localMinCluster = semver.Must(semver.NewVersion(version.MinClusterVersion))
+	if compareMajorMinorVersion(server, localMinCluster) == -1 {
+		return localServer, localMinCluster, fmt.Errorf("remote version is too low: remote[%s]=%s, local=%s", name, server, localServer)
+	}
+	if compareMajorMinorVersion(minCluster, localServer) == 1 {
+		return localServer, localMinCluster, fmt.Errorf("local version is too low: remote[%s]=%s, local=%s", name, server, localServer)
+	}
+	return localServer, localMinCluster, nil
+}
+
+// setPeerURLsHeader reports local urls for peer discovery
+func setPeerURLsHeader(req *http.Request, urls types.URLs) {
+	if urls == nil {
+		// often not set in unit tests
+		return
+	}
+	peerURLs := make([]string, urls.Len())
+	for i := range urls {
+		peerURLs[i] = urls[i].String()
+	}
+	req.Header.Set("X-PeerURLs", strings.Join(peerURLs, ","))
+}
+
+// addRemoteFromRequest adds a remote peer according to an http request header
+func addRemoteFromRequest(tr Transporter, r *http.Request) {
+	if from, err := types.IDFromString(r.Header.Get("X-Server-From")); err == nil {
+		if urls := r.Header.Get("X-PeerURLs"); urls != "" {
+			tr.AddRemote(from, strings.Split(urls, ","))
+		}
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/snap/db.go b/vendor/go.etcd.io/etcd/etcdserver/api/snap/db.go
new file mode 100644
index 0000000..3002ccd
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/snap/db.go
@@ -0,0 +1,104 @@
+// Copyright 2015 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 snap
+
+import (
+	"errors"
+	"fmt"
+	"io"
+	"io/ioutil"
+	"os"
+	"path/filepath"
+	"time"
+
+	"go.etcd.io/etcd/pkg/fileutil"
+
+	humanize "github.com/dustin/go-humanize"
+	"go.uber.org/zap"
+)
+
+var ErrNoDBSnapshot = errors.New("snap: snapshot file doesn't exist")
+
+// SaveDBFrom saves snapshot of the database from the given reader. It
+// guarantees the save operation is atomic.
+func (s *Snapshotter) SaveDBFrom(r io.Reader, id uint64) (int64, error) {
+	start := time.Now()
+
+	f, err := ioutil.TempFile(s.dir, "tmp")
+	if err != nil {
+		return 0, err
+	}
+	var n int64
+	n, err = io.Copy(f, r)
+	if err == nil {
+		fsyncStart := time.Now()
+		err = fileutil.Fsync(f)
+		snapDBFsyncSec.Observe(time.Since(fsyncStart).Seconds())
+	}
+	f.Close()
+	if err != nil {
+		os.Remove(f.Name())
+		return n, err
+	}
+	fn := s.dbFilePath(id)
+	if fileutil.Exist(fn) {
+		os.Remove(f.Name())
+		return n, nil
+	}
+	err = os.Rename(f.Name(), fn)
+	if err != nil {
+		os.Remove(f.Name())
+		return n, err
+	}
+
+	if s.lg != nil {
+		s.lg.Info(
+			"saved database snapshot to disk",
+			zap.String("path", fn),
+			zap.Int64("bytes", n),
+			zap.String("size", humanize.Bytes(uint64(n))),
+		)
+	} else {
+		plog.Infof("saved database snapshot to disk [total bytes: %d]", n)
+	}
+
+	snapDBSaveSec.Observe(time.Since(start).Seconds())
+	return n, nil
+}
+
+// DBFilePath returns the file path for the snapshot of the database with
+// given id. If the snapshot does not exist, it returns error.
+func (s *Snapshotter) DBFilePath(id uint64) (string, error) {
+	if _, err := fileutil.ReadDir(s.dir); err != nil {
+		return "", err
+	}
+	fn := s.dbFilePath(id)
+	if fileutil.Exist(fn) {
+		return fn, nil
+	}
+	if s.lg != nil {
+		s.lg.Warn(
+			"failed to find [SNAPSHOT-INDEX].snap.db",
+			zap.Uint64("snapshot-index", id),
+			zap.String("snapshot-file-path", fn),
+			zap.Error(ErrNoDBSnapshot),
+		)
+	}
+	return "", ErrNoDBSnapshot
+}
+
+func (s *Snapshotter) dbFilePath(id uint64) string {
+	return filepath.Join(s.dir, fmt.Sprintf("%016x.snap.db", id))
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/snap/doc.go b/vendor/go.etcd.io/etcd/etcdserver/api/snap/doc.go
new file mode 100644
index 0000000..dcc5db5
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/snap/doc.go
@@ -0,0 +1,17 @@
+// Copyright 2015 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 snap handles Raft nodes' states with snapshots.
+// The snapshot logic is internal to etcd server and raft package.
+package snap
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/snap/message.go b/vendor/go.etcd.io/etcd/etcdserver/api/snap/message.go
new file mode 100644
index 0000000..c1151e2
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/snap/message.go
@@ -0,0 +1,64 @@
+// Copyright 2015 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 snap
+
+import (
+	"io"
+
+	"go.etcd.io/etcd/pkg/ioutil"
+	"go.etcd.io/etcd/raft/raftpb"
+)
+
+// Message is a struct that contains a raft Message and a ReadCloser. The type
+// of raft message MUST be MsgSnap, which contains the raft meta-data and an
+// additional data []byte field that contains the snapshot of the actual state
+// machine.
+// Message contains the ReadCloser field for handling large snapshot. This avoid
+// copying the entire snapshot into a byte array, which consumes a lot of memory.
+//
+// User of Message should close the Message after sending it.
+type Message struct {
+	raftpb.Message
+	ReadCloser io.ReadCloser
+	TotalSize  int64
+	closeC     chan bool
+}
+
+func NewMessage(rs raftpb.Message, rc io.ReadCloser, rcSize int64) *Message {
+	return &Message{
+		Message:    rs,
+		ReadCloser: ioutil.NewExactReadCloser(rc, rcSize),
+		TotalSize:  int64(rs.Size()) + rcSize,
+		closeC:     make(chan bool, 1),
+	}
+}
+
+// CloseNotify returns a channel that receives a single value
+// when the message sent is finished. true indicates the sent
+// is successful.
+func (m Message) CloseNotify() <-chan bool {
+	return m.closeC
+}
+
+func (m Message) CloseWithError(err error) {
+	if cerr := m.ReadCloser.Close(); cerr != nil {
+		err = cerr
+	}
+	if err == nil {
+		m.closeC <- true
+	} else {
+		m.closeC <- false
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/snap/metrics.go b/vendor/go.etcd.io/etcd/etcdserver/api/snap/metrics.go
new file mode 100644
index 0000000..2affecf
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/snap/metrics.go
@@ -0,0 +1,82 @@
+// Copyright 2015 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 snap
+
+import "github.com/prometheus/client_golang/prometheus"
+
+var (
+	snapMarshallingSec = prometheus.NewHistogram(prometheus.HistogramOpts{
+		Namespace: "etcd_debugging",
+		Subsystem: "snap",
+		Name:      "save_marshalling_duration_seconds",
+		Help:      "The marshalling cost distributions of save called by snapshot.",
+
+		// lowest bucket start of upper bound 0.001 sec (1 ms) with factor 2
+		// highest bucket start of 0.001 sec * 2^13 == 8.192 sec
+		Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
+	})
+
+	snapSaveSec = prometheus.NewHistogram(prometheus.HistogramOpts{
+		Namespace: "etcd_debugging",
+		Subsystem: "snap",
+		Name:      "save_total_duration_seconds",
+		Help:      "The total latency distributions of save called by snapshot.",
+
+		// lowest bucket start of upper bound 0.001 sec (1 ms) with factor 2
+		// highest bucket start of 0.001 sec * 2^13 == 8.192 sec
+		Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
+	})
+
+	snapFsyncSec = prometheus.NewHistogram(prometheus.HistogramOpts{
+		Namespace: "etcd",
+		Subsystem: "snap",
+		Name:      "fsync_duration_seconds",
+		Help:      "The latency distributions of fsync called by snap.",
+
+		// lowest bucket start of upper bound 0.001 sec (1 ms) with factor 2
+		// highest bucket start of 0.001 sec * 2^13 == 8.192 sec
+		Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
+	})
+
+	snapDBSaveSec = prometheus.NewHistogram(prometheus.HistogramOpts{
+		Namespace: "etcd",
+		Subsystem: "snap_db",
+		Name:      "save_total_duration_seconds",
+		Help:      "The total latency distributions of v3 snapshot save",
+
+		// lowest bucket start of upper bound 0.1 sec (100 ms) with factor 2
+		// highest bucket start of 0.1 sec * 2^9 == 51.2 sec
+		Buckets: prometheus.ExponentialBuckets(0.1, 2, 10),
+	})
+
+	snapDBFsyncSec = prometheus.NewHistogram(prometheus.HistogramOpts{
+		Namespace: "etcd",
+		Subsystem: "snap_db",
+		Name:      "fsync_duration_seconds",
+		Help:      "The latency distributions of fsyncing .snap.db file",
+
+		// lowest bucket start of upper bound 0.001 sec (1 ms) with factor 2
+		// highest bucket start of 0.001 sec * 2^13 == 8.192 sec
+		Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
+	})
+)
+
+func init() {
+	prometheus.MustRegister(snapMarshallingSec)
+	prometheus.MustRegister(snapSaveSec)
+	prometheus.MustRegister(snapFsyncSec)
+	prometheus.MustRegister(snapDBSaveSec)
+	prometheus.MustRegister(snapDBFsyncSec)
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/snap/snappb/snap.pb.go b/vendor/go.etcd.io/etcd/etcdserver/api/snap/snappb/snap.pb.go
new file mode 100644
index 0000000..e72b577
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/snap/snappb/snap.pb.go
@@ -0,0 +1,336 @@
+// Code generated by protoc-gen-gogo. DO NOT EDIT.
+// source: snap.proto
+
+/*
+	Package snappb is a generated protocol buffer package.
+
+	It is generated from these files:
+		snap.proto
+
+	It has these top-level messages:
+		Snapshot
+*/
+package snappb
+
+import (
+	"fmt"
+
+	proto "github.com/golang/protobuf/proto"
+
+	math "math"
+
+	_ "github.com/gogo/protobuf/gogoproto"
+
+	io "io"
+)
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
+
+type Snapshot struct {
+	Crc              uint32 `protobuf:"varint,1,opt,name=crc" json:"crc"`
+	Data             []byte `protobuf:"bytes,2,opt,name=data" json:"data,omitempty"`
+	XXX_unrecognized []byte `json:"-"`
+}
+
+func (m *Snapshot) Reset()                    { *m = Snapshot{} }
+func (m *Snapshot) String() string            { return proto.CompactTextString(m) }
+func (*Snapshot) ProtoMessage()               {}
+func (*Snapshot) Descriptor() ([]byte, []int) { return fileDescriptorSnap, []int{0} }
+
+func init() {
+	proto.RegisterType((*Snapshot)(nil), "snappb.snapshot")
+}
+func (m *Snapshot) 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 *Snapshot) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	dAtA[i] = 0x8
+	i++
+	i = encodeVarintSnap(dAtA, i, uint64(m.Crc))
+	if m.Data != nil {
+		dAtA[i] = 0x12
+		i++
+		i = encodeVarintSnap(dAtA, i, uint64(len(m.Data)))
+		i += copy(dAtA[i:], m.Data)
+	}
+	if m.XXX_unrecognized != nil {
+		i += copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	return i, nil
+}
+
+func encodeVarintSnap(dAtA []byte, offset int, v uint64) int {
+	for v >= 1<<7 {
+		dAtA[offset] = uint8(v&0x7f | 0x80)
+		v >>= 7
+		offset++
+	}
+	dAtA[offset] = uint8(v)
+	return offset + 1
+}
+func (m *Snapshot) Size() (n int) {
+	var l int
+	_ = l
+	n += 1 + sovSnap(uint64(m.Crc))
+	if m.Data != nil {
+		l = len(m.Data)
+		n += 1 + l + sovSnap(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func sovSnap(x uint64) (n int) {
+	for {
+		n++
+		x >>= 7
+		if x == 0 {
+			break
+		}
+	}
+	return n
+}
+func sozSnap(x uint64) (n int) {
+	return sovSnap(uint64((x << 1) ^ uint64((int64(x) >> 63))))
+}
+func (m *Snapshot) 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 ErrIntOverflowSnap
+			}
+			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: snapshot: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: snapshot: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Crc", wireType)
+			}
+			m.Crc = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowSnap
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Crc |= (uint32(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowSnap
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthSnap
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Data = append(m.Data[:0], dAtA[iNdEx:postIndex]...)
+			if m.Data == nil {
+				m.Data = []byte{}
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipSnap(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthSnap
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...)
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func skipSnap(dAtA []byte) (n int, err error) {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return 0, ErrIntOverflowSnap
+			}
+			if iNdEx >= l {
+				return 0, io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= (uint64(b) & 0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		wireType := int(wire & 0x7)
+		switch wireType {
+		case 0:
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return 0, ErrIntOverflowSnap
+				}
+				if iNdEx >= l {
+					return 0, io.ErrUnexpectedEOF
+				}
+				iNdEx++
+				if dAtA[iNdEx-1] < 0x80 {
+					break
+				}
+			}
+			return iNdEx, nil
+		case 1:
+			iNdEx += 8
+			return iNdEx, nil
+		case 2:
+			var length int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return 0, ErrIntOverflowSnap
+				}
+				if iNdEx >= l {
+					return 0, io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				length |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			iNdEx += length
+			if length < 0 {
+				return 0, ErrInvalidLengthSnap
+			}
+			return iNdEx, nil
+		case 3:
+			for {
+				var innerWire uint64
+				var start int = iNdEx
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return 0, ErrIntOverflowSnap
+					}
+					if iNdEx >= l {
+						return 0, io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					innerWire |= (uint64(b) & 0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				innerWireType := int(innerWire & 0x7)
+				if innerWireType == 4 {
+					break
+				}
+				next, err := skipSnap(dAtA[start:])
+				if err != nil {
+					return 0, err
+				}
+				iNdEx = start + next
+			}
+			return iNdEx, nil
+		case 4:
+			return iNdEx, nil
+		case 5:
+			iNdEx += 4
+			return iNdEx, nil
+		default:
+			return 0, fmt.Errorf("proto: illegal wireType %d", wireType)
+		}
+	}
+	panic("unreachable")
+}
+
+var (
+	ErrInvalidLengthSnap = fmt.Errorf("proto: negative length found during unmarshaling")
+	ErrIntOverflowSnap   = fmt.Errorf("proto: integer overflow")
+)
+
+func init() { proto.RegisterFile("snap.proto", fileDescriptorSnap) }
+
+var fileDescriptorSnap = []byte{
+	// 126 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0xe2, 0x2a, 0xce, 0x4b, 0x2c,
+	0xd0, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x62, 0x03, 0xb1, 0x0b, 0x92, 0xa4, 0x44, 0xd2, 0xf3,
+	0xd3, 0xf3, 0xc1, 0x42, 0xfa, 0x20, 0x16, 0x44, 0x56, 0xc9, 0x8c, 0x8b, 0x03, 0x24, 0x5f, 0x9c,
+	0x91, 0x5f, 0x22, 0x24, 0xc6, 0xc5, 0x9c, 0x5c, 0x94, 0x2c, 0xc1, 0xa8, 0xc0, 0xa8, 0xc1, 0xeb,
+	0xc4, 0x72, 0xe2, 0x9e, 0x3c, 0x43, 0x10, 0x48, 0x40, 0x48, 0x88, 0x8b, 0x25, 0x25, 0xb1, 0x24,
+	0x51, 0x82, 0x49, 0x81, 0x51, 0x83, 0x27, 0x08, 0xcc, 0x76, 0x12, 0x39, 0xf1, 0x50, 0x8e, 0xe1,
+	0xc4, 0x23, 0x39, 0xc6, 0x0b, 0x8f, 0xe4, 0x18, 0x1f, 0x3c, 0x92, 0x63, 0x9c, 0xf1, 0x58, 0x8e,
+	0x01, 0x10, 0x00, 0x00, 0xff, 0xff, 0xd8, 0x0f, 0x32, 0xb2, 0x78, 0x00, 0x00, 0x00,
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/snap/snappb/snap.proto b/vendor/go.etcd.io/etcd/etcdserver/api/snap/snappb/snap.proto
new file mode 100644
index 0000000..cd3d21d
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/snap/snappb/snap.proto
@@ -0,0 +1,14 @@
+syntax = "proto2";
+package snappb;
+
+import "gogoproto/gogo.proto";
+
+option (gogoproto.marshaler_all) = true;
+option (gogoproto.sizer_all) = true;
+option (gogoproto.unmarshaler_all) = true;
+option (gogoproto.goproto_getters_all) = false;
+
+message snapshot {
+	optional uint32 crc  = 1 [(gogoproto.nullable) = false];
+	optional bytes data  = 2;
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/snap/snapshotter.go b/vendor/go.etcd.io/etcd/etcdserver/api/snap/snapshotter.go
new file mode 100644
index 0000000..7e79333
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/snap/snapshotter.go
@@ -0,0 +1,255 @@
+// Copyright 2015 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 snap
+
+import (
+	"errors"
+	"fmt"
+	"hash/crc32"
+	"io/ioutil"
+	"os"
+	"path/filepath"
+	"sort"
+	"strings"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver/api/snap/snappb"
+	pioutil "go.etcd.io/etcd/pkg/ioutil"
+	"go.etcd.io/etcd/pkg/pbutil"
+	"go.etcd.io/etcd/raft"
+	"go.etcd.io/etcd/raft/raftpb"
+
+	"github.com/coreos/pkg/capnslog"
+	"go.uber.org/zap"
+)
+
+const snapSuffix = ".snap"
+
+var (
+	plog = capnslog.NewPackageLogger("go.etcd.io/etcd/v3", "snap")
+
+	ErrNoSnapshot    = errors.New("snap: no available snapshot")
+	ErrEmptySnapshot = errors.New("snap: empty snapshot")
+	ErrCRCMismatch   = errors.New("snap: crc mismatch")
+	crcTable         = crc32.MakeTable(crc32.Castagnoli)
+
+	// A map of valid files that can be present in the snap folder.
+	validFiles = map[string]bool{
+		"db": true,
+	}
+)
+
+type Snapshotter struct {
+	lg  *zap.Logger
+	dir string
+}
+
+func New(lg *zap.Logger, dir string) *Snapshotter {
+	return &Snapshotter{
+		lg:  lg,
+		dir: dir,
+	}
+}
+
+func (s *Snapshotter) SaveSnap(snapshot raftpb.Snapshot) error {
+	if raft.IsEmptySnap(snapshot) {
+		return nil
+	}
+	return s.save(&snapshot)
+}
+
+func (s *Snapshotter) save(snapshot *raftpb.Snapshot) error {
+	start := time.Now()
+
+	fname := fmt.Sprintf("%016x-%016x%s", snapshot.Metadata.Term, snapshot.Metadata.Index, snapSuffix)
+	b := pbutil.MustMarshal(snapshot)
+	crc := crc32.Update(0, crcTable, b)
+	snap := snappb.Snapshot{Crc: crc, Data: b}
+	d, err := snap.Marshal()
+	if err != nil {
+		return err
+	}
+	snapMarshallingSec.Observe(time.Since(start).Seconds())
+
+	spath := filepath.Join(s.dir, fname)
+
+	fsyncStart := time.Now()
+	err = pioutil.WriteAndSyncFile(spath, d, 0666)
+	snapFsyncSec.Observe(time.Since(fsyncStart).Seconds())
+
+	if err != nil {
+		if s.lg != nil {
+			s.lg.Warn("failed to write a snap file", zap.String("path", spath), zap.Error(err))
+		}
+		rerr := os.Remove(spath)
+		if rerr != nil {
+			if s.lg != nil {
+				s.lg.Warn("failed to remove a broken snap file", zap.String("path", spath), zap.Error(err))
+			} else {
+				plog.Errorf("failed to remove broken snapshot file %s", spath)
+			}
+		}
+		return err
+	}
+
+	snapSaveSec.Observe(time.Since(start).Seconds())
+	return nil
+}
+
+func (s *Snapshotter) Load() (*raftpb.Snapshot, error) {
+	names, err := s.snapNames()
+	if err != nil {
+		return nil, err
+	}
+	var snap *raftpb.Snapshot
+	for _, name := range names {
+		if snap, err = loadSnap(s.lg, s.dir, name); err == nil {
+			break
+		}
+	}
+	if err != nil {
+		return nil, ErrNoSnapshot
+	}
+	return snap, nil
+}
+
+func loadSnap(lg *zap.Logger, dir, name string) (*raftpb.Snapshot, error) {
+	fpath := filepath.Join(dir, name)
+	snap, err := Read(lg, fpath)
+	if err != nil {
+		brokenPath := fpath + ".broken"
+		if lg != nil {
+			lg.Warn("failed to read a snap file", zap.String("path", fpath), zap.Error(err))
+		}
+		if rerr := os.Rename(fpath, brokenPath); rerr != nil {
+			if lg != nil {
+				lg.Warn("failed to rename a broken snap file", zap.String("path", fpath), zap.String("broken-path", brokenPath), zap.Error(rerr))
+			} else {
+				plog.Warningf("cannot rename broken snapshot file %v to %v: %v", fpath, brokenPath, rerr)
+			}
+		} else {
+			if lg != nil {
+				lg.Warn("renamed to a broken snap file", zap.String("path", fpath), zap.String("broken-path", brokenPath))
+			}
+		}
+	}
+	return snap, err
+}
+
+// Read reads the snapshot named by snapname and returns the snapshot.
+func Read(lg *zap.Logger, snapname string) (*raftpb.Snapshot, error) {
+	b, err := ioutil.ReadFile(snapname)
+	if err != nil {
+		if lg != nil {
+			lg.Warn("failed to read a snap file", zap.String("path", snapname), zap.Error(err))
+		} else {
+			plog.Errorf("cannot read file %v: %v", snapname, err)
+		}
+		return nil, err
+	}
+
+	if len(b) == 0 {
+		if lg != nil {
+			lg.Warn("failed to read empty snapshot file", zap.String("path", snapname))
+		} else {
+			plog.Errorf("unexpected empty snapshot")
+		}
+		return nil, ErrEmptySnapshot
+	}
+
+	var serializedSnap snappb.Snapshot
+	if err = serializedSnap.Unmarshal(b); err != nil {
+		if lg != nil {
+			lg.Warn("failed to unmarshal snappb.Snapshot", zap.String("path", snapname), zap.Error(err))
+		} else {
+			plog.Errorf("corrupted snapshot file %v: %v", snapname, err)
+		}
+		return nil, err
+	}
+
+	if len(serializedSnap.Data) == 0 || serializedSnap.Crc == 0 {
+		if lg != nil {
+			lg.Warn("failed to read empty snapshot data", zap.String("path", snapname))
+		} else {
+			plog.Errorf("unexpected empty snapshot")
+		}
+		return nil, ErrEmptySnapshot
+	}
+
+	crc := crc32.Update(0, crcTable, serializedSnap.Data)
+	if crc != serializedSnap.Crc {
+		if lg != nil {
+			lg.Warn("snap file is corrupt",
+				zap.String("path", snapname),
+				zap.Uint32("prev-crc", serializedSnap.Crc),
+				zap.Uint32("new-crc", crc),
+			)
+		} else {
+			plog.Errorf("corrupted snapshot file %v: crc mismatch", snapname)
+		}
+		return nil, ErrCRCMismatch
+	}
+
+	var snap raftpb.Snapshot
+	if err = snap.Unmarshal(serializedSnap.Data); err != nil {
+		if lg != nil {
+			lg.Warn("failed to unmarshal raftpb.Snapshot", zap.String("path", snapname), zap.Error(err))
+		} else {
+			plog.Errorf("corrupted snapshot file %v: %v", snapname, err)
+		}
+		return nil, err
+	}
+	return &snap, nil
+}
+
+// snapNames returns the filename of the snapshots in logical time order (from newest to oldest).
+// If there is no available snapshots, an ErrNoSnapshot will be returned.
+func (s *Snapshotter) snapNames() ([]string, error) {
+	dir, err := os.Open(s.dir)
+	if err != nil {
+		return nil, err
+	}
+	defer dir.Close()
+	names, err := dir.Readdirnames(-1)
+	if err != nil {
+		return nil, err
+	}
+	snaps := checkSuffix(s.lg, names)
+	if len(snaps) == 0 {
+		return nil, ErrNoSnapshot
+	}
+	sort.Sort(sort.Reverse(sort.StringSlice(snaps)))
+	return snaps, nil
+}
+
+func checkSuffix(lg *zap.Logger, names []string) []string {
+	snaps := []string{}
+	for i := range names {
+		if strings.HasSuffix(names[i], snapSuffix) {
+			snaps = append(snaps, names[i])
+		} else {
+			// If we find a file which is not a snapshot then check if it's
+			// a vaild file. If not throw out a warning.
+			if _, ok := validFiles[names[i]]; !ok {
+				if lg != nil {
+					lg.Warn("found unexpected non-snap file; skipping", zap.String("path", names[i]))
+				} else {
+					plog.Warningf("skipped unexpected non snapshot file %v", names[i])
+				}
+			}
+		}
+	}
+	return snaps
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2auth/auth.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2auth/auth.go
new file mode 100644
index 0000000..b438074
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2auth/auth.go
@@ -0,0 +1,736 @@
+// Copyright 2015 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 v2auth implements etcd authentication.
+package v2auth
+
+import (
+	"context"
+	"encoding/json"
+	"fmt"
+	"net/http"
+	"path"
+	"reflect"
+	"sort"
+	"strings"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/api/v2error"
+	"go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/pkg/types"
+
+	"github.com/coreos/pkg/capnslog"
+	"go.uber.org/zap"
+	"golang.org/x/crypto/bcrypt"
+)
+
+const (
+	// StorePermsPrefix is the internal prefix of the storage layer dedicated to storing user data.
+	StorePermsPrefix = "/2"
+
+	// RootRoleName is the name of the ROOT role, with privileges to manage the cluster.
+	RootRoleName = "root"
+
+	// GuestRoleName is the name of the role that defines the privileges of an unauthenticated user.
+	GuestRoleName = "guest"
+)
+
+var (
+	plog = capnslog.NewPackageLogger("go.etcd.io/etcd/v3", "etcdserver/auth")
+)
+
+var rootRole = Role{
+	Role: RootRoleName,
+	Permissions: Permissions{
+		KV: RWPermission{
+			Read:  []string{"/*"},
+			Write: []string{"/*"},
+		},
+	},
+}
+
+var guestRole = Role{
+	Role: GuestRoleName,
+	Permissions: Permissions{
+		KV: RWPermission{
+			Read:  []string{"/*"},
+			Write: []string{"/*"},
+		},
+	},
+}
+
+type doer interface {
+	Do(context.Context, etcdserverpb.Request) (etcdserver.Response, error)
+}
+
+type Store interface {
+	AllUsers() ([]string, error)
+	GetUser(name string) (User, error)
+	CreateOrUpdateUser(user User) (out User, created bool, err error)
+	CreateUser(user User) (User, error)
+	DeleteUser(name string) error
+	UpdateUser(user User) (User, error)
+	AllRoles() ([]string, error)
+	GetRole(name string) (Role, error)
+	CreateRole(role Role) error
+	DeleteRole(name string) error
+	UpdateRole(role Role) (Role, error)
+	AuthEnabled() bool
+	EnableAuth() error
+	DisableAuth() error
+	PasswordStore
+}
+
+type PasswordStore interface {
+	CheckPassword(user User, password string) bool
+	HashPassword(password string) (string, error)
+}
+
+type store struct {
+	lg          *zap.Logger
+	server      doer
+	timeout     time.Duration
+	ensuredOnce bool
+
+	PasswordStore
+}
+
+type User struct {
+	User     string   `json:"user"`
+	Password string   `json:"password,omitempty"`
+	Roles    []string `json:"roles"`
+	Grant    []string `json:"grant,omitempty"`
+	Revoke   []string `json:"revoke,omitempty"`
+}
+
+type Role struct {
+	Role        string       `json:"role"`
+	Permissions Permissions  `json:"permissions"`
+	Grant       *Permissions `json:"grant,omitempty"`
+	Revoke      *Permissions `json:"revoke,omitempty"`
+}
+
+type Permissions struct {
+	KV RWPermission `json:"kv"`
+}
+
+func (p *Permissions) IsEmpty() bool {
+	return p == nil || (len(p.KV.Read) == 0 && len(p.KV.Write) == 0)
+}
+
+type RWPermission struct {
+	Read  []string `json:"read"`
+	Write []string `json:"write"`
+}
+
+type Error struct {
+	Status int
+	Errmsg string
+}
+
+func (ae Error) Error() string   { return ae.Errmsg }
+func (ae Error) HTTPStatus() int { return ae.Status }
+
+func authErr(hs int, s string, v ...interface{}) Error {
+	return Error{Status: hs, Errmsg: fmt.Sprintf("auth: "+s, v...)}
+}
+
+func NewStore(lg *zap.Logger, server doer, timeout time.Duration) Store {
+	s := &store{
+		lg:            lg,
+		server:        server,
+		timeout:       timeout,
+		PasswordStore: passwordStore{},
+	}
+	return s
+}
+
+// passwordStore implements PasswordStore using bcrypt to hash user passwords
+type passwordStore struct{}
+
+func (passwordStore) CheckPassword(user User, password string) bool {
+	err := bcrypt.CompareHashAndPassword([]byte(user.Password), []byte(password))
+	return err == nil
+}
+
+func (passwordStore) HashPassword(password string) (string, error) {
+	hash, err := bcrypt.GenerateFromPassword([]byte(password), bcrypt.DefaultCost)
+	return string(hash), err
+}
+
+func (s *store) AllUsers() ([]string, error) {
+	resp, err := s.requestResource("/users/", false)
+	if err != nil {
+		if e, ok := err.(*v2error.Error); ok {
+			if e.ErrorCode == v2error.EcodeKeyNotFound {
+				return []string{}, nil
+			}
+		}
+		return nil, err
+	}
+	var nodes []string
+	for _, n := range resp.Event.Node.Nodes {
+		_, user := path.Split(n.Key)
+		nodes = append(nodes, user)
+	}
+	sort.Strings(nodes)
+	return nodes, nil
+}
+
+func (s *store) GetUser(name string) (User, error) { return s.getUser(name, false) }
+
+// CreateOrUpdateUser should be only used for creating the new user or when you are not
+// sure if it is a create or update. (When only password is passed in, we are not sure
+// if it is a update or create)
+func (s *store) CreateOrUpdateUser(user User) (out User, created bool, err error) {
+	_, err = s.getUser(user.User, true)
+	if err == nil {
+		out, err = s.UpdateUser(user)
+		return out, false, err
+	}
+	u, err := s.CreateUser(user)
+	return u, true, err
+}
+
+func (s *store) CreateUser(user User) (User, error) {
+	// Attach root role to root user.
+	if user.User == "root" {
+		user = attachRootRole(user)
+	}
+	u, err := s.createUserInternal(user)
+	if err == nil {
+		if s.lg != nil {
+			s.lg.Info("created a user", zap.String("user-name", user.User))
+		} else {
+			plog.Noticef("created user %s", user.User)
+		}
+	}
+	return u, err
+}
+
+func (s *store) createUserInternal(user User) (User, error) {
+	if user.Password == "" {
+		return user, authErr(http.StatusBadRequest, "Cannot create user %s with an empty password", user.User)
+	}
+	hash, err := s.HashPassword(user.Password)
+	if err != nil {
+		return user, err
+	}
+	user.Password = hash
+
+	_, err = s.createResource("/users/"+user.User, user)
+	if err != nil {
+		if e, ok := err.(*v2error.Error); ok {
+			if e.ErrorCode == v2error.EcodeNodeExist {
+				return user, authErr(http.StatusConflict, "User %s already exists.", user.User)
+			}
+		}
+	}
+	return user, err
+}
+
+func (s *store) DeleteUser(name string) error {
+	if s.AuthEnabled() && name == "root" {
+		return authErr(http.StatusForbidden, "Cannot delete root user while auth is enabled.")
+	}
+	err := s.deleteResource("/users/" + name)
+	if err != nil {
+		if e, ok := err.(*v2error.Error); ok {
+			if e.ErrorCode == v2error.EcodeKeyNotFound {
+				return authErr(http.StatusNotFound, "User %s does not exist", name)
+			}
+		}
+		return err
+	}
+	if s.lg != nil {
+		s.lg.Info("deleted a user", zap.String("user-name", name))
+	} else {
+		plog.Noticef("deleted user %s", name)
+	}
+	return nil
+}
+
+func (s *store) UpdateUser(user User) (User, error) {
+	old, err := s.getUser(user.User, true)
+	if err != nil {
+		if e, ok := err.(*v2error.Error); ok {
+			if e.ErrorCode == v2error.EcodeKeyNotFound {
+				return user, authErr(http.StatusNotFound, "User %s doesn't exist.", user.User)
+			}
+		}
+		return old, err
+	}
+
+	newUser, err := old.merge(s.lg, user, s.PasswordStore)
+	if err != nil {
+		return old, err
+	}
+	if reflect.DeepEqual(old, newUser) {
+		return old, authErr(http.StatusBadRequest, "User not updated. Use grant/revoke/password to update the user.")
+	}
+	_, err = s.updateResource("/users/"+user.User, newUser)
+	if err == nil {
+		if s.lg != nil {
+			s.lg.Info("updated a user", zap.String("user-name", user.User))
+		} else {
+			plog.Noticef("updated user %s", user.User)
+		}
+	}
+	return newUser, err
+}
+
+func (s *store) AllRoles() ([]string, error) {
+	nodes := []string{RootRoleName}
+	resp, err := s.requestResource("/roles/", false)
+	if err != nil {
+		if e, ok := err.(*v2error.Error); ok {
+			if e.ErrorCode == v2error.EcodeKeyNotFound {
+				return nodes, nil
+			}
+		}
+		return nil, err
+	}
+	for _, n := range resp.Event.Node.Nodes {
+		_, role := path.Split(n.Key)
+		nodes = append(nodes, role)
+	}
+	sort.Strings(nodes)
+	return nodes, nil
+}
+
+func (s *store) GetRole(name string) (Role, error) { return s.getRole(name, false) }
+
+func (s *store) CreateRole(role Role) error {
+	if role.Role == RootRoleName {
+		return authErr(http.StatusForbidden, "Cannot modify role %s: is root role.", role.Role)
+	}
+	_, err := s.createResource("/roles/"+role.Role, role)
+	if err != nil {
+		if e, ok := err.(*v2error.Error); ok {
+			if e.ErrorCode == v2error.EcodeNodeExist {
+				return authErr(http.StatusConflict, "Role %s already exists.", role.Role)
+			}
+		}
+	}
+	if err == nil {
+		if s.lg != nil {
+			s.lg.Info("created a new role", zap.String("role-name", role.Role))
+		} else {
+			plog.Noticef("created new role %s", role.Role)
+		}
+	}
+	return err
+}
+
+func (s *store) DeleteRole(name string) error {
+	if name == RootRoleName {
+		return authErr(http.StatusForbidden, "Cannot modify role %s: is root role.", name)
+	}
+	err := s.deleteResource("/roles/" + name)
+	if err != nil {
+		if e, ok := err.(*v2error.Error); ok {
+			if e.ErrorCode == v2error.EcodeKeyNotFound {
+				return authErr(http.StatusNotFound, "Role %s doesn't exist.", name)
+			}
+		}
+	}
+	if err == nil {
+		if s.lg != nil {
+			s.lg.Info("delete a new role", zap.String("role-name", name))
+		} else {
+			plog.Noticef("deleted role %s", name)
+		}
+	}
+	return err
+}
+
+func (s *store) UpdateRole(role Role) (Role, error) {
+	if role.Role == RootRoleName {
+		return Role{}, authErr(http.StatusForbidden, "Cannot modify role %s: is root role.", role.Role)
+	}
+	old, err := s.getRole(role.Role, true)
+	if err != nil {
+		if e, ok := err.(*v2error.Error); ok {
+			if e.ErrorCode == v2error.EcodeKeyNotFound {
+				return role, authErr(http.StatusNotFound, "Role %s doesn't exist.", role.Role)
+			}
+		}
+		return old, err
+	}
+	newRole, err := old.merge(s.lg, role)
+	if err != nil {
+		return old, err
+	}
+	if reflect.DeepEqual(old, newRole) {
+		return old, authErr(http.StatusBadRequest, "Role not updated. Use grant/revoke to update the role.")
+	}
+	_, err = s.updateResource("/roles/"+role.Role, newRole)
+	if err == nil {
+		if s.lg != nil {
+			s.lg.Info("updated a new role", zap.String("role-name", role.Role))
+		} else {
+			plog.Noticef("updated role %s", role.Role)
+		}
+	}
+	return newRole, err
+}
+
+func (s *store) AuthEnabled() bool {
+	return s.detectAuth()
+}
+
+func (s *store) EnableAuth() error {
+	if s.AuthEnabled() {
+		return authErr(http.StatusConflict, "already enabled")
+	}
+
+	if _, err := s.getUser("root", true); err != nil {
+		return authErr(http.StatusConflict, "No root user available, please create one")
+	}
+	if _, err := s.getRole(GuestRoleName, true); err != nil {
+		if s.lg != nil {
+			s.lg.Info(
+				"no guest role access found; creating default",
+				zap.String("role-name", GuestRoleName),
+			)
+		} else {
+			plog.Printf("no guest role access found, creating default")
+		}
+		if err := s.CreateRole(guestRole); err != nil {
+			if s.lg != nil {
+				s.lg.Warn(
+					"failed to create a guest role; aborting auth enable",
+					zap.String("role-name", GuestRoleName),
+					zap.Error(err),
+				)
+			} else {
+				plog.Errorf("error creating guest role. aborting auth enable.")
+			}
+			return err
+		}
+	}
+
+	if err := s.enableAuth(); err != nil {
+		if s.lg != nil {
+			s.lg.Warn("failed to enable auth", zap.Error(err))
+		} else {
+			plog.Errorf("error enabling auth (%v)", err)
+		}
+		return err
+	}
+
+	if s.lg != nil {
+		s.lg.Info("enabled auth")
+	} else {
+		plog.Noticef("auth: enabled auth")
+	}
+	return nil
+}
+
+func (s *store) DisableAuth() error {
+	if !s.AuthEnabled() {
+		return authErr(http.StatusConflict, "already disabled")
+	}
+
+	err := s.disableAuth()
+	if err == nil {
+		if s.lg != nil {
+			s.lg.Info("disabled auth")
+		} else {
+			plog.Noticef("auth: disabled auth")
+		}
+	} else {
+		if s.lg != nil {
+			s.lg.Warn("failed to disable auth", zap.Error(err))
+		} else {
+			plog.Errorf("error disabling auth (%v)", err)
+		}
+	}
+	return err
+}
+
+// merge applies the properties of the passed-in User to the User on which it
+// is called and returns a new User with these modifications applied. Think of
+// all Users as immutable sets of data. Merge allows you to perform the set
+// operations (desired grants and revokes) atomically
+func (ou User) merge(lg *zap.Logger, nu User, s PasswordStore) (User, error) {
+	var out User
+	if ou.User != nu.User {
+		return out, authErr(http.StatusConflict, "Merging user data with conflicting usernames: %s %s", ou.User, nu.User)
+	}
+	out.User = ou.User
+	if nu.Password != "" {
+		hash, err := s.HashPassword(nu.Password)
+		if err != nil {
+			return ou, err
+		}
+		out.Password = hash
+	} else {
+		out.Password = ou.Password
+	}
+	currentRoles := types.NewUnsafeSet(ou.Roles...)
+	for _, g := range nu.Grant {
+		if currentRoles.Contains(g) {
+			if lg != nil {
+				lg.Warn(
+					"attempted to grant a duplicate role for a user",
+					zap.String("user-name", nu.User),
+					zap.String("role-name", g),
+				)
+			} else {
+				plog.Noticef("granting duplicate role %s for user %s", g, nu.User)
+			}
+			return User{}, authErr(http.StatusConflict, fmt.Sprintf("Granting duplicate role %s for user %s", g, nu.User))
+		}
+		currentRoles.Add(g)
+	}
+	for _, r := range nu.Revoke {
+		if !currentRoles.Contains(r) {
+			if lg != nil {
+				lg.Warn(
+					"attempted to revoke a ungranted role for a user",
+					zap.String("user-name", nu.User),
+					zap.String("role-name", r),
+				)
+			} else {
+				plog.Noticef("revoking ungranted role %s for user %s", r, nu.User)
+			}
+			return User{}, authErr(http.StatusConflict, fmt.Sprintf("Revoking ungranted role %s for user %s", r, nu.User))
+		}
+		currentRoles.Remove(r)
+	}
+	out.Roles = currentRoles.Values()
+	sort.Strings(out.Roles)
+	return out, nil
+}
+
+// merge for a role works the same as User above -- atomic Role application to
+// each of the substructures.
+func (r Role) merge(lg *zap.Logger, n Role) (Role, error) {
+	var out Role
+	var err error
+	if r.Role != n.Role {
+		return out, authErr(http.StatusConflict, "Merging role with conflicting names: %s %s", r.Role, n.Role)
+	}
+	out.Role = r.Role
+	out.Permissions, err = r.Permissions.Grant(n.Grant)
+	if err != nil {
+		return out, err
+	}
+	out.Permissions, err = out.Permissions.Revoke(lg, n.Revoke)
+	return out, err
+}
+
+func (r Role) HasKeyAccess(key string, write bool) bool {
+	if r.Role == RootRoleName {
+		return true
+	}
+	return r.Permissions.KV.HasAccess(key, write)
+}
+
+func (r Role) HasRecursiveAccess(key string, write bool) bool {
+	if r.Role == RootRoleName {
+		return true
+	}
+	return r.Permissions.KV.HasRecursiveAccess(key, write)
+}
+
+// Grant adds a set of permissions to the permission object on which it is called,
+// returning a new permission object.
+func (p Permissions) Grant(n *Permissions) (Permissions, error) {
+	var out Permissions
+	var err error
+	if n == nil {
+		return p, nil
+	}
+	out.KV, err = p.KV.Grant(n.KV)
+	return out, err
+}
+
+// Revoke removes a set of permissions to the permission object on which it is called,
+// returning a new permission object.
+func (p Permissions) Revoke(lg *zap.Logger, n *Permissions) (Permissions, error) {
+	var out Permissions
+	var err error
+	if n == nil {
+		return p, nil
+	}
+	out.KV, err = p.KV.Revoke(lg, n.KV)
+	return out, err
+}
+
+// Grant adds a set of permissions to the permission object on which it is called,
+// returning a new permission object.
+func (rw RWPermission) Grant(n RWPermission) (RWPermission, error) {
+	var out RWPermission
+	currentRead := types.NewUnsafeSet(rw.Read...)
+	for _, r := range n.Read {
+		if currentRead.Contains(r) {
+			return out, authErr(http.StatusConflict, "Granting duplicate read permission %s", r)
+		}
+		currentRead.Add(r)
+	}
+	currentWrite := types.NewUnsafeSet(rw.Write...)
+	for _, w := range n.Write {
+		if currentWrite.Contains(w) {
+			return out, authErr(http.StatusConflict, "Granting duplicate write permission %s", w)
+		}
+		currentWrite.Add(w)
+	}
+	out.Read = currentRead.Values()
+	out.Write = currentWrite.Values()
+	sort.Strings(out.Read)
+	sort.Strings(out.Write)
+	return out, nil
+}
+
+// Revoke removes a set of permissions to the permission object on which it is called,
+// returning a new permission object.
+func (rw RWPermission) Revoke(lg *zap.Logger, n RWPermission) (RWPermission, error) {
+	var out RWPermission
+	currentRead := types.NewUnsafeSet(rw.Read...)
+	for _, r := range n.Read {
+		if !currentRead.Contains(r) {
+			if lg != nil {
+				lg.Info(
+					"revoking ungranted read permission",
+					zap.String("read-permission", r),
+				)
+			} else {
+				plog.Noticef("revoking ungranted read permission %s", r)
+			}
+			continue
+		}
+		currentRead.Remove(r)
+	}
+	currentWrite := types.NewUnsafeSet(rw.Write...)
+	for _, w := range n.Write {
+		if !currentWrite.Contains(w) {
+			if lg != nil {
+				lg.Info(
+					"revoking ungranted write permission",
+					zap.String("write-permission", w),
+				)
+			} else {
+				plog.Noticef("revoking ungranted write permission %s", w)
+			}
+			continue
+		}
+		currentWrite.Remove(w)
+	}
+	out.Read = currentRead.Values()
+	out.Write = currentWrite.Values()
+	sort.Strings(out.Read)
+	sort.Strings(out.Write)
+	return out, nil
+}
+
+func (rw RWPermission) HasAccess(key string, write bool) bool {
+	var list []string
+	if write {
+		list = rw.Write
+	} else {
+		list = rw.Read
+	}
+	for _, pat := range list {
+		match, err := simpleMatch(pat, key)
+		if err == nil && match {
+			return true
+		}
+	}
+	return false
+}
+
+func (rw RWPermission) HasRecursiveAccess(key string, write bool) bool {
+	list := rw.Read
+	if write {
+		list = rw.Write
+	}
+	for _, pat := range list {
+		match, err := prefixMatch(pat, key)
+		if err == nil && match {
+			return true
+		}
+	}
+	return false
+}
+
+func simpleMatch(pattern string, key string) (match bool, err error) {
+	if pattern[len(pattern)-1] == '*' {
+		return strings.HasPrefix(key, pattern[:len(pattern)-1]), nil
+	}
+	return key == pattern, nil
+}
+
+func prefixMatch(pattern string, key string) (match bool, err error) {
+	if pattern[len(pattern)-1] != '*' {
+		return false, nil
+	}
+	return strings.HasPrefix(key, pattern[:len(pattern)-1]), nil
+}
+
+func attachRootRole(u User) User {
+	inRoles := false
+	for _, r := range u.Roles {
+		if r == RootRoleName {
+			inRoles = true
+			break
+		}
+	}
+	if !inRoles {
+		u.Roles = append(u.Roles, RootRoleName)
+	}
+	return u
+}
+
+func (s *store) getUser(name string, quorum bool) (User, error) {
+	resp, err := s.requestResource("/users/"+name, quorum)
+	if err != nil {
+		if e, ok := err.(*v2error.Error); ok {
+			if e.ErrorCode == v2error.EcodeKeyNotFound {
+				return User{}, authErr(http.StatusNotFound, "User %s does not exist.", name)
+			}
+		}
+		return User{}, err
+	}
+	var u User
+	err = json.Unmarshal([]byte(*resp.Event.Node.Value), &u)
+	if err != nil {
+		return u, err
+	}
+	// Attach root role to root user.
+	if u.User == "root" {
+		u = attachRootRole(u)
+	}
+	return u, nil
+}
+
+func (s *store) getRole(name string, quorum bool) (Role, error) {
+	if name == RootRoleName {
+		return rootRole, nil
+	}
+	resp, err := s.requestResource("/roles/"+name, quorum)
+	if err != nil {
+		if e, ok := err.(*v2error.Error); ok {
+			if e.ErrorCode == v2error.EcodeKeyNotFound {
+				return Role{}, authErr(http.StatusNotFound, "Role %s does not exist.", name)
+			}
+		}
+		return Role{}, err
+	}
+	var r Role
+	err = json.Unmarshal([]byte(*resp.Event.Node.Value), &r)
+	return r, err
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2auth/auth_requests.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2auth/auth_requests.go
new file mode 100644
index 0000000..d6574ec
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2auth/auth_requests.go
@@ -0,0 +1,189 @@
+// Copyright 2015 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 v2auth
+
+import (
+	"context"
+	"encoding/json"
+	"path"
+
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/api/v2error"
+	"go.etcd.io/etcd/etcdserver/etcdserverpb"
+
+	"go.uber.org/zap"
+)
+
+func (s *store) ensureAuthDirectories() error {
+	if s.ensuredOnce {
+		return nil
+	}
+	for _, res := range []string{StorePermsPrefix, StorePermsPrefix + "/users/", StorePermsPrefix + "/roles/"} {
+		ctx, cancel := context.WithTimeout(context.Background(), s.timeout)
+		pe := false
+		rr := etcdserverpb.Request{
+			Method:    "PUT",
+			Path:      res,
+			Dir:       true,
+			PrevExist: &pe,
+		}
+		_, err := s.server.Do(ctx, rr)
+		cancel()
+		if err != nil {
+			if e, ok := err.(*v2error.Error); ok {
+				if e.ErrorCode == v2error.EcodeNodeExist {
+					continue
+				}
+			}
+			if s.lg != nil {
+				s.lg.Warn(
+					"failed to create auth directories",
+					zap.Error(err),
+				)
+			} else {
+				plog.Errorf("failed to create auth directories in the store (%v)", err)
+			}
+			return err
+		}
+	}
+	ctx, cancel := context.WithTimeout(context.Background(), s.timeout)
+	defer cancel()
+	pe := false
+	rr := etcdserverpb.Request{
+		Method:    "PUT",
+		Path:      StorePermsPrefix + "/enabled",
+		Val:       "false",
+		PrevExist: &pe,
+	}
+	_, err := s.server.Do(ctx, rr)
+	if err != nil {
+		if e, ok := err.(*v2error.Error); ok {
+			if e.ErrorCode == v2error.EcodeNodeExist {
+				s.ensuredOnce = true
+				return nil
+			}
+		}
+		return err
+	}
+	s.ensuredOnce = true
+	return nil
+}
+
+func (s *store) enableAuth() error {
+	_, err := s.updateResource("/enabled", true)
+	return err
+}
+func (s *store) disableAuth() error {
+	_, err := s.updateResource("/enabled", false)
+	return err
+}
+
+func (s *store) detectAuth() bool {
+	if s.server == nil {
+		return false
+	}
+	value, err := s.requestResource("/enabled", false)
+	if err != nil {
+		if e, ok := err.(*v2error.Error); ok {
+			if e.ErrorCode == v2error.EcodeKeyNotFound {
+				return false
+			}
+		}
+		if s.lg != nil {
+			s.lg.Warn(
+				"failed to detect auth settings",
+				zap.Error(err),
+			)
+		} else {
+			plog.Errorf("failed to detect auth settings (%s)", err)
+		}
+		return false
+	}
+
+	var u bool
+	err = json.Unmarshal([]byte(*value.Event.Node.Value), &u)
+	if err != nil {
+		if s.lg != nil {
+			s.lg.Warn(
+				"internal bookkeeping value for enabled isn't valid JSON",
+				zap.Error(err),
+			)
+		} else {
+			plog.Errorf("internal bookkeeping value for enabled isn't valid JSON (%v)", err)
+		}
+		return false
+	}
+	return u
+}
+
+func (s *store) requestResource(res string, quorum bool) (etcdserver.Response, error) {
+	ctx, cancel := context.WithTimeout(context.Background(), s.timeout)
+	defer cancel()
+	p := path.Join(StorePermsPrefix, res)
+	method := "GET"
+	if quorum {
+		method = "QGET"
+	}
+	rr := etcdserverpb.Request{
+		Method: method,
+		Path:   p,
+		Dir:    false, // TODO: always false?
+	}
+	return s.server.Do(ctx, rr)
+}
+
+func (s *store) updateResource(res string, value interface{}) (etcdserver.Response, error) {
+	return s.setResource(res, value, true)
+}
+func (s *store) createResource(res string, value interface{}) (etcdserver.Response, error) {
+	return s.setResource(res, value, false)
+}
+func (s *store) setResource(res string, value interface{}, prevexist bool) (etcdserver.Response, error) {
+	err := s.ensureAuthDirectories()
+	if err != nil {
+		return etcdserver.Response{}, err
+	}
+	ctx, cancel := context.WithTimeout(context.Background(), s.timeout)
+	defer cancel()
+	data, err := json.Marshal(value)
+	if err != nil {
+		return etcdserver.Response{}, err
+	}
+	p := path.Join(StorePermsPrefix, res)
+	rr := etcdserverpb.Request{
+		Method:    "PUT",
+		Path:      p,
+		Val:       string(data),
+		PrevExist: &prevexist,
+	}
+	return s.server.Do(ctx, rr)
+}
+
+func (s *store) deleteResource(res string) error {
+	err := s.ensureAuthDirectories()
+	if err != nil {
+		return err
+	}
+	ctx, cancel := context.WithTimeout(context.Background(), s.timeout)
+	defer cancel()
+	pex := true
+	p := path.Join(StorePermsPrefix, res)
+	_, err = s.server.Do(ctx, etcdserverpb.Request{
+		Method:    "DELETE",
+		Path:      p,
+		PrevExist: &pex,
+	})
+	return err
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2discovery/discovery.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2discovery/discovery.go
new file mode 100644
index 0000000..cf770b3
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2discovery/discovery.go
@@ -0,0 +1,440 @@
+// Copyright 2015 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 v2discovery provides an implementation of the cluster discovery that
+// is used by etcd with v2 client.
+package v2discovery
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"math"
+	"net/http"
+	"net/url"
+	"path"
+	"sort"
+	"strconv"
+	"strings"
+	"time"
+
+	"go.etcd.io/etcd/client"
+	"go.etcd.io/etcd/pkg/transport"
+	"go.etcd.io/etcd/pkg/types"
+
+	"github.com/coreos/pkg/capnslog"
+	"github.com/jonboulle/clockwork"
+	"go.uber.org/zap"
+)
+
+var (
+	plog = capnslog.NewPackageLogger("go.etcd.io/etcd", "discovery")
+
+	ErrInvalidURL           = errors.New("discovery: invalid URL")
+	ErrBadSizeKey           = errors.New("discovery: size key is bad")
+	ErrSizeNotFound         = errors.New("discovery: size key not found")
+	ErrTokenNotFound        = errors.New("discovery: token not found")
+	ErrDuplicateID          = errors.New("discovery: found duplicate id")
+	ErrDuplicateName        = errors.New("discovery: found duplicate name")
+	ErrFullCluster          = errors.New("discovery: cluster is full")
+	ErrTooManyRetries       = errors.New("discovery: too many retries")
+	ErrBadDiscoveryEndpoint = errors.New("discovery: bad discovery endpoint")
+)
+
+var (
+	// Number of retries discovery will attempt before giving up and erroring out.
+	nRetries             = uint(math.MaxUint32)
+	maxExpoentialRetries = uint(8)
+)
+
+// JoinCluster will connect to the discovery service at the given url, and
+// register the server represented by the given id and config to the cluster
+func JoinCluster(lg *zap.Logger, durl, dproxyurl string, id types.ID, config string) (string, error) {
+	d, err := newDiscovery(lg, durl, dproxyurl, id)
+	if err != nil {
+		return "", err
+	}
+	return d.joinCluster(config)
+}
+
+// GetCluster will connect to the discovery service at the given url and
+// retrieve a string describing the cluster
+func GetCluster(lg *zap.Logger, durl, dproxyurl string) (string, error) {
+	d, err := newDiscovery(lg, durl, dproxyurl, 0)
+	if err != nil {
+		return "", err
+	}
+	return d.getCluster()
+}
+
+type discovery struct {
+	lg      *zap.Logger
+	cluster string
+	id      types.ID
+	c       client.KeysAPI
+	retries uint
+	url     *url.URL
+
+	clock clockwork.Clock
+}
+
+// newProxyFunc builds a proxy function from the given string, which should
+// represent a URL that can be used as a proxy. It performs basic
+// sanitization of the URL and returns any error encountered.
+func newProxyFunc(lg *zap.Logger, proxy string) (func(*http.Request) (*url.URL, error), error) {
+	if proxy == "" {
+		return nil, nil
+	}
+	// Do a small amount of URL sanitization to help the user
+	// Derived from net/http.ProxyFromEnvironment
+	proxyURL, err := url.Parse(proxy)
+	if err != nil || !strings.HasPrefix(proxyURL.Scheme, "http") {
+		// proxy was bogus. Try prepending "http://" to it and
+		// see if that parses correctly. If not, we ignore the
+		// error and complain about the original one
+		var err2 error
+		proxyURL, err2 = url.Parse("http://" + proxy)
+		if err2 == nil {
+			err = nil
+		}
+	}
+	if err != nil {
+		return nil, fmt.Errorf("invalid proxy address %q: %v", proxy, err)
+	}
+
+	if lg != nil {
+		lg.Info("running proxy with discovery", zap.String("proxy-url", proxyURL.String()))
+	} else {
+		plog.Infof("using proxy %q", proxyURL.String())
+	}
+	return http.ProxyURL(proxyURL), nil
+}
+
+func newDiscovery(lg *zap.Logger, durl, dproxyurl string, id types.ID) (*discovery, error) {
+	u, err := url.Parse(durl)
+	if err != nil {
+		return nil, err
+	}
+	token := u.Path
+	u.Path = ""
+	pf, err := newProxyFunc(lg, dproxyurl)
+	if err != nil {
+		return nil, err
+	}
+
+	// TODO: add ResponseHeaderTimeout back when watch on discovery service writes header early
+	tr, err := transport.NewTransport(transport.TLSInfo{}, 30*time.Second)
+	if err != nil {
+		return nil, err
+	}
+	tr.Proxy = pf
+	cfg := client.Config{
+		Transport: tr,
+		Endpoints: []string{u.String()},
+	}
+	c, err := client.New(cfg)
+	if err != nil {
+		return nil, err
+	}
+	dc := client.NewKeysAPIWithPrefix(c, "")
+	return &discovery{
+		lg:      lg,
+		cluster: token,
+		c:       dc,
+		id:      id,
+		url:     u,
+		clock:   clockwork.NewRealClock(),
+	}, nil
+}
+
+func (d *discovery) joinCluster(config string) (string, error) {
+	// fast path: if the cluster is full, return the error
+	// do not need to register to the cluster in this case.
+	if _, _, _, err := d.checkCluster(); err != nil {
+		return "", err
+	}
+
+	if err := d.createSelf(config); err != nil {
+		// Fails, even on a timeout, if createSelf times out.
+		// TODO(barakmich): Retrying the same node might want to succeed here
+		// (ie, createSelf should be idempotent for discovery).
+		return "", err
+	}
+
+	nodes, size, index, err := d.checkCluster()
+	if err != nil {
+		return "", err
+	}
+
+	all, err := d.waitNodes(nodes, size, index)
+	if err != nil {
+		return "", err
+	}
+
+	return nodesToCluster(all, size)
+}
+
+func (d *discovery) getCluster() (string, error) {
+	nodes, size, index, err := d.checkCluster()
+	if err != nil {
+		if err == ErrFullCluster {
+			return nodesToCluster(nodes, size)
+		}
+		return "", err
+	}
+
+	all, err := d.waitNodes(nodes, size, index)
+	if err != nil {
+		return "", err
+	}
+	return nodesToCluster(all, size)
+}
+
+func (d *discovery) createSelf(contents string) error {
+	ctx, cancel := context.WithTimeout(context.Background(), client.DefaultRequestTimeout)
+	resp, err := d.c.Create(ctx, d.selfKey(), contents)
+	cancel()
+	if err != nil {
+		if eerr, ok := err.(client.Error); ok && eerr.Code == client.ErrorCodeNodeExist {
+			return ErrDuplicateID
+		}
+		return err
+	}
+
+	// ensure self appears on the server we connected to
+	w := d.c.Watcher(d.selfKey(), &client.WatcherOptions{AfterIndex: resp.Node.CreatedIndex - 1})
+	_, err = w.Next(context.Background())
+	return err
+}
+
+func (d *discovery) checkCluster() ([]*client.Node, int, uint64, error) {
+	configKey := path.Join("/", d.cluster, "_config")
+	ctx, cancel := context.WithTimeout(context.Background(), client.DefaultRequestTimeout)
+	// find cluster size
+	resp, err := d.c.Get(ctx, path.Join(configKey, "size"), nil)
+	cancel()
+	if err != nil {
+		if eerr, ok := err.(*client.Error); ok && eerr.Code == client.ErrorCodeKeyNotFound {
+			return nil, 0, 0, ErrSizeNotFound
+		}
+		if err == client.ErrInvalidJSON {
+			return nil, 0, 0, ErrBadDiscoveryEndpoint
+		}
+		if ce, ok := err.(*client.ClusterError); ok {
+			if d.lg != nil {
+				d.lg.Warn(
+					"failed to get from discovery server",
+					zap.String("discovery-url", d.url.String()),
+					zap.String("path", path.Join(configKey, "size")),
+					zap.Error(err),
+					zap.String("err-detail", ce.Detail()),
+				)
+			} else {
+				plog.Error(ce.Detail())
+			}
+			return d.checkClusterRetry()
+		}
+		return nil, 0, 0, err
+	}
+	size, err := strconv.Atoi(resp.Node.Value)
+	if err != nil {
+		return nil, 0, 0, ErrBadSizeKey
+	}
+
+	ctx, cancel = context.WithTimeout(context.Background(), client.DefaultRequestTimeout)
+	resp, err = d.c.Get(ctx, d.cluster, nil)
+	cancel()
+	if err != nil {
+		if ce, ok := err.(*client.ClusterError); ok {
+			if d.lg != nil {
+				d.lg.Warn(
+					"failed to get from discovery server",
+					zap.String("discovery-url", d.url.String()),
+					zap.String("path", d.cluster),
+					zap.Error(err),
+					zap.String("err-detail", ce.Detail()),
+				)
+			} else {
+				plog.Error(ce.Detail())
+			}
+			return d.checkClusterRetry()
+		}
+		return nil, 0, 0, err
+	}
+	var nodes []*client.Node
+	// append non-config keys to nodes
+	for _, n := range resp.Node.Nodes {
+		if path.Base(n.Key) != path.Base(configKey) {
+			nodes = append(nodes, n)
+		}
+	}
+
+	snodes := sortableNodes{nodes}
+	sort.Sort(snodes)
+
+	// find self position
+	for i := range nodes {
+		if path.Base(nodes[i].Key) == path.Base(d.selfKey()) {
+			break
+		}
+		if i >= size-1 {
+			return nodes[:size], size, resp.Index, ErrFullCluster
+		}
+	}
+	return nodes, size, resp.Index, nil
+}
+
+func (d *discovery) logAndBackoffForRetry(step string) {
+	d.retries++
+	// logAndBackoffForRetry stops exponential backoff when the retries are more than maxExpoentialRetries and is set to a constant backoff afterward.
+	retries := d.retries
+	if retries > maxExpoentialRetries {
+		retries = maxExpoentialRetries
+	}
+	retryTimeInSecond := time.Duration(0x1<<retries) * time.Second
+	if d.lg != nil {
+		d.lg.Info(
+			"retry connecting to discovery service",
+			zap.String("url", d.url.String()),
+			zap.String("reason", step),
+			zap.Duration("backoff", retryTimeInSecond),
+		)
+	} else {
+		plog.Infof("%s: error connecting to %s, retrying in %s", step, d.url, retryTimeInSecond)
+	}
+	d.clock.Sleep(retryTimeInSecond)
+}
+
+func (d *discovery) checkClusterRetry() ([]*client.Node, int, uint64, error) {
+	if d.retries < nRetries {
+		d.logAndBackoffForRetry("cluster status check")
+		return d.checkCluster()
+	}
+	return nil, 0, 0, ErrTooManyRetries
+}
+
+func (d *discovery) waitNodesRetry() ([]*client.Node, error) {
+	if d.retries < nRetries {
+		d.logAndBackoffForRetry("waiting for other nodes")
+		nodes, n, index, err := d.checkCluster()
+		if err != nil {
+			return nil, err
+		}
+		return d.waitNodes(nodes, n, index)
+	}
+	return nil, ErrTooManyRetries
+}
+
+func (d *discovery) waitNodes(nodes []*client.Node, size int, index uint64) ([]*client.Node, error) {
+	if len(nodes) > size {
+		nodes = nodes[:size]
+	}
+	// watch from the next index
+	w := d.c.Watcher(d.cluster, &client.WatcherOptions{AfterIndex: index, Recursive: true})
+	all := make([]*client.Node, len(nodes))
+	copy(all, nodes)
+	for _, n := range all {
+		if path.Base(n.Key) == path.Base(d.selfKey()) {
+			if d.lg != nil {
+				d.lg.Info(
+					"found self from discovery server",
+					zap.String("discovery-url", d.url.String()),
+					zap.String("self", path.Base(d.selfKey())),
+				)
+			} else {
+				plog.Noticef("found self %s in the cluster", path.Base(d.selfKey()))
+			}
+		} else {
+			if d.lg != nil {
+				d.lg.Info(
+					"found peer from discovery server",
+					zap.String("discovery-url", d.url.String()),
+					zap.String("peer", path.Base(n.Key)),
+				)
+			} else {
+				plog.Noticef("found peer %s in the cluster", path.Base(n.Key))
+			}
+		}
+	}
+
+	// wait for others
+	for len(all) < size {
+		if d.lg != nil {
+			d.lg.Info(
+				"found peers from discovery server; waiting for more",
+				zap.String("discovery-url", d.url.String()),
+				zap.Int("found-peers", len(all)),
+				zap.Int("needed-peers", size-len(all)),
+			)
+		} else {
+			plog.Noticef("found %d peer(s), waiting for %d more", len(all), size-len(all))
+		}
+		resp, err := w.Next(context.Background())
+		if err != nil {
+			if ce, ok := err.(*client.ClusterError); ok {
+				plog.Error(ce.Detail())
+				return d.waitNodesRetry()
+			}
+			return nil, err
+		}
+		if d.lg != nil {
+			d.lg.Info(
+				"found peer from discovery server",
+				zap.String("discovery-url", d.url.String()),
+				zap.String("peer", path.Base(resp.Node.Key)),
+			)
+		} else {
+			plog.Noticef("found peer %s in the cluster", path.Base(resp.Node.Key))
+		}
+		all = append(all, resp.Node)
+	}
+	if d.lg != nil {
+		d.lg.Info(
+			"found all needed peers from discovery server",
+			zap.String("discovery-url", d.url.String()),
+			zap.Int("found-peers", len(all)),
+		)
+	} else {
+		plog.Noticef("found %d needed peer(s)", len(all))
+	}
+	return all, nil
+}
+
+func (d *discovery) selfKey() string {
+	return path.Join("/", d.cluster, d.id.String())
+}
+
+func nodesToCluster(ns []*client.Node, size int) (string, error) {
+	s := make([]string, len(ns))
+	for i, n := range ns {
+		s[i] = n.Value
+	}
+	us := strings.Join(s, ",")
+	m, err := types.NewURLsMap(us)
+	if err != nil {
+		return us, ErrInvalidURL
+	}
+	if m.Len() != size {
+		return us, ErrDuplicateName
+	}
+	return us, nil
+}
+
+type sortableNodes struct{ Nodes []*client.Node }
+
+func (ns sortableNodes) Len() int { return len(ns.Nodes) }
+func (ns sortableNodes) Less(i, j int) bool {
+	return ns.Nodes[i].CreatedIndex < ns.Nodes[j].CreatedIndex
+}
+func (ns sortableNodes) Swap(i, j int) { ns.Nodes[i], ns.Nodes[j] = ns.Nodes[j], ns.Nodes[i] }
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2error/error.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2error/error.go
new file mode 100644
index 0000000..1244290
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2error/error.go
@@ -0,0 +1,164 @@
+// Copyright 2015 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 v2error describes errors in etcd project. When any change happens,
+// Documentation/v2/errorcode.md needs to be updated correspondingly.
+// To be deprecated in favor of v3 APIs.
+package v2error
+
+import (
+	"encoding/json"
+	"fmt"
+	"net/http"
+)
+
+var errors = map[int]string{
+	// command related errors
+	EcodeKeyNotFound:      "Key not found",
+	EcodeTestFailed:       "Compare failed", //test and set
+	EcodeNotFile:          "Not a file",
+	ecodeNoMorePeer:       "Reached the max number of peers in the cluster",
+	EcodeNotDir:           "Not a directory",
+	EcodeNodeExist:        "Key already exists", // create
+	ecodeKeyIsPreserved:   "The prefix of given key is a keyword in etcd",
+	EcodeRootROnly:        "Root is read only",
+	EcodeDirNotEmpty:      "Directory not empty",
+	ecodeExistingPeerAddr: "Peer address has existed",
+	EcodeUnauthorized:     "The request requires user authentication",
+
+	// Post form related errors
+	ecodeValueRequired:        "Value is Required in POST form",
+	EcodePrevValueRequired:    "PrevValue is Required in POST form",
+	EcodeTTLNaN:               "The given TTL in POST form is not a number",
+	EcodeIndexNaN:             "The given index in POST form is not a number",
+	ecodeValueOrTTLRequired:   "Value or TTL is required in POST form",
+	ecodeTimeoutNaN:           "The given timeout in POST form is not a number",
+	ecodeNameRequired:         "Name is required in POST form",
+	ecodeIndexOrValueRequired: "Index or value is required",
+	ecodeIndexValueMutex:      "Index and value cannot both be specified",
+	EcodeInvalidField:         "Invalid field",
+	EcodeInvalidForm:          "Invalid POST form",
+	EcodeRefreshValue:         "Value provided on refresh",
+	EcodeRefreshTTLRequired:   "A TTL must be provided on refresh",
+
+	// raft related errors
+	EcodeRaftInternal: "Raft Internal Error",
+	EcodeLeaderElect:  "During Leader Election",
+
+	// etcd related errors
+	EcodeWatcherCleared:     "watcher is cleared due to etcd recovery",
+	EcodeEventIndexCleared:  "The event in requested index is outdated and cleared",
+	ecodeStandbyInternal:    "Standby Internal Error",
+	ecodeInvalidActiveSize:  "Invalid active size",
+	ecodeInvalidRemoveDelay: "Standby remove delay",
+
+	// client related errors
+	ecodeClientInternal: "Client Internal Error",
+}
+
+var errorStatus = map[int]int{
+	EcodeKeyNotFound:  http.StatusNotFound,
+	EcodeNotFile:      http.StatusForbidden,
+	EcodeDirNotEmpty:  http.StatusForbidden,
+	EcodeUnauthorized: http.StatusUnauthorized,
+	EcodeTestFailed:   http.StatusPreconditionFailed,
+	EcodeNodeExist:    http.StatusPreconditionFailed,
+	EcodeRaftInternal: http.StatusInternalServerError,
+	EcodeLeaderElect:  http.StatusInternalServerError,
+}
+
+const (
+	EcodeKeyNotFound      = 100
+	EcodeTestFailed       = 101
+	EcodeNotFile          = 102
+	ecodeNoMorePeer       = 103
+	EcodeNotDir           = 104
+	EcodeNodeExist        = 105
+	ecodeKeyIsPreserved   = 106
+	EcodeRootROnly        = 107
+	EcodeDirNotEmpty      = 108
+	ecodeExistingPeerAddr = 109
+	EcodeUnauthorized     = 110
+
+	ecodeValueRequired        = 200
+	EcodePrevValueRequired    = 201
+	EcodeTTLNaN               = 202
+	EcodeIndexNaN             = 203
+	ecodeValueOrTTLRequired   = 204
+	ecodeTimeoutNaN           = 205
+	ecodeNameRequired         = 206
+	ecodeIndexOrValueRequired = 207
+	ecodeIndexValueMutex      = 208
+	EcodeInvalidField         = 209
+	EcodeInvalidForm          = 210
+	EcodeRefreshValue         = 211
+	EcodeRefreshTTLRequired   = 212
+
+	EcodeRaftInternal = 300
+	EcodeLeaderElect  = 301
+
+	EcodeWatcherCleared     = 400
+	EcodeEventIndexCleared  = 401
+	ecodeStandbyInternal    = 402
+	ecodeInvalidActiveSize  = 403
+	ecodeInvalidRemoveDelay = 404
+
+	ecodeClientInternal = 500
+)
+
+type Error struct {
+	ErrorCode int    `json:"errorCode"`
+	Message   string `json:"message"`
+	Cause     string `json:"cause,omitempty"`
+	Index     uint64 `json:"index"`
+}
+
+func NewRequestError(errorCode int, cause string) *Error {
+	return NewError(errorCode, cause, 0)
+}
+
+func NewError(errorCode int, cause string, index uint64) *Error {
+	return &Error{
+		ErrorCode: errorCode,
+		Message:   errors[errorCode],
+		Cause:     cause,
+		Index:     index,
+	}
+}
+
+// Error is for the error interface
+func (e Error) Error() string {
+	return e.Message + " (" + e.Cause + ")"
+}
+
+func (e Error) toJsonString() string {
+	b, _ := json.Marshal(e)
+	return string(b)
+}
+
+func (e Error) StatusCode() int {
+	status, ok := errorStatus[e.ErrorCode]
+	if !ok {
+		status = http.StatusBadRequest
+	}
+	return status
+}
+
+func (e Error) WriteTo(w http.ResponseWriter) error {
+	w.Header().Add("X-Etcd-Index", fmt.Sprint(e.Index))
+	w.Header().Set("Content-Type", "application/json")
+	w.WriteHeader(e.StatusCode())
+	_, err := w.Write([]byte(e.toJsonString() + "\n"))
+	return err
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2http/capability.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2http/capability.go
new file mode 100644
index 0000000..ed6c456
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2http/capability.go
@@ -0,0 +1,40 @@
+// Copyright 2015 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 v2http
+
+import (
+	"fmt"
+	"net/http"
+
+	"go.etcd.io/etcd/etcdserver/api"
+	"go.etcd.io/etcd/etcdserver/api/v2http/httptypes"
+)
+
+func authCapabilityHandler(fn func(http.ResponseWriter, *http.Request)) http.HandlerFunc {
+	return func(w http.ResponseWriter, r *http.Request) {
+		if !api.IsCapabilityEnabled(api.AuthCapability) {
+			notCapable(w, r, api.AuthCapability)
+			return
+		}
+		fn(w, r)
+	}
+}
+
+func notCapable(w http.ResponseWriter, r *http.Request, c api.Capability) {
+	herr := httptypes.NewHTTPError(http.StatusInternalServerError, fmt.Sprintf("Not capable of accessing %s feature during rolling upgrades.", c))
+	if err := herr.WriteTo(w); err != nil {
+		plog.Debugf("error writing HTTPError (%v) to %s", err, r.RemoteAddr)
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2http/client.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2http/client.go
new file mode 100644
index 0000000..1d1e592
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2http/client.go
@@ -0,0 +1,788 @@
+// Copyright 2015 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 v2http
+
+import (
+	"context"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"io/ioutil"
+	"net/http"
+	"net/url"
+	"path"
+	"strconv"
+	"strings"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/api"
+	"go.etcd.io/etcd/etcdserver/api/etcdhttp"
+	"go.etcd.io/etcd/etcdserver/api/membership"
+	"go.etcd.io/etcd/etcdserver/api/v2auth"
+	"go.etcd.io/etcd/etcdserver/api/v2error"
+	"go.etcd.io/etcd/etcdserver/api/v2http/httptypes"
+	stats "go.etcd.io/etcd/etcdserver/api/v2stats"
+	"go.etcd.io/etcd/etcdserver/api/v2store"
+	"go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/pkg/types"
+
+	"github.com/jonboulle/clockwork"
+	"go.uber.org/zap"
+)
+
+const (
+	authPrefix     = "/v2/auth"
+	keysPrefix     = "/v2/keys"
+	machinesPrefix = "/v2/machines"
+	membersPrefix  = "/v2/members"
+	statsPrefix    = "/v2/stats"
+)
+
+// NewClientHandler generates a muxed http.Handler with the given parameters to serve etcd client requests.
+func NewClientHandler(lg *zap.Logger, server etcdserver.ServerPeer, timeout time.Duration) http.Handler {
+	mux := http.NewServeMux()
+	etcdhttp.HandleBasic(mux, server)
+	handleV2(lg, mux, server, timeout)
+	return requestLogger(lg, mux)
+}
+
+func handleV2(lg *zap.Logger, mux *http.ServeMux, server etcdserver.ServerV2, timeout time.Duration) {
+	sec := v2auth.NewStore(lg, server, timeout)
+	kh := &keysHandler{
+		lg:                    lg,
+		sec:                   sec,
+		server:                server,
+		cluster:               server.Cluster(),
+		timeout:               timeout,
+		clientCertAuthEnabled: server.ClientCertAuthEnabled(),
+	}
+
+	sh := &statsHandler{
+		lg:    lg,
+		stats: server,
+	}
+
+	mh := &membersHandler{
+		lg:                    lg,
+		sec:                   sec,
+		server:                server,
+		cluster:               server.Cluster(),
+		timeout:               timeout,
+		clock:                 clockwork.NewRealClock(),
+		clientCertAuthEnabled: server.ClientCertAuthEnabled(),
+	}
+
+	mah := &machinesHandler{cluster: server.Cluster()}
+
+	sech := &authHandler{
+		lg:                    lg,
+		sec:                   sec,
+		cluster:               server.Cluster(),
+		clientCertAuthEnabled: server.ClientCertAuthEnabled(),
+	}
+	mux.HandleFunc("/", http.NotFound)
+	mux.Handle(keysPrefix, kh)
+	mux.Handle(keysPrefix+"/", kh)
+	mux.HandleFunc(statsPrefix+"/store", sh.serveStore)
+	mux.HandleFunc(statsPrefix+"/self", sh.serveSelf)
+	mux.HandleFunc(statsPrefix+"/leader", sh.serveLeader)
+	mux.Handle(membersPrefix, mh)
+	mux.Handle(membersPrefix+"/", mh)
+	mux.Handle(machinesPrefix, mah)
+	handleAuth(mux, sech)
+}
+
+type keysHandler struct {
+	lg                    *zap.Logger
+	sec                   v2auth.Store
+	server                etcdserver.ServerV2
+	cluster               api.Cluster
+	timeout               time.Duration
+	clientCertAuthEnabled bool
+}
+
+func (h *keysHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
+	if !allowMethod(w, r.Method, "HEAD", "GET", "PUT", "POST", "DELETE") {
+		return
+	}
+
+	w.Header().Set("X-Etcd-Cluster-ID", h.cluster.ID().String())
+
+	ctx, cancel := context.WithTimeout(context.Background(), h.timeout)
+	defer cancel()
+	clock := clockwork.NewRealClock()
+	startTime := clock.Now()
+	rr, noValueOnSuccess, err := parseKeyRequest(r, clock)
+	if err != nil {
+		writeKeyError(h.lg, w, err)
+		return
+	}
+	// The path must be valid at this point (we've parsed the request successfully).
+	if !hasKeyPrefixAccess(h.lg, h.sec, r, r.URL.Path[len(keysPrefix):], rr.Recursive, h.clientCertAuthEnabled) {
+		writeKeyNoAuth(w)
+		return
+	}
+	if !rr.Wait {
+		reportRequestReceived(rr)
+	}
+	resp, err := h.server.Do(ctx, rr)
+	if err != nil {
+		err = trimErrorPrefix(err, etcdserver.StoreKeysPrefix)
+		writeKeyError(h.lg, w, err)
+		reportRequestFailed(rr, err)
+		return
+	}
+	switch {
+	case resp.Event != nil:
+		if err := writeKeyEvent(w, resp, noValueOnSuccess); err != nil {
+			// Should never be reached
+			if h.lg != nil {
+				h.lg.Warn("failed to write key event", zap.Error(err))
+			} else {
+				plog.Errorf("error writing event (%v)", err)
+			}
+		}
+		reportRequestCompleted(rr, startTime)
+	case resp.Watcher != nil:
+		ctx, cancel := context.WithTimeout(context.Background(), defaultWatchTimeout)
+		defer cancel()
+		handleKeyWatch(ctx, h.lg, w, resp, rr.Stream)
+	default:
+		writeKeyError(h.lg, w, errors.New("received response with no Event/Watcher"))
+	}
+}
+
+type machinesHandler struct {
+	cluster api.Cluster
+}
+
+func (h *machinesHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
+	if !allowMethod(w, r.Method, "GET", "HEAD") {
+		return
+	}
+	endpoints := h.cluster.ClientURLs()
+	w.Write([]byte(strings.Join(endpoints, ", ")))
+}
+
+type membersHandler struct {
+	lg                    *zap.Logger
+	sec                   v2auth.Store
+	server                etcdserver.ServerV2
+	cluster               api.Cluster
+	timeout               time.Duration
+	clock                 clockwork.Clock
+	clientCertAuthEnabled bool
+}
+
+func (h *membersHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
+	if !allowMethod(w, r.Method, "GET", "POST", "DELETE", "PUT") {
+		return
+	}
+	if !hasWriteRootAccess(h.lg, h.sec, r, h.clientCertAuthEnabled) {
+		writeNoAuth(h.lg, w, r)
+		return
+	}
+	w.Header().Set("X-Etcd-Cluster-ID", h.cluster.ID().String())
+
+	ctx, cancel := context.WithTimeout(context.Background(), h.timeout)
+	defer cancel()
+
+	switch r.Method {
+	case "GET":
+		switch trimPrefix(r.URL.Path, membersPrefix) {
+		case "":
+			mc := newMemberCollection(h.cluster.Members())
+			w.Header().Set("Content-Type", "application/json")
+			if err := json.NewEncoder(w).Encode(mc); err != nil {
+				if h.lg != nil {
+					h.lg.Warn("failed to encode members response", zap.Error(err))
+				} else {
+					plog.Warningf("failed to encode members response (%v)", err)
+				}
+			}
+		case "leader":
+			id := h.server.Leader()
+			if id == 0 {
+				writeError(h.lg, w, r, httptypes.NewHTTPError(http.StatusServiceUnavailable, "During election"))
+				return
+			}
+			m := newMember(h.cluster.Member(id))
+			w.Header().Set("Content-Type", "application/json")
+			if err := json.NewEncoder(w).Encode(m); err != nil {
+				if h.lg != nil {
+					h.lg.Warn("failed to encode members response", zap.Error(err))
+				} else {
+					plog.Warningf("failed to encode members response (%v)", err)
+				}
+			}
+		default:
+			writeError(h.lg, w, r, httptypes.NewHTTPError(http.StatusNotFound, "Not found"))
+		}
+
+	case "POST":
+		req := httptypes.MemberCreateRequest{}
+		if ok := unmarshalRequest(h.lg, r, &req, w); !ok {
+			return
+		}
+		now := h.clock.Now()
+		m := membership.NewMember("", req.PeerURLs, "", &now)
+		_, err := h.server.AddMember(ctx, *m)
+		switch {
+		case err == membership.ErrIDExists || err == membership.ErrPeerURLexists:
+			writeError(h.lg, w, r, httptypes.NewHTTPError(http.StatusConflict, err.Error()))
+			return
+		case err != nil:
+			if h.lg != nil {
+				h.lg.Warn(
+					"failed to add a member",
+					zap.String("member-id", m.ID.String()),
+					zap.Error(err),
+				)
+			} else {
+				plog.Errorf("error adding member %s (%v)", m.ID, err)
+			}
+			writeError(h.lg, w, r, err)
+			return
+		}
+		res := newMember(m)
+		w.Header().Set("Content-Type", "application/json")
+		w.WriteHeader(http.StatusCreated)
+		if err := json.NewEncoder(w).Encode(res); err != nil {
+			if h.lg != nil {
+				h.lg.Warn("failed to encode members response", zap.Error(err))
+			} else {
+				plog.Warningf("failed to encode members response (%v)", err)
+			}
+		}
+
+	case "DELETE":
+		id, ok := getID(h.lg, r.URL.Path, w)
+		if !ok {
+			return
+		}
+		_, err := h.server.RemoveMember(ctx, uint64(id))
+		switch {
+		case err == membership.ErrIDRemoved:
+			writeError(h.lg, w, r, httptypes.NewHTTPError(http.StatusGone, fmt.Sprintf("Member permanently removed: %s", id)))
+		case err == membership.ErrIDNotFound:
+			writeError(h.lg, w, r, httptypes.NewHTTPError(http.StatusNotFound, fmt.Sprintf("No such member: %s", id)))
+		case err != nil:
+			if h.lg != nil {
+				h.lg.Warn(
+					"failed to remove a member",
+					zap.String("member-id", id.String()),
+					zap.Error(err),
+				)
+			} else {
+				plog.Errorf("error removing member %s (%v)", id, err)
+			}
+			writeError(h.lg, w, r, err)
+		default:
+			w.WriteHeader(http.StatusNoContent)
+		}
+
+	case "PUT":
+		id, ok := getID(h.lg, r.URL.Path, w)
+		if !ok {
+			return
+		}
+		req := httptypes.MemberUpdateRequest{}
+		if ok := unmarshalRequest(h.lg, r, &req, w); !ok {
+			return
+		}
+		m := membership.Member{
+			ID:             id,
+			RaftAttributes: membership.RaftAttributes{PeerURLs: req.PeerURLs.StringSlice()},
+		}
+		_, err := h.server.UpdateMember(ctx, m)
+		switch {
+		case err == membership.ErrPeerURLexists:
+			writeError(h.lg, w, r, httptypes.NewHTTPError(http.StatusConflict, err.Error()))
+		case err == membership.ErrIDNotFound:
+			writeError(h.lg, w, r, httptypes.NewHTTPError(http.StatusNotFound, fmt.Sprintf("No such member: %s", id)))
+		case err != nil:
+			if h.lg != nil {
+				h.lg.Warn(
+					"failed to update a member",
+					zap.String("member-id", m.ID.String()),
+					zap.Error(err),
+				)
+			} else {
+				plog.Errorf("error updating member %s (%v)", m.ID, err)
+			}
+			writeError(h.lg, w, r, err)
+		default:
+			w.WriteHeader(http.StatusNoContent)
+		}
+	}
+}
+
+type statsHandler struct {
+	lg    *zap.Logger
+	stats stats.Stats
+}
+
+func (h *statsHandler) serveStore(w http.ResponseWriter, r *http.Request) {
+	if !allowMethod(w, r.Method, "GET") {
+		return
+	}
+	w.Header().Set("Content-Type", "application/json")
+	w.Write(h.stats.StoreStats())
+}
+
+func (h *statsHandler) serveSelf(w http.ResponseWriter, r *http.Request) {
+	if !allowMethod(w, r.Method, "GET") {
+		return
+	}
+	w.Header().Set("Content-Type", "application/json")
+	w.Write(h.stats.SelfStats())
+}
+
+func (h *statsHandler) serveLeader(w http.ResponseWriter, r *http.Request) {
+	if !allowMethod(w, r.Method, "GET") {
+		return
+	}
+	stats := h.stats.LeaderStats()
+	if stats == nil {
+		etcdhttp.WriteError(h.lg, w, r, httptypes.NewHTTPError(http.StatusForbidden, "not current leader"))
+		return
+	}
+	w.Header().Set("Content-Type", "application/json")
+	w.Write(stats)
+}
+
+// parseKeyRequest converts a received http.Request on keysPrefix to
+// a server Request, performing validation of supplied fields as appropriate.
+// If any validation fails, an empty Request and non-nil error is returned.
+func parseKeyRequest(r *http.Request, clock clockwork.Clock) (etcdserverpb.Request, bool, error) {
+	var noValueOnSuccess bool
+	emptyReq := etcdserverpb.Request{}
+
+	err := r.ParseForm()
+	if err != nil {
+		return emptyReq, false, v2error.NewRequestError(
+			v2error.EcodeInvalidForm,
+			err.Error(),
+		)
+	}
+
+	if !strings.HasPrefix(r.URL.Path, keysPrefix) {
+		return emptyReq, false, v2error.NewRequestError(
+			v2error.EcodeInvalidForm,
+			"incorrect key prefix",
+		)
+	}
+	p := path.Join(etcdserver.StoreKeysPrefix, r.URL.Path[len(keysPrefix):])
+
+	var pIdx, wIdx uint64
+	if pIdx, err = getUint64(r.Form, "prevIndex"); err != nil {
+		return emptyReq, false, v2error.NewRequestError(
+			v2error.EcodeIndexNaN,
+			`invalid value for "prevIndex"`,
+		)
+	}
+	if wIdx, err = getUint64(r.Form, "waitIndex"); err != nil {
+		return emptyReq, false, v2error.NewRequestError(
+			v2error.EcodeIndexNaN,
+			`invalid value for "waitIndex"`,
+		)
+	}
+
+	var rec, sort, wait, dir, quorum, stream bool
+	if rec, err = getBool(r.Form, "recursive"); err != nil {
+		return emptyReq, false, v2error.NewRequestError(
+			v2error.EcodeInvalidField,
+			`invalid value for "recursive"`,
+		)
+	}
+	if sort, err = getBool(r.Form, "sorted"); err != nil {
+		return emptyReq, false, v2error.NewRequestError(
+			v2error.EcodeInvalidField,
+			`invalid value for "sorted"`,
+		)
+	}
+	if wait, err = getBool(r.Form, "wait"); err != nil {
+		return emptyReq, false, v2error.NewRequestError(
+			v2error.EcodeInvalidField,
+			`invalid value for "wait"`,
+		)
+	}
+	// TODO(jonboulle): define what parameters dir is/isn't compatible with?
+	if dir, err = getBool(r.Form, "dir"); err != nil {
+		return emptyReq, false, v2error.NewRequestError(
+			v2error.EcodeInvalidField,
+			`invalid value for "dir"`,
+		)
+	}
+	if quorum, err = getBool(r.Form, "quorum"); err != nil {
+		return emptyReq, false, v2error.NewRequestError(
+			v2error.EcodeInvalidField,
+			`invalid value for "quorum"`,
+		)
+	}
+	if stream, err = getBool(r.Form, "stream"); err != nil {
+		return emptyReq, false, v2error.NewRequestError(
+			v2error.EcodeInvalidField,
+			`invalid value for "stream"`,
+		)
+	}
+
+	if wait && r.Method != "GET" {
+		return emptyReq, false, v2error.NewRequestError(
+			v2error.EcodeInvalidField,
+			`"wait" can only be used with GET requests`,
+		)
+	}
+
+	pV := r.FormValue("prevValue")
+	if _, ok := r.Form["prevValue"]; ok && pV == "" {
+		return emptyReq, false, v2error.NewRequestError(
+			v2error.EcodePrevValueRequired,
+			`"prevValue" cannot be empty`,
+		)
+	}
+
+	if noValueOnSuccess, err = getBool(r.Form, "noValueOnSuccess"); err != nil {
+		return emptyReq, false, v2error.NewRequestError(
+			v2error.EcodeInvalidField,
+			`invalid value for "noValueOnSuccess"`,
+		)
+	}
+
+	// TTL is nullable, so leave it null if not specified
+	// or an empty string
+	var ttl *uint64
+	if len(r.FormValue("ttl")) > 0 {
+		i, err := getUint64(r.Form, "ttl")
+		if err != nil {
+			return emptyReq, false, v2error.NewRequestError(
+				v2error.EcodeTTLNaN,
+				`invalid value for "ttl"`,
+			)
+		}
+		ttl = &i
+	}
+
+	// prevExist is nullable, so leave it null if not specified
+	var pe *bool
+	if _, ok := r.Form["prevExist"]; ok {
+		bv, err := getBool(r.Form, "prevExist")
+		if err != nil {
+			return emptyReq, false, v2error.NewRequestError(
+				v2error.EcodeInvalidField,
+				"invalid value for prevExist",
+			)
+		}
+		pe = &bv
+	}
+
+	// refresh is nullable, so leave it null if not specified
+	var refresh *bool
+	if _, ok := r.Form["refresh"]; ok {
+		bv, err := getBool(r.Form, "refresh")
+		if err != nil {
+			return emptyReq, false, v2error.NewRequestError(
+				v2error.EcodeInvalidField,
+				"invalid value for refresh",
+			)
+		}
+		refresh = &bv
+		if refresh != nil && *refresh {
+			val := r.FormValue("value")
+			if _, ok := r.Form["value"]; ok && val != "" {
+				return emptyReq, false, v2error.NewRequestError(
+					v2error.EcodeRefreshValue,
+					`A value was provided on a refresh`,
+				)
+			}
+			if ttl == nil {
+				return emptyReq, false, v2error.NewRequestError(
+					v2error.EcodeRefreshTTLRequired,
+					`No TTL value set`,
+				)
+			}
+		}
+	}
+
+	rr := etcdserverpb.Request{
+		Method:    r.Method,
+		Path:      p,
+		Val:       r.FormValue("value"),
+		Dir:       dir,
+		PrevValue: pV,
+		PrevIndex: pIdx,
+		PrevExist: pe,
+		Wait:      wait,
+		Since:     wIdx,
+		Recursive: rec,
+		Sorted:    sort,
+		Quorum:    quorum,
+		Stream:    stream,
+	}
+
+	if pe != nil {
+		rr.PrevExist = pe
+	}
+
+	if refresh != nil {
+		rr.Refresh = refresh
+	}
+
+	// Null TTL is equivalent to unset Expiration
+	if ttl != nil {
+		expr := time.Duration(*ttl) * time.Second
+		rr.Expiration = clock.Now().Add(expr).UnixNano()
+	}
+
+	return rr, noValueOnSuccess, nil
+}
+
+// writeKeyEvent trims the prefix of key path in a single Event under
+// StoreKeysPrefix, serializes it and writes the resulting JSON to the given
+// ResponseWriter, along with the appropriate headers.
+func writeKeyEvent(w http.ResponseWriter, resp etcdserver.Response, noValueOnSuccess bool) error {
+	ev := resp.Event
+	if ev == nil {
+		return errors.New("cannot write empty Event")
+	}
+	w.Header().Set("Content-Type", "application/json")
+	w.Header().Set("X-Etcd-Index", fmt.Sprint(ev.EtcdIndex))
+	w.Header().Set("X-Raft-Index", fmt.Sprint(resp.Index))
+	w.Header().Set("X-Raft-Term", fmt.Sprint(resp.Term))
+
+	if ev.IsCreated() {
+		w.WriteHeader(http.StatusCreated)
+	}
+
+	ev = trimEventPrefix(ev, etcdserver.StoreKeysPrefix)
+	if noValueOnSuccess &&
+		(ev.Action == v2store.Set || ev.Action == v2store.CompareAndSwap ||
+			ev.Action == v2store.Create || ev.Action == v2store.Update) {
+		ev.Node = nil
+		ev.PrevNode = nil
+	}
+	return json.NewEncoder(w).Encode(ev)
+}
+
+func writeKeyNoAuth(w http.ResponseWriter) {
+	e := v2error.NewError(v2error.EcodeUnauthorized, "Insufficient credentials", 0)
+	e.WriteTo(w)
+}
+
+// writeKeyError logs and writes the given Error to the ResponseWriter.
+// If Error is not an etcdErr, the error will be converted to an etcd error.
+func writeKeyError(lg *zap.Logger, w http.ResponseWriter, err error) {
+	if err == nil {
+		return
+	}
+	switch e := err.(type) {
+	case *v2error.Error:
+		e.WriteTo(w)
+	default:
+		switch err {
+		case etcdserver.ErrTimeoutDueToLeaderFail, etcdserver.ErrTimeoutDueToConnectionLost:
+			if lg != nil {
+				lg.Warn(
+					"v2 response error",
+					zap.String("internal-server-error", err.Error()),
+				)
+			} else {
+				mlog.MergeError(err)
+			}
+		default:
+			if lg != nil {
+				lg.Warn(
+					"unexpected v2 response error",
+					zap.String("internal-server-error", err.Error()),
+				)
+			} else {
+				mlog.MergeErrorf("got unexpected response error (%v)", err)
+			}
+		}
+		ee := v2error.NewError(v2error.EcodeRaftInternal, err.Error(), 0)
+		ee.WriteTo(w)
+	}
+}
+
+func handleKeyWatch(ctx context.Context, lg *zap.Logger, w http.ResponseWriter, resp etcdserver.Response, stream bool) {
+	wa := resp.Watcher
+	defer wa.Remove()
+	ech := wa.EventChan()
+	var nch <-chan bool
+	if x, ok := w.(http.CloseNotifier); ok {
+		nch = x.CloseNotify()
+	}
+
+	w.Header().Set("Content-Type", "application/json")
+	w.Header().Set("X-Etcd-Index", fmt.Sprint(wa.StartIndex()))
+	w.Header().Set("X-Raft-Index", fmt.Sprint(resp.Index))
+	w.Header().Set("X-Raft-Term", fmt.Sprint(resp.Term))
+	w.WriteHeader(http.StatusOK)
+
+	// Ensure headers are flushed early, in case of long polling
+	w.(http.Flusher).Flush()
+
+	for {
+		select {
+		case <-nch:
+			// Client closed connection. Nothing to do.
+			return
+		case <-ctx.Done():
+			// Timed out. net/http will close the connection for us, so nothing to do.
+			return
+		case ev, ok := <-ech:
+			if !ok {
+				// If the channel is closed this may be an indication of
+				// that notifications are much more than we are able to
+				// send to the client in time. Then we simply end streaming.
+				return
+			}
+			ev = trimEventPrefix(ev, etcdserver.StoreKeysPrefix)
+			if err := json.NewEncoder(w).Encode(ev); err != nil {
+				// Should never be reached
+				if lg != nil {
+					lg.Warn("failed to encode event", zap.Error(err))
+				} else {
+					plog.Warningf("error writing event (%v)", err)
+				}
+				return
+			}
+			if !stream {
+				return
+			}
+			w.(http.Flusher).Flush()
+		}
+	}
+}
+
+func trimEventPrefix(ev *v2store.Event, prefix string) *v2store.Event {
+	if ev == nil {
+		return nil
+	}
+	// Since the *Event may reference one in the store history
+	// history, we must copy it before modifying
+	e := ev.Clone()
+	trimNodeExternPrefix(e.Node, prefix)
+	trimNodeExternPrefix(e.PrevNode, prefix)
+	return e
+}
+
+func trimNodeExternPrefix(n *v2store.NodeExtern, prefix string) {
+	if n == nil {
+		return
+	}
+	n.Key = strings.TrimPrefix(n.Key, prefix)
+	for _, nn := range n.Nodes {
+		trimNodeExternPrefix(nn, prefix)
+	}
+}
+
+func trimErrorPrefix(err error, prefix string) error {
+	if e, ok := err.(*v2error.Error); ok {
+		e.Cause = strings.TrimPrefix(e.Cause, prefix)
+	}
+	return err
+}
+
+func unmarshalRequest(lg *zap.Logger, r *http.Request, req json.Unmarshaler, w http.ResponseWriter) bool {
+	ctype := r.Header.Get("Content-Type")
+	semicolonPosition := strings.Index(ctype, ";")
+	if semicolonPosition != -1 {
+		ctype = strings.TrimSpace(strings.ToLower(ctype[0:semicolonPosition]))
+	}
+	if ctype != "application/json" {
+		writeError(lg, w, r, httptypes.NewHTTPError(http.StatusUnsupportedMediaType, fmt.Sprintf("Bad Content-Type %s, accept application/json", ctype)))
+		return false
+	}
+	b, err := ioutil.ReadAll(r.Body)
+	if err != nil {
+		writeError(lg, w, r, httptypes.NewHTTPError(http.StatusBadRequest, err.Error()))
+		return false
+	}
+	if err := req.UnmarshalJSON(b); err != nil {
+		writeError(lg, w, r, httptypes.NewHTTPError(http.StatusBadRequest, err.Error()))
+		return false
+	}
+	return true
+}
+
+func getID(lg *zap.Logger, p string, w http.ResponseWriter) (types.ID, bool) {
+	idStr := trimPrefix(p, membersPrefix)
+	if idStr == "" {
+		http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed)
+		return 0, false
+	}
+	id, err := types.IDFromString(idStr)
+	if err != nil {
+		writeError(lg, w, nil, httptypes.NewHTTPError(http.StatusNotFound, fmt.Sprintf("No such member: %s", idStr)))
+		return 0, false
+	}
+	return id, true
+}
+
+// getUint64 extracts a uint64 by the given key from a Form. If the key does
+// not exist in the form, 0 is returned. If the key exists but the value is
+// badly formed, an error is returned. If multiple values are present only the
+// first is considered.
+func getUint64(form url.Values, key string) (i uint64, err error) {
+	if vals, ok := form[key]; ok {
+		i, err = strconv.ParseUint(vals[0], 10, 64)
+	}
+	return
+}
+
+// getBool extracts a bool by the given key from a Form. If the key does not
+// exist in the form, false is returned. If the key exists but the value is
+// badly formed, an error is returned. If multiple values are present only the
+// first is considered.
+func getBool(form url.Values, key string) (b bool, err error) {
+	if vals, ok := form[key]; ok {
+		b, err = strconv.ParseBool(vals[0])
+	}
+	return
+}
+
+// trimPrefix removes a given prefix and any slash following the prefix
+// e.g.: trimPrefix("foo", "foo") == trimPrefix("foo/", "foo") == ""
+func trimPrefix(p, prefix string) (s string) {
+	s = strings.TrimPrefix(p, prefix)
+	s = strings.TrimPrefix(s, "/")
+	return
+}
+
+func newMemberCollection(ms []*membership.Member) *httptypes.MemberCollection {
+	c := httptypes.MemberCollection(make([]httptypes.Member, len(ms)))
+
+	for i, m := range ms {
+		c[i] = newMember(m)
+	}
+
+	return &c
+}
+
+func newMember(m *membership.Member) httptypes.Member {
+	tm := httptypes.Member{
+		ID:         m.ID.String(),
+		Name:       m.Name,
+		PeerURLs:   make([]string, len(m.PeerURLs)),
+		ClientURLs: make([]string, len(m.ClientURLs)),
+	}
+
+	copy(tm.PeerURLs, m.PeerURLs)
+	copy(tm.ClientURLs, m.ClientURLs)
+
+	return tm
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2http/client_auth.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2http/client_auth.go
new file mode 100644
index 0000000..d8d6a88
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2http/client_auth.go
@@ -0,0 +1,664 @@
+// Copyright 2015 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 v2http
+
+import (
+	"encoding/json"
+	"net/http"
+	"path"
+	"strings"
+
+	"go.etcd.io/etcd/etcdserver/api"
+	"go.etcd.io/etcd/etcdserver/api/v2auth"
+	"go.etcd.io/etcd/etcdserver/api/v2http/httptypes"
+
+	"go.uber.org/zap"
+)
+
+type authHandler struct {
+	lg                    *zap.Logger
+	sec                   v2auth.Store
+	cluster               api.Cluster
+	clientCertAuthEnabled bool
+}
+
+func hasWriteRootAccess(lg *zap.Logger, sec v2auth.Store, r *http.Request, clientCertAuthEnabled bool) bool {
+	if r.Method == "GET" || r.Method == "HEAD" {
+		return true
+	}
+	return hasRootAccess(lg, sec, r, clientCertAuthEnabled)
+}
+
+func userFromBasicAuth(lg *zap.Logger, sec v2auth.Store, r *http.Request) *v2auth.User {
+	username, password, ok := r.BasicAuth()
+	if !ok {
+		if lg != nil {
+			lg.Warn("malformed basic auth encoding")
+		} else {
+			plog.Warningf("auth: malformed basic auth encoding")
+		}
+		return nil
+	}
+	user, err := sec.GetUser(username)
+	if err != nil {
+		return nil
+	}
+
+	ok = sec.CheckPassword(user, password)
+	if !ok {
+		if lg != nil {
+			lg.Warn("incorrect password", zap.String("user-name", username))
+		} else {
+			plog.Warningf("auth: incorrect password for user: %s", username)
+		}
+		return nil
+	}
+	return &user
+}
+
+func userFromClientCertificate(lg *zap.Logger, sec v2auth.Store, r *http.Request) *v2auth.User {
+	if r.TLS == nil {
+		return nil
+	}
+
+	for _, chains := range r.TLS.VerifiedChains {
+		for _, chain := range chains {
+			if lg != nil {
+				lg.Debug("found common name", zap.String("common-name", chain.Subject.CommonName))
+			} else {
+				plog.Debugf("auth: found common name %s.\n", chain.Subject.CommonName)
+			}
+			user, err := sec.GetUser(chain.Subject.CommonName)
+			if err == nil {
+				if lg != nil {
+					lg.Debug(
+						"authenticated a user via common name",
+						zap.String("user-name", user.User),
+						zap.String("common-name", chain.Subject.CommonName),
+					)
+				} else {
+					plog.Debugf("auth: authenticated user %s by cert common name.", user.User)
+				}
+				return &user
+			}
+		}
+	}
+	return nil
+}
+
+func hasRootAccess(lg *zap.Logger, sec v2auth.Store, r *http.Request, clientCertAuthEnabled bool) bool {
+	if sec == nil {
+		// No store means no auth available, eg, tests.
+		return true
+	}
+	if !sec.AuthEnabled() {
+		return true
+	}
+
+	var rootUser *v2auth.User
+	if r.Header.Get("Authorization") == "" && clientCertAuthEnabled {
+		rootUser = userFromClientCertificate(lg, sec, r)
+		if rootUser == nil {
+			return false
+		}
+	} else {
+		rootUser = userFromBasicAuth(lg, sec, r)
+		if rootUser == nil {
+			return false
+		}
+	}
+
+	for _, role := range rootUser.Roles {
+		if role == v2auth.RootRoleName {
+			return true
+		}
+	}
+
+	if lg != nil {
+		lg.Warn(
+			"a user does not have root role for resource",
+			zap.String("root-user", rootUser.User),
+			zap.String("root-role-name", v2auth.RootRoleName),
+			zap.String("resource-path", r.URL.Path),
+		)
+	} else {
+		plog.Warningf("auth: user %s does not have the %s role for resource %s.", rootUser.User, v2auth.RootRoleName, r.URL.Path)
+	}
+	return false
+}
+
+func hasKeyPrefixAccess(lg *zap.Logger, sec v2auth.Store, r *http.Request, key string, recursive, clientCertAuthEnabled bool) bool {
+	if sec == nil {
+		// No store means no auth available, eg, tests.
+		return true
+	}
+	if !sec.AuthEnabled() {
+		return true
+	}
+
+	var user *v2auth.User
+	if r.Header.Get("Authorization") == "" {
+		if clientCertAuthEnabled {
+			user = userFromClientCertificate(lg, sec, r)
+		}
+		if user == nil {
+			return hasGuestAccess(lg, sec, r, key)
+		}
+	} else {
+		user = userFromBasicAuth(lg, sec, r)
+		if user == nil {
+			return false
+		}
+	}
+
+	writeAccess := r.Method != "GET" && r.Method != "HEAD"
+	for _, roleName := range user.Roles {
+		role, err := sec.GetRole(roleName)
+		if err != nil {
+			continue
+		}
+		if recursive {
+			if role.HasRecursiveAccess(key, writeAccess) {
+				return true
+			}
+		} else if role.HasKeyAccess(key, writeAccess) {
+			return true
+		}
+	}
+
+	if lg != nil {
+		lg.Warn(
+			"invalid access for user on key",
+			zap.String("user-name", user.User),
+			zap.String("key", key),
+		)
+	} else {
+		plog.Warningf("auth: invalid access for user %s on key %s.", user.User, key)
+	}
+	return false
+}
+
+func hasGuestAccess(lg *zap.Logger, sec v2auth.Store, r *http.Request, key string) bool {
+	writeAccess := r.Method != "GET" && r.Method != "HEAD"
+	role, err := sec.GetRole(v2auth.GuestRoleName)
+	if err != nil {
+		return false
+	}
+	if role.HasKeyAccess(key, writeAccess) {
+		return true
+	}
+
+	if lg != nil {
+		lg.Warn(
+			"invalid access for a guest role on key",
+			zap.String("role-name", v2auth.GuestRoleName),
+			zap.String("key", key),
+		)
+	} else {
+		plog.Warningf("auth: invalid access for unauthenticated user on resource %s.", key)
+	}
+	return false
+}
+
+func writeNoAuth(lg *zap.Logger, w http.ResponseWriter, r *http.Request) {
+	herr := httptypes.NewHTTPError(http.StatusUnauthorized, "Insufficient credentials")
+	if err := herr.WriteTo(w); err != nil {
+		if lg != nil {
+			lg.Debug(
+				"failed to write v2 HTTP error",
+				zap.String("remote-addr", r.RemoteAddr),
+				zap.Error(err),
+			)
+		} else {
+			plog.Debugf("error writing HTTPError (%v) to %s", err, r.RemoteAddr)
+		}
+	}
+}
+
+func handleAuth(mux *http.ServeMux, sh *authHandler) {
+	mux.HandleFunc(authPrefix+"/roles", authCapabilityHandler(sh.baseRoles))
+	mux.HandleFunc(authPrefix+"/roles/", authCapabilityHandler(sh.handleRoles))
+	mux.HandleFunc(authPrefix+"/users", authCapabilityHandler(sh.baseUsers))
+	mux.HandleFunc(authPrefix+"/users/", authCapabilityHandler(sh.handleUsers))
+	mux.HandleFunc(authPrefix+"/enable", authCapabilityHandler(sh.enableDisable))
+}
+
+func (sh *authHandler) baseRoles(w http.ResponseWriter, r *http.Request) {
+	if !allowMethod(w, r.Method, "GET") {
+		return
+	}
+	if !hasRootAccess(sh.lg, sh.sec, r, sh.clientCertAuthEnabled) {
+		writeNoAuth(sh.lg, w, r)
+		return
+	}
+
+	w.Header().Set("X-Etcd-Cluster-ID", sh.cluster.ID().String())
+	w.Header().Set("Content-Type", "application/json")
+
+	roles, err := sh.sec.AllRoles()
+	if err != nil {
+		writeError(sh.lg, w, r, err)
+		return
+	}
+	if roles == nil {
+		roles = make([]string, 0)
+	}
+
+	err = r.ParseForm()
+	if err != nil {
+		writeError(sh.lg, w, r, err)
+		return
+	}
+
+	var rolesCollections struct {
+		Roles []v2auth.Role `json:"roles"`
+	}
+	for _, roleName := range roles {
+		var role v2auth.Role
+		role, err = sh.sec.GetRole(roleName)
+		if err != nil {
+			writeError(sh.lg, w, r, err)
+			return
+		}
+		rolesCollections.Roles = append(rolesCollections.Roles, role)
+	}
+	err = json.NewEncoder(w).Encode(rolesCollections)
+
+	if err != nil {
+		if sh.lg != nil {
+			sh.lg.Warn(
+				"failed to encode base roles",
+				zap.String("url", r.URL.String()),
+				zap.Error(err),
+			)
+		} else {
+			plog.Warningf("baseRoles error encoding on %s", r.URL)
+		}
+		writeError(sh.lg, w, r, err)
+		return
+	}
+}
+
+func (sh *authHandler) handleRoles(w http.ResponseWriter, r *http.Request) {
+	subpath := path.Clean(r.URL.Path[len(authPrefix):])
+	// Split "/roles/rolename/command".
+	// First item is an empty string, second is "roles"
+	pieces := strings.Split(subpath, "/")
+	if len(pieces) == 2 {
+		sh.baseRoles(w, r)
+		return
+	}
+	if len(pieces) != 3 {
+		writeError(sh.lg, w, r, httptypes.NewHTTPError(http.StatusBadRequest, "Invalid path"))
+		return
+	}
+	sh.forRole(w, r, pieces[2])
+}
+
+func (sh *authHandler) forRole(w http.ResponseWriter, r *http.Request, role string) {
+	if !allowMethod(w, r.Method, "GET", "PUT", "DELETE") {
+		return
+	}
+	if !hasRootAccess(sh.lg, sh.sec, r, sh.clientCertAuthEnabled) {
+		writeNoAuth(sh.lg, w, r)
+		return
+	}
+	w.Header().Set("X-Etcd-Cluster-ID", sh.cluster.ID().String())
+	w.Header().Set("Content-Type", "application/json")
+
+	switch r.Method {
+	case "GET":
+		data, err := sh.sec.GetRole(role)
+		if err != nil {
+			writeError(sh.lg, w, r, err)
+			return
+		}
+		err = json.NewEncoder(w).Encode(data)
+		if err != nil {
+			if sh.lg != nil {
+				sh.lg.Warn(
+					"failed to encode a role",
+					zap.String("url", r.URL.String()),
+					zap.Error(err),
+				)
+			} else {
+				plog.Warningf("forRole error encoding on %s", r.URL)
+			}
+			return
+		}
+		return
+
+	case "PUT":
+		var in v2auth.Role
+		err := json.NewDecoder(r.Body).Decode(&in)
+		if err != nil {
+			writeError(sh.lg, w, r, httptypes.NewHTTPError(http.StatusBadRequest, "Invalid JSON in request body."))
+			return
+		}
+		if in.Role != role {
+			writeError(sh.lg, w, r, httptypes.NewHTTPError(http.StatusBadRequest, "Role JSON name does not match the name in the URL"))
+			return
+		}
+
+		var out v2auth.Role
+
+		// create
+		if in.Grant.IsEmpty() && in.Revoke.IsEmpty() {
+			err = sh.sec.CreateRole(in)
+			if err != nil {
+				writeError(sh.lg, w, r, err)
+				return
+			}
+			w.WriteHeader(http.StatusCreated)
+			out = in
+		} else {
+			if !in.Permissions.IsEmpty() {
+				writeError(sh.lg, w, r, httptypes.NewHTTPError(http.StatusBadRequest, "Role JSON contains both permissions and grant/revoke"))
+				return
+			}
+			out, err = sh.sec.UpdateRole(in)
+			if err != nil {
+				writeError(sh.lg, w, r, err)
+				return
+			}
+			w.WriteHeader(http.StatusOK)
+		}
+
+		err = json.NewEncoder(w).Encode(out)
+		if err != nil {
+			if sh.lg != nil {
+				sh.lg.Warn(
+					"failed to encode a role",
+					zap.String("url", r.URL.String()),
+					zap.Error(err),
+				)
+			} else {
+				plog.Warningf("forRole error encoding on %s", r.URL)
+			}
+			return
+		}
+		return
+
+	case "DELETE":
+		err := sh.sec.DeleteRole(role)
+		if err != nil {
+			writeError(sh.lg, w, r, err)
+			return
+		}
+	}
+}
+
+type userWithRoles struct {
+	User  string        `json:"user"`
+	Roles []v2auth.Role `json:"roles,omitempty"`
+}
+
+type usersCollections struct {
+	Users []userWithRoles `json:"users"`
+}
+
+func (sh *authHandler) baseUsers(w http.ResponseWriter, r *http.Request) {
+	if !allowMethod(w, r.Method, "GET") {
+		return
+	}
+	if !hasRootAccess(sh.lg, sh.sec, r, sh.clientCertAuthEnabled) {
+		writeNoAuth(sh.lg, w, r)
+		return
+	}
+	w.Header().Set("X-Etcd-Cluster-ID", sh.cluster.ID().String())
+	w.Header().Set("Content-Type", "application/json")
+
+	users, err := sh.sec.AllUsers()
+	if err != nil {
+		writeError(sh.lg, w, r, err)
+		return
+	}
+	if users == nil {
+		users = make([]string, 0)
+	}
+
+	err = r.ParseForm()
+	if err != nil {
+		writeError(sh.lg, w, r, err)
+		return
+	}
+
+	ucs := usersCollections{}
+	for _, userName := range users {
+		var user v2auth.User
+		user, err = sh.sec.GetUser(userName)
+		if err != nil {
+			writeError(sh.lg, w, r, err)
+			return
+		}
+
+		uwr := userWithRoles{User: user.User}
+		for _, roleName := range user.Roles {
+			var role v2auth.Role
+			role, err = sh.sec.GetRole(roleName)
+			if err != nil {
+				continue
+			}
+			uwr.Roles = append(uwr.Roles, role)
+		}
+
+		ucs.Users = append(ucs.Users, uwr)
+	}
+	err = json.NewEncoder(w).Encode(ucs)
+
+	if err != nil {
+		if sh.lg != nil {
+			sh.lg.Warn(
+				"failed to encode users",
+				zap.String("url", r.URL.String()),
+				zap.Error(err),
+			)
+		} else {
+			plog.Warningf("baseUsers error encoding on %s", r.URL)
+		}
+		writeError(sh.lg, w, r, err)
+		return
+	}
+}
+
+func (sh *authHandler) handleUsers(w http.ResponseWriter, r *http.Request) {
+	subpath := path.Clean(r.URL.Path[len(authPrefix):])
+	// Split "/users/username".
+	// First item is an empty string, second is "users"
+	pieces := strings.Split(subpath, "/")
+	if len(pieces) == 2 {
+		sh.baseUsers(w, r)
+		return
+	}
+	if len(pieces) != 3 {
+		writeError(sh.lg, w, r, httptypes.NewHTTPError(http.StatusBadRequest, "Invalid path"))
+		return
+	}
+	sh.forUser(w, r, pieces[2])
+}
+
+func (sh *authHandler) forUser(w http.ResponseWriter, r *http.Request, user string) {
+	if !allowMethod(w, r.Method, "GET", "PUT", "DELETE") {
+		return
+	}
+	if !hasRootAccess(sh.lg, sh.sec, r, sh.clientCertAuthEnabled) {
+		writeNoAuth(sh.lg, w, r)
+		return
+	}
+	w.Header().Set("X-Etcd-Cluster-ID", sh.cluster.ID().String())
+	w.Header().Set("Content-Type", "application/json")
+
+	switch r.Method {
+	case "GET":
+		u, err := sh.sec.GetUser(user)
+		if err != nil {
+			writeError(sh.lg, w, r, err)
+			return
+		}
+
+		err = r.ParseForm()
+		if err != nil {
+			writeError(sh.lg, w, r, err)
+			return
+		}
+
+		uwr := userWithRoles{User: u.User}
+		for _, roleName := range u.Roles {
+			var role v2auth.Role
+			role, err = sh.sec.GetRole(roleName)
+			if err != nil {
+				writeError(sh.lg, w, r, err)
+				return
+			}
+			uwr.Roles = append(uwr.Roles, role)
+		}
+		err = json.NewEncoder(w).Encode(uwr)
+
+		if err != nil {
+			if sh.lg != nil {
+				sh.lg.Warn(
+					"failed to encode roles",
+					zap.String("url", r.URL.String()),
+					zap.Error(err),
+				)
+			} else {
+				plog.Warningf("forUser error encoding on %s", r.URL)
+			}
+			return
+		}
+		return
+
+	case "PUT":
+		var u v2auth.User
+		err := json.NewDecoder(r.Body).Decode(&u)
+		if err != nil {
+			writeError(sh.lg, w, r, httptypes.NewHTTPError(http.StatusBadRequest, "Invalid JSON in request body."))
+			return
+		}
+		if u.User != user {
+			writeError(sh.lg, w, r, httptypes.NewHTTPError(http.StatusBadRequest, "User JSON name does not match the name in the URL"))
+			return
+		}
+
+		var (
+			out     v2auth.User
+			created bool
+		)
+
+		if len(u.Grant) == 0 && len(u.Revoke) == 0 {
+			// create or update
+			if len(u.Roles) != 0 {
+				out, err = sh.sec.CreateUser(u)
+			} else {
+				// if user passes in both password and roles, we are unsure about his/her
+				// intention.
+				out, created, err = sh.sec.CreateOrUpdateUser(u)
+			}
+
+			if err != nil {
+				writeError(sh.lg, w, r, err)
+				return
+			}
+		} else {
+			// update case
+			if len(u.Roles) != 0 {
+				writeError(sh.lg, w, r, httptypes.NewHTTPError(http.StatusBadRequest, "User JSON contains both roles and grant/revoke"))
+				return
+			}
+			out, err = sh.sec.UpdateUser(u)
+			if err != nil {
+				writeError(sh.lg, w, r, err)
+				return
+			}
+		}
+
+		if created {
+			w.WriteHeader(http.StatusCreated)
+		} else {
+			w.WriteHeader(http.StatusOK)
+		}
+
+		out.Password = ""
+
+		err = json.NewEncoder(w).Encode(out)
+		if err != nil {
+			if sh.lg != nil {
+				sh.lg.Warn(
+					"failed to encode a user",
+					zap.String("url", r.URL.String()),
+					zap.Error(err),
+				)
+			} else {
+				plog.Warningf("forUser error encoding on %s", r.URL)
+			}
+			return
+		}
+		return
+
+	case "DELETE":
+		err := sh.sec.DeleteUser(user)
+		if err != nil {
+			writeError(sh.lg, w, r, err)
+			return
+		}
+	}
+}
+
+type enabled struct {
+	Enabled bool `json:"enabled"`
+}
+
+func (sh *authHandler) enableDisable(w http.ResponseWriter, r *http.Request) {
+	if !allowMethod(w, r.Method, "GET", "PUT", "DELETE") {
+		return
+	}
+	if !hasWriteRootAccess(sh.lg, sh.sec, r, sh.clientCertAuthEnabled) {
+		writeNoAuth(sh.lg, w, r)
+		return
+	}
+	w.Header().Set("X-Etcd-Cluster-ID", sh.cluster.ID().String())
+	w.Header().Set("Content-Type", "application/json")
+	isEnabled := sh.sec.AuthEnabled()
+	switch r.Method {
+	case "GET":
+		jsonDict := enabled{isEnabled}
+		err := json.NewEncoder(w).Encode(jsonDict)
+		if err != nil {
+			if sh.lg != nil {
+				sh.lg.Warn(
+					"failed to encode a auth state",
+					zap.String("url", r.URL.String()),
+					zap.Error(err),
+				)
+			} else {
+				plog.Warningf("error encoding auth state on %s", r.URL)
+			}
+		}
+
+	case "PUT":
+		err := sh.sec.EnableAuth()
+		if err != nil {
+			writeError(sh.lg, w, r, err)
+			return
+		}
+
+	case "DELETE":
+		err := sh.sec.DisableAuth()
+		if err != nil {
+			writeError(sh.lg, w, r, err)
+			return
+		}
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2http/doc.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2http/doc.go
new file mode 100644
index 0000000..475c4b1
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2http/doc.go
@@ -0,0 +1,16 @@
+// Copyright 2015 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 v2http provides etcd client and server implementations.
+package v2http
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2http/http.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2http/http.go
new file mode 100644
index 0000000..c695689
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2http/http.go
@@ -0,0 +1,93 @@
+// Copyright 2015 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 v2http
+
+import (
+	"math"
+	"net/http"
+	"strings"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver/api/etcdhttp"
+	"go.etcd.io/etcd/etcdserver/api/v2auth"
+	"go.etcd.io/etcd/etcdserver/api/v2http/httptypes"
+	"go.etcd.io/etcd/pkg/logutil"
+
+	"github.com/coreos/pkg/capnslog"
+	"go.uber.org/zap"
+)
+
+const (
+	// time to wait for a Watch request
+	defaultWatchTimeout = time.Duration(math.MaxInt64)
+)
+
+var (
+	plog = capnslog.NewPackageLogger("go.etcd.io/etcd", "etcdserver/api/v2http")
+	mlog = logutil.NewMergeLogger(plog)
+)
+
+func writeError(lg *zap.Logger, w http.ResponseWriter, r *http.Request, err error) {
+	if err == nil {
+		return
+	}
+	if e, ok := err.(v2auth.Error); ok {
+		herr := httptypes.NewHTTPError(e.HTTPStatus(), e.Error())
+		if et := herr.WriteTo(w); et != nil {
+			if lg != nil {
+				lg.Debug(
+					"failed to write v2 HTTP error",
+					zap.String("remote-addr", r.RemoteAddr),
+					zap.String("v2auth-error", e.Error()),
+					zap.Error(et),
+				)
+			} else {
+				plog.Debugf("error writing HTTPError (%v) to %s", et, r.RemoteAddr)
+			}
+		}
+		return
+	}
+	etcdhttp.WriteError(lg, w, r, err)
+}
+
+// allowMethod verifies that the given method is one of the allowed methods,
+// and if not, it writes an error to w.  A boolean is returned indicating
+// whether or not the method is allowed.
+func allowMethod(w http.ResponseWriter, m string, ms ...string) bool {
+	for _, meth := range ms {
+		if m == meth {
+			return true
+		}
+	}
+	w.Header().Set("Allow", strings.Join(ms, ","))
+	http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed)
+	return false
+}
+
+func requestLogger(lg *zap.Logger, handler http.Handler) http.Handler {
+	return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		if lg != nil {
+			lg.Debug(
+				"handling HTTP request",
+				zap.String("method", r.Method),
+				zap.String("request-uri", r.RequestURI),
+				zap.String("remote-addr", r.RemoteAddr),
+			)
+		} else {
+			plog.Debugf("[%s] %s remote:%s", r.Method, r.RequestURI, r.RemoteAddr)
+		}
+		handler.ServeHTTP(w, r)
+	})
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2http/httptypes/errors.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2http/httptypes/errors.go
new file mode 100644
index 0000000..245c089
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2http/httptypes/errors.go
@@ -0,0 +1,56 @@
+// Copyright 2015 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 httptypes
+
+import (
+	"encoding/json"
+	"net/http"
+
+	"github.com/coreos/pkg/capnslog"
+)
+
+var (
+	plog = capnslog.NewPackageLogger("go.etcd.io/etcd", "etcdserver/api/v2http/httptypes")
+)
+
+type HTTPError struct {
+	Message string `json:"message"`
+	// Code is the HTTP status code
+	Code int `json:"-"`
+}
+
+func (e HTTPError) Error() string {
+	return e.Message
+}
+
+func (e HTTPError) WriteTo(w http.ResponseWriter) error {
+	w.Header().Set("Content-Type", "application/json")
+	w.WriteHeader(e.Code)
+	b, err := json.Marshal(e)
+	if err != nil {
+		plog.Panicf("marshal HTTPError should never fail (%v)", err)
+	}
+	if _, err := w.Write(b); err != nil {
+		return err
+	}
+	return nil
+}
+
+func NewHTTPError(code int, m string) *HTTPError {
+	return &HTTPError{
+		Message: m,
+		Code:    code,
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2http/httptypes/member.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2http/httptypes/member.go
new file mode 100644
index 0000000..95fd443
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2http/httptypes/member.go
@@ -0,0 +1,69 @@
+// Copyright 2015 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 httptypes defines how etcd's HTTP API entities are serialized to and
+// deserialized from JSON.
+package httptypes
+
+import (
+	"encoding/json"
+
+	"go.etcd.io/etcd/pkg/types"
+)
+
+type Member struct {
+	ID         string   `json:"id"`
+	Name       string   `json:"name"`
+	PeerURLs   []string `json:"peerURLs"`
+	ClientURLs []string `json:"clientURLs"`
+}
+
+type MemberCreateRequest struct {
+	PeerURLs types.URLs
+}
+
+type MemberUpdateRequest struct {
+	MemberCreateRequest
+}
+
+func (m *MemberCreateRequest) UnmarshalJSON(data []byte) error {
+	s := struct {
+		PeerURLs []string `json:"peerURLs"`
+	}{}
+
+	err := json.Unmarshal(data, &s)
+	if err != nil {
+		return err
+	}
+
+	urls, err := types.NewURLs(s.PeerURLs)
+	if err != nil {
+		return err
+	}
+
+	m.PeerURLs = urls
+	return nil
+}
+
+type MemberCollection []Member
+
+func (c *MemberCollection) MarshalJSON() ([]byte, error) {
+	d := struct {
+		Members []Member `json:"members"`
+	}{
+		Members: []Member(*c),
+	}
+
+	return json.Marshal(d)
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2http/metrics.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2http/metrics.go
new file mode 100644
index 0000000..14f7da0
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2http/metrics.go
@@ -0,0 +1,99 @@
+// Copyright 2015 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 v2http
+
+import (
+	"strconv"
+	"time"
+
+	"net/http"
+
+	"go.etcd.io/etcd/etcdserver/api/v2error"
+	"go.etcd.io/etcd/etcdserver/api/v2http/httptypes"
+	"go.etcd.io/etcd/etcdserver/etcdserverpb"
+
+	"github.com/prometheus/client_golang/prometheus"
+)
+
+var (
+	incomingEvents = prometheus.NewCounterVec(
+		prometheus.CounterOpts{
+			Namespace: "etcd",
+			Subsystem: "http",
+			Name:      "received_total",
+			Help:      "Counter of requests received into the system (successfully parsed and authd).",
+		}, []string{"method"})
+
+	failedEvents = prometheus.NewCounterVec(
+		prometheus.CounterOpts{
+			Namespace: "etcd",
+			Subsystem: "http",
+			Name:      "failed_total",
+			Help:      "Counter of handle failures of requests (non-watches), by method (GET/PUT etc.) and code (400, 500 etc.).",
+		}, []string{"method", "code"})
+
+	successfulEventsHandlingSec = prometheus.NewHistogramVec(
+		prometheus.HistogramOpts{
+			Namespace: "etcd",
+			Subsystem: "http",
+			Name:      "successful_duration_seconds",
+			Help:      "Bucketed histogram of processing time (s) of successfully handled requests (non-watches), by method (GET/PUT etc.).",
+
+			// lowest bucket start of upper bound 0.0005 sec (0.5 ms) with factor 2
+			// highest bucket start of 0.0005 sec * 2^12 == 2.048 sec
+			Buckets: prometheus.ExponentialBuckets(0.0005, 2, 13),
+		}, []string{"method"})
+)
+
+func init() {
+	prometheus.MustRegister(incomingEvents)
+	prometheus.MustRegister(failedEvents)
+	prometheus.MustRegister(successfulEventsHandlingSec)
+}
+
+func reportRequestReceived(request etcdserverpb.Request) {
+	incomingEvents.WithLabelValues(methodFromRequest(request)).Inc()
+}
+
+func reportRequestCompleted(request etcdserverpb.Request, startTime time.Time) {
+	method := methodFromRequest(request)
+	successfulEventsHandlingSec.WithLabelValues(method).Observe(time.Since(startTime).Seconds())
+}
+
+func reportRequestFailed(request etcdserverpb.Request, err error) {
+	method := methodFromRequest(request)
+	failedEvents.WithLabelValues(method, strconv.Itoa(codeFromError(err))).Inc()
+}
+
+func methodFromRequest(request etcdserverpb.Request) string {
+	if request.Method == "GET" && request.Quorum {
+		return "QGET"
+	}
+	return request.Method
+}
+
+func codeFromError(err error) int {
+	if err == nil {
+		return http.StatusInternalServerError
+	}
+	switch e := err.(type) {
+	case *v2error.Error:
+		return e.StatusCode()
+	case *httptypes.HTTPError:
+		return e.Code
+	default:
+		return http.StatusInternalServerError
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2stats/leader.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2stats/leader.go
new file mode 100644
index 0000000..ca47f0f
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2stats/leader.go
@@ -0,0 +1,128 @@
+// Copyright 2015 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 v2stats
+
+import (
+	"encoding/json"
+	"math"
+	"sync"
+	"time"
+)
+
+// LeaderStats is used by the leader in an etcd cluster, and encapsulates
+// statistics about communication with its followers
+type LeaderStats struct {
+	leaderStats
+	sync.Mutex
+}
+
+type leaderStats struct {
+	// Leader is the ID of the leader in the etcd cluster.
+	// TODO(jonboulle): clarify that these are IDs, not names
+	Leader    string                    `json:"leader"`
+	Followers map[string]*FollowerStats `json:"followers"`
+}
+
+// NewLeaderStats generates a new LeaderStats with the given id as leader
+func NewLeaderStats(id string) *LeaderStats {
+	return &LeaderStats{
+		leaderStats: leaderStats{
+			Leader:    id,
+			Followers: make(map[string]*FollowerStats),
+		},
+	}
+}
+
+func (ls *LeaderStats) JSON() []byte {
+	ls.Lock()
+	stats := ls.leaderStats
+	ls.Unlock()
+	b, err := json.Marshal(stats)
+	// TODO(jonboulle): appropriate error handling?
+	if err != nil {
+		plog.Errorf("error marshalling leader stats (%v)", err)
+	}
+	return b
+}
+
+func (ls *LeaderStats) Follower(name string) *FollowerStats {
+	ls.Lock()
+	defer ls.Unlock()
+	fs, ok := ls.Followers[name]
+	if !ok {
+		fs = &FollowerStats{}
+		fs.Latency.Minimum = 1 << 63
+		ls.Followers[name] = fs
+	}
+	return fs
+}
+
+// FollowerStats encapsulates various statistics about a follower in an etcd cluster
+type FollowerStats struct {
+	Latency LatencyStats `json:"latency"`
+	Counts  CountsStats  `json:"counts"`
+
+	sync.Mutex
+}
+
+// LatencyStats encapsulates latency statistics.
+type LatencyStats struct {
+	Current           float64 `json:"current"`
+	Average           float64 `json:"average"`
+	averageSquare     float64
+	StandardDeviation float64 `json:"standardDeviation"`
+	Minimum           float64 `json:"minimum"`
+	Maximum           float64 `json:"maximum"`
+}
+
+// CountsStats encapsulates raft statistics.
+type CountsStats struct {
+	Fail    uint64 `json:"fail"`
+	Success uint64 `json:"success"`
+}
+
+// Succ updates the FollowerStats with a successful send
+func (fs *FollowerStats) Succ(d time.Duration) {
+	fs.Lock()
+	defer fs.Unlock()
+
+	total := float64(fs.Counts.Success) * fs.Latency.Average
+	totalSquare := float64(fs.Counts.Success) * fs.Latency.averageSquare
+
+	fs.Counts.Success++
+
+	fs.Latency.Current = float64(d) / (1000000.0)
+
+	if fs.Latency.Current > fs.Latency.Maximum {
+		fs.Latency.Maximum = fs.Latency.Current
+	}
+
+	if fs.Latency.Current < fs.Latency.Minimum {
+		fs.Latency.Minimum = fs.Latency.Current
+	}
+
+	fs.Latency.Average = (total + fs.Latency.Current) / float64(fs.Counts.Success)
+	fs.Latency.averageSquare = (totalSquare + fs.Latency.Current*fs.Latency.Current) / float64(fs.Counts.Success)
+
+	// sdv = sqrt(avg(x^2) - avg(x)^2)
+	fs.Latency.StandardDeviation = math.Sqrt(fs.Latency.averageSquare - fs.Latency.Average*fs.Latency.Average)
+}
+
+// Fail updates the FollowerStats with an unsuccessful send
+func (fs *FollowerStats) Fail() {
+	fs.Lock()
+	defer fs.Unlock()
+	fs.Counts.Fail++
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2stats/queue.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2stats/queue.go
new file mode 100644
index 0000000..2c3dff3
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2stats/queue.go
@@ -0,0 +1,110 @@
+// Copyright 2015 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 v2stats
+
+import (
+	"sync"
+	"time"
+)
+
+const (
+	queueCapacity = 200
+)
+
+// RequestStats represent the stats for a request.
+// It encapsulates the sending time and the size of the request.
+type RequestStats struct {
+	SendingTime time.Time
+	Size        int
+}
+
+type statsQueue struct {
+	items        [queueCapacity]*RequestStats
+	size         int
+	front        int
+	back         int
+	totalReqSize int
+	rwl          sync.RWMutex
+}
+
+func (q *statsQueue) Len() int {
+	return q.size
+}
+
+func (q *statsQueue) ReqSize() int {
+	return q.totalReqSize
+}
+
+// FrontAndBack gets the front and back elements in the queue
+// We must grab front and back together with the protection of the lock
+func (q *statsQueue) frontAndBack() (*RequestStats, *RequestStats) {
+	q.rwl.RLock()
+	defer q.rwl.RUnlock()
+	if q.size != 0 {
+		return q.items[q.front], q.items[q.back]
+	}
+	return nil, nil
+}
+
+// Insert function insert a RequestStats into the queue and update the records
+func (q *statsQueue) Insert(p *RequestStats) {
+	q.rwl.Lock()
+	defer q.rwl.Unlock()
+
+	q.back = (q.back + 1) % queueCapacity
+
+	if q.size == queueCapacity { //dequeue
+		q.totalReqSize -= q.items[q.front].Size
+		q.front = (q.back + 1) % queueCapacity
+	} else {
+		q.size++
+	}
+
+	q.items[q.back] = p
+	q.totalReqSize += q.items[q.back].Size
+
+}
+
+// Rate function returns the package rate and byte rate
+func (q *statsQueue) Rate() (float64, float64) {
+	front, back := q.frontAndBack()
+
+	if front == nil || back == nil {
+		return 0, 0
+	}
+
+	if time.Since(back.SendingTime) > time.Second {
+		q.Clear()
+		return 0, 0
+	}
+
+	sampleDuration := back.SendingTime.Sub(front.SendingTime)
+
+	pr := float64(q.Len()) / float64(sampleDuration) * float64(time.Second)
+
+	br := float64(q.ReqSize()) / float64(sampleDuration) * float64(time.Second)
+
+	return pr, br
+}
+
+// Clear function clear up the statsQueue
+func (q *statsQueue) Clear() {
+	q.rwl.Lock()
+	defer q.rwl.Unlock()
+	q.back = -1
+	q.front = 0
+	q.size = 0
+	q.totalReqSize = 0
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2stats/server.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2stats/server.go
new file mode 100644
index 0000000..c4accc7
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2stats/server.go
@@ -0,0 +1,142 @@
+// Copyright 2015 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 v2stats
+
+import (
+	"encoding/json"
+	"log"
+	"sync"
+	"time"
+
+	"go.etcd.io/etcd/raft"
+)
+
+// ServerStats encapsulates various statistics about an EtcdServer and its
+// communication with other members of the cluster
+type ServerStats struct {
+	serverStats
+	sync.Mutex
+}
+
+func NewServerStats(name, id string) *ServerStats {
+	ss := &ServerStats{
+		serverStats: serverStats{
+			Name: name,
+			ID:   id,
+		},
+	}
+	now := time.Now()
+	ss.StartTime = now
+	ss.LeaderInfo.StartTime = now
+	ss.sendRateQueue = &statsQueue{back: -1}
+	ss.recvRateQueue = &statsQueue{back: -1}
+	return ss
+}
+
+type serverStats struct {
+	Name string `json:"name"`
+	// ID is the raft ID of the node.
+	// TODO(jonboulle): use ID instead of name?
+	ID        string         `json:"id"`
+	State     raft.StateType `json:"state"`
+	StartTime time.Time      `json:"startTime"`
+
+	LeaderInfo struct {
+		Name      string    `json:"leader"`
+		Uptime    string    `json:"uptime"`
+		StartTime time.Time `json:"startTime"`
+	} `json:"leaderInfo"`
+
+	RecvAppendRequestCnt uint64  `json:"recvAppendRequestCnt,"`
+	RecvingPkgRate       float64 `json:"recvPkgRate,omitempty"`
+	RecvingBandwidthRate float64 `json:"recvBandwidthRate,omitempty"`
+
+	SendAppendRequestCnt uint64  `json:"sendAppendRequestCnt"`
+	SendingPkgRate       float64 `json:"sendPkgRate,omitempty"`
+	SendingBandwidthRate float64 `json:"sendBandwidthRate,omitempty"`
+
+	sendRateQueue *statsQueue
+	recvRateQueue *statsQueue
+}
+
+func (ss *ServerStats) JSON() []byte {
+	ss.Lock()
+	stats := ss.serverStats
+	stats.SendingPkgRate, stats.SendingBandwidthRate = stats.sendRateQueue.Rate()
+	stats.RecvingPkgRate, stats.RecvingBandwidthRate = stats.recvRateQueue.Rate()
+	stats.LeaderInfo.Uptime = time.Since(stats.LeaderInfo.StartTime).String()
+	ss.Unlock()
+	b, err := json.Marshal(stats)
+	// TODO(jonboulle): appropriate error handling?
+	if err != nil {
+		log.Printf("stats: error marshalling server stats: %v", err)
+	}
+	return b
+}
+
+// RecvAppendReq updates the ServerStats in response to an AppendRequest
+// from the given leader being received
+func (ss *ServerStats) RecvAppendReq(leader string, reqSize int) {
+	ss.Lock()
+	defer ss.Unlock()
+
+	now := time.Now()
+
+	ss.State = raft.StateFollower
+	if leader != ss.LeaderInfo.Name {
+		ss.LeaderInfo.Name = leader
+		ss.LeaderInfo.StartTime = now
+	}
+
+	ss.recvRateQueue.Insert(
+		&RequestStats{
+			SendingTime: now,
+			Size:        reqSize,
+		},
+	)
+	ss.RecvAppendRequestCnt++
+}
+
+// SendAppendReq updates the ServerStats in response to an AppendRequest
+// being sent by this server
+func (ss *ServerStats) SendAppendReq(reqSize int) {
+	ss.Lock()
+	defer ss.Unlock()
+
+	ss.becomeLeader()
+
+	ss.sendRateQueue.Insert(
+		&RequestStats{
+			SendingTime: time.Now(),
+			Size:        reqSize,
+		},
+	)
+
+	ss.SendAppendRequestCnt++
+}
+
+func (ss *ServerStats) BecomeLeader() {
+	ss.Lock()
+	defer ss.Unlock()
+	ss.becomeLeader()
+}
+
+func (ss *ServerStats) becomeLeader() {
+	if ss.State != raft.StateLeader {
+		ss.State = raft.StateLeader
+		ss.LeaderInfo.Name = ss.ID
+		ss.LeaderInfo.StartTime = time.Now()
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2stats/stats.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2stats/stats.go
new file mode 100644
index 0000000..c50a200
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2stats/stats.go
@@ -0,0 +1,30 @@
+// Copyright 2015 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 v2stats defines a standard interface for etcd cluster statistics.
+package v2stats
+
+import "github.com/coreos/pkg/capnslog"
+
+var plog = capnslog.NewPackageLogger("go.etcd.io/etcd", "etcdserver/stats")
+
+type Stats interface {
+	// SelfStats returns the struct representing statistics of this server
+	SelfStats() []byte
+	// LeaderStats returns the statistics of all followers in the cluster
+	// if this server is leader. Otherwise, nil is returned.
+	LeaderStats() []byte
+	// StoreStats returns statistics of the store backing this EtcdServer
+	StoreStats() []byte
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2store/doc.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/doc.go
new file mode 100644
index 0000000..1933e4c
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/doc.go
@@ -0,0 +1,17 @@
+// Copyright 2015 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 v2store defines etcd's in-memory key/value store in v2 API.
+// To be deprecated in favor of v3 storage.
+package v2store
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2store/event.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/event.go
new file mode 100644
index 0000000..33e9017
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/event.go
@@ -0,0 +1,71 @@
+// Copyright 2015 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 v2store
+
+const (
+	Get              = "get"
+	Create           = "create"
+	Set              = "set"
+	Update           = "update"
+	Delete           = "delete"
+	CompareAndSwap   = "compareAndSwap"
+	CompareAndDelete = "compareAndDelete"
+	Expire           = "expire"
+)
+
+type Event struct {
+	Action    string      `json:"action"`
+	Node      *NodeExtern `json:"node,omitempty"`
+	PrevNode  *NodeExtern `json:"prevNode,omitempty"`
+	EtcdIndex uint64      `json:"-"`
+	Refresh   bool        `json:"refresh,omitempty"`
+}
+
+func newEvent(action string, key string, modifiedIndex, createdIndex uint64) *Event {
+	n := &NodeExtern{
+		Key:           key,
+		ModifiedIndex: modifiedIndex,
+		CreatedIndex:  createdIndex,
+	}
+
+	return &Event{
+		Action: action,
+		Node:   n,
+	}
+}
+
+func (e *Event) IsCreated() bool {
+	if e.Action == Create {
+		return true
+	}
+	return e.Action == Set && e.PrevNode == nil
+}
+
+func (e *Event) Index() uint64 {
+	return e.Node.ModifiedIndex
+}
+
+func (e *Event) Clone() *Event {
+	return &Event{
+		Action:    e.Action,
+		EtcdIndex: e.EtcdIndex,
+		Node:      e.Node.Clone(),
+		PrevNode:  e.PrevNode.Clone(),
+	}
+}
+
+func (e *Event) SetRefresh() {
+	e.Refresh = true
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2store/event_history.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/event_history.go
new file mode 100644
index 0000000..e4a969f
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/event_history.go
@@ -0,0 +1,129 @@
+// Copyright 2015 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 v2store
+
+import (
+	"fmt"
+	"path"
+	"strings"
+	"sync"
+
+	"go.etcd.io/etcd/etcdserver/api/v2error"
+)
+
+type EventHistory struct {
+	Queue      eventQueue
+	StartIndex uint64
+	LastIndex  uint64
+	rwl        sync.RWMutex
+}
+
+func newEventHistory(capacity int) *EventHistory {
+	return &EventHistory{
+		Queue: eventQueue{
+			Capacity: capacity,
+			Events:   make([]*Event, capacity),
+		},
+	}
+}
+
+// addEvent function adds event into the eventHistory
+func (eh *EventHistory) addEvent(e *Event) *Event {
+	eh.rwl.Lock()
+	defer eh.rwl.Unlock()
+
+	eh.Queue.insert(e)
+
+	eh.LastIndex = e.Index()
+
+	eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index()
+
+	return e
+}
+
+// scan enumerates events from the index history and stops at the first point
+// where the key matches.
+func (eh *EventHistory) scan(key string, recursive bool, index uint64) (*Event, *v2error.Error) {
+	eh.rwl.RLock()
+	defer eh.rwl.RUnlock()
+
+	// index should be after the event history's StartIndex
+	if index < eh.StartIndex {
+		return nil,
+			v2error.NewError(v2error.EcodeEventIndexCleared,
+				fmt.Sprintf("the requested history has been cleared [%v/%v]",
+					eh.StartIndex, index), 0)
+	}
+
+	// the index should come before the size of the queue minus the duplicate count
+	if index > eh.LastIndex { // future index
+		return nil, nil
+	}
+
+	offset := index - eh.StartIndex
+	i := (eh.Queue.Front + int(offset)) % eh.Queue.Capacity
+
+	for {
+		e := eh.Queue.Events[i]
+
+		if !e.Refresh {
+			ok := e.Node.Key == key
+
+			if recursive {
+				// add tailing slash
+				nkey := path.Clean(key)
+				if nkey[len(nkey)-1] != '/' {
+					nkey = nkey + "/"
+				}
+
+				ok = ok || strings.HasPrefix(e.Node.Key, nkey)
+			}
+
+			if (e.Action == Delete || e.Action == Expire) && e.PrevNode != nil && e.PrevNode.Dir {
+				ok = ok || strings.HasPrefix(key, e.PrevNode.Key)
+			}
+
+			if ok {
+				return e, nil
+			}
+		}
+
+		i = (i + 1) % eh.Queue.Capacity
+
+		if i == eh.Queue.Back {
+			return nil, nil
+		}
+	}
+}
+
+// clone will be protected by a stop-world lock
+// do not need to obtain internal lock
+func (eh *EventHistory) clone() *EventHistory {
+	clonedQueue := eventQueue{
+		Capacity: eh.Queue.Capacity,
+		Events:   make([]*Event, eh.Queue.Capacity),
+		Size:     eh.Queue.Size,
+		Front:    eh.Queue.Front,
+		Back:     eh.Queue.Back,
+	}
+
+	copy(clonedQueue.Events, eh.Queue.Events)
+	return &EventHistory{
+		StartIndex: eh.StartIndex,
+		Queue:      clonedQueue,
+		LastIndex:  eh.LastIndex,
+	}
+
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2store/event_queue.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/event_queue.go
new file mode 100644
index 0000000..7ea03de
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/event_queue.go
@@ -0,0 +1,34 @@
+// Copyright 2015 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 v2store
+
+type eventQueue struct {
+	Events   []*Event
+	Size     int
+	Front    int
+	Back     int
+	Capacity int
+}
+
+func (eq *eventQueue) insert(e *Event) {
+	eq.Events[eq.Back] = e
+	eq.Back = (eq.Back + 1) % eq.Capacity
+
+	if eq.Size == eq.Capacity { //dequeue
+		eq.Front = (eq.Front + 1) % eq.Capacity
+	} else {
+		eq.Size++
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2store/metrics.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/metrics.go
new file mode 100644
index 0000000..5adea1e
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/metrics.go
@@ -0,0 +1,130 @@
+// Copyright 2015 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 v2store
+
+import "github.com/prometheus/client_golang/prometheus"
+
+// Set of raw Prometheus metrics.
+// Labels
+// * action = declared in event.go
+// * outcome = Outcome
+// Do not increment directly, use Report* methods.
+var (
+	readCounter = prometheus.NewCounterVec(
+		prometheus.CounterOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "store",
+			Name:      "reads_total",
+			Help:      "Total number of reads action by (get/getRecursive), local to this member.",
+		}, []string{"action"})
+
+	writeCounter = prometheus.NewCounterVec(
+		prometheus.CounterOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "store",
+			Name:      "writes_total",
+			Help:      "Total number of writes (e.g. set/compareAndDelete) seen by this member.",
+		}, []string{"action"})
+
+	readFailedCounter = prometheus.NewCounterVec(
+		prometheus.CounterOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "store",
+			Name:      "reads_failed_total",
+			Help:      "Failed read actions by (get/getRecursive), local to this member.",
+		}, []string{"action"})
+
+	writeFailedCounter = prometheus.NewCounterVec(
+		prometheus.CounterOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "store",
+			Name:      "writes_failed_total",
+			Help:      "Failed write actions (e.g. set/compareAndDelete), seen by this member.",
+		}, []string{"action"})
+
+	expireCounter = prometheus.NewCounter(
+		prometheus.CounterOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "store",
+			Name:      "expires_total",
+			Help:      "Total number of expired keys.",
+		})
+
+	watchRequests = prometheus.NewCounter(
+		prometheus.CounterOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "store",
+			Name:      "watch_requests_total",
+			Help:      "Total number of incoming watch requests (new or reestablished).",
+		})
+
+	watcherCount = prometheus.NewGauge(
+		prometheus.GaugeOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "store",
+			Name:      "watchers",
+			Help:      "Count of currently active watchers.",
+		})
+)
+
+const (
+	GetRecursive = "getRecursive"
+)
+
+func init() {
+	if prometheus.Register(readCounter) != nil {
+		// Tests will try to double register since the tests use both
+		// store and store_test packages; ignore second attempts.
+		return
+	}
+	prometheus.MustRegister(writeCounter)
+	prometheus.MustRegister(expireCounter)
+	prometheus.MustRegister(watchRequests)
+	prometheus.MustRegister(watcherCount)
+}
+
+func reportReadSuccess(readAction string) {
+	readCounter.WithLabelValues(readAction).Inc()
+}
+
+func reportReadFailure(readAction string) {
+	readCounter.WithLabelValues(readAction).Inc()
+	readFailedCounter.WithLabelValues(readAction).Inc()
+}
+
+func reportWriteSuccess(writeAction string) {
+	writeCounter.WithLabelValues(writeAction).Inc()
+}
+
+func reportWriteFailure(writeAction string) {
+	writeCounter.WithLabelValues(writeAction).Inc()
+	writeFailedCounter.WithLabelValues(writeAction).Inc()
+}
+
+func reportExpiredKey() {
+	expireCounter.Inc()
+}
+
+func reportWatchRequest() {
+	watchRequests.Inc()
+}
+
+func reportWatcherAdded() {
+	watcherCount.Inc()
+}
+
+func reportWatcherRemoved() {
+	watcherCount.Dec()
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2store/node.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/node.go
new file mode 100644
index 0000000..38a6984
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/node.go
@@ -0,0 +1,396 @@
+// Copyright 2015 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 v2store
+
+import (
+	"path"
+	"sort"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver/api/v2error"
+
+	"github.com/jonboulle/clockwork"
+)
+
+// explanations of Compare function result
+const (
+	CompareMatch = iota
+	CompareIndexNotMatch
+	CompareValueNotMatch
+	CompareNotMatch
+)
+
+var Permanent time.Time
+
+// node is the basic element in the store system.
+// A key-value pair will have a string value
+// A directory will have a children map
+type node struct {
+	Path string
+
+	CreatedIndex  uint64
+	ModifiedIndex uint64
+
+	Parent *node `json:"-"` // should not encode this field! avoid circular dependency.
+
+	ExpireTime time.Time
+	Value      string           // for key-value pair
+	Children   map[string]*node // for directory
+
+	// A reference to the store this node is attached to.
+	store *store
+}
+
+// newKV creates a Key-Value pair
+func newKV(store *store, nodePath string, value string, createdIndex uint64, parent *node, expireTime time.Time) *node {
+	return &node{
+		Path:          nodePath,
+		CreatedIndex:  createdIndex,
+		ModifiedIndex: createdIndex,
+		Parent:        parent,
+		store:         store,
+		ExpireTime:    expireTime,
+		Value:         value,
+	}
+}
+
+// newDir creates a directory
+func newDir(store *store, nodePath string, createdIndex uint64, parent *node, expireTime time.Time) *node {
+	return &node{
+		Path:          nodePath,
+		CreatedIndex:  createdIndex,
+		ModifiedIndex: createdIndex,
+		Parent:        parent,
+		ExpireTime:    expireTime,
+		Children:      make(map[string]*node),
+		store:         store,
+	}
+}
+
+// IsHidden function checks if the node is a hidden node. A hidden node
+// will begin with '_'
+// A hidden node will not be shown via get command under a directory
+// For example if we have /foo/_hidden and /foo/notHidden, get "/foo"
+// will only return /foo/notHidden
+func (n *node) IsHidden() bool {
+	_, name := path.Split(n.Path)
+
+	return name[0] == '_'
+}
+
+// IsPermanent function checks if the node is a permanent one.
+func (n *node) IsPermanent() bool {
+	// we use a uninitialized time.Time to indicate the node is a
+	// permanent one.
+	// the uninitialized time.Time should equal zero.
+	return n.ExpireTime.IsZero()
+}
+
+// IsDir function checks whether the node is a directory.
+// If the node is a directory, the function will return true.
+// Otherwise the function will return false.
+func (n *node) IsDir() bool {
+	return n.Children != nil
+}
+
+// Read function gets the value of the node.
+// If the receiver node is not a key-value pair, a "Not A File" error will be returned.
+func (n *node) Read() (string, *v2error.Error) {
+	if n.IsDir() {
+		return "", v2error.NewError(v2error.EcodeNotFile, "", n.store.CurrentIndex)
+	}
+
+	return n.Value, nil
+}
+
+// Write function set the value of the node to the given value.
+// If the receiver node is a directory, a "Not A File" error will be returned.
+func (n *node) Write(value string, index uint64) *v2error.Error {
+	if n.IsDir() {
+		return v2error.NewError(v2error.EcodeNotFile, "", n.store.CurrentIndex)
+	}
+
+	n.Value = value
+	n.ModifiedIndex = index
+
+	return nil
+}
+
+func (n *node) expirationAndTTL(clock clockwork.Clock) (*time.Time, int64) {
+	if !n.IsPermanent() {
+		/* compute ttl as:
+		   ceiling( (expireTime - timeNow) / nanosecondsPerSecond )
+		   which ranges from 1..n
+		   rather than as:
+		   ( (expireTime - timeNow) / nanosecondsPerSecond ) + 1
+		   which ranges 1..n+1
+		*/
+		ttlN := n.ExpireTime.Sub(clock.Now())
+		ttl := ttlN / time.Second
+		if (ttlN % time.Second) > 0 {
+			ttl++
+		}
+		t := n.ExpireTime.UTC()
+		return &t, int64(ttl)
+	}
+	return nil, 0
+}
+
+// List function return a slice of nodes under the receiver node.
+// If the receiver node is not a directory, a "Not A Directory" error will be returned.
+func (n *node) List() ([]*node, *v2error.Error) {
+	if !n.IsDir() {
+		return nil, v2error.NewError(v2error.EcodeNotDir, "", n.store.CurrentIndex)
+	}
+
+	nodes := make([]*node, len(n.Children))
+
+	i := 0
+	for _, node := range n.Children {
+		nodes[i] = node
+		i++
+	}
+
+	return nodes, nil
+}
+
+// GetChild function returns the child node under the directory node.
+// On success, it returns the file node
+func (n *node) GetChild(name string) (*node, *v2error.Error) {
+	if !n.IsDir() {
+		return nil, v2error.NewError(v2error.EcodeNotDir, n.Path, n.store.CurrentIndex)
+	}
+
+	child, ok := n.Children[name]
+
+	if ok {
+		return child, nil
+	}
+
+	return nil, nil
+}
+
+// Add function adds a node to the receiver node.
+// If the receiver is not a directory, a "Not A Directory" error will be returned.
+// If there is an existing node with the same name under the directory, a "Already Exist"
+// error will be returned
+func (n *node) Add(child *node) *v2error.Error {
+	if !n.IsDir() {
+		return v2error.NewError(v2error.EcodeNotDir, "", n.store.CurrentIndex)
+	}
+
+	_, name := path.Split(child.Path)
+
+	if _, ok := n.Children[name]; ok {
+		return v2error.NewError(v2error.EcodeNodeExist, "", n.store.CurrentIndex)
+	}
+
+	n.Children[name] = child
+
+	return nil
+}
+
+// Remove function remove the node.
+func (n *node) Remove(dir, recursive bool, callback func(path string)) *v2error.Error {
+	if !n.IsDir() { // key-value pair
+		_, name := path.Split(n.Path)
+
+		// find its parent and remove the node from the map
+		if n.Parent != nil && n.Parent.Children[name] == n {
+			delete(n.Parent.Children, name)
+		}
+
+		if callback != nil {
+			callback(n.Path)
+		}
+
+		if !n.IsPermanent() {
+			n.store.ttlKeyHeap.remove(n)
+		}
+
+		return nil
+	}
+
+	if !dir {
+		// cannot delete a directory without dir set to true
+		return v2error.NewError(v2error.EcodeNotFile, n.Path, n.store.CurrentIndex)
+	}
+
+	if len(n.Children) != 0 && !recursive {
+		// cannot delete a directory if it is not empty and the operation
+		// is not recursive
+		return v2error.NewError(v2error.EcodeDirNotEmpty, n.Path, n.store.CurrentIndex)
+	}
+
+	for _, child := range n.Children { // delete all children
+		child.Remove(true, true, callback)
+	}
+
+	// delete self
+	_, name := path.Split(n.Path)
+	if n.Parent != nil && n.Parent.Children[name] == n {
+		delete(n.Parent.Children, name)
+
+		if callback != nil {
+			callback(n.Path)
+		}
+
+		if !n.IsPermanent() {
+			n.store.ttlKeyHeap.remove(n)
+		}
+	}
+
+	return nil
+}
+
+func (n *node) Repr(recursive, sorted bool, clock clockwork.Clock) *NodeExtern {
+	if n.IsDir() {
+		node := &NodeExtern{
+			Key:           n.Path,
+			Dir:           true,
+			ModifiedIndex: n.ModifiedIndex,
+			CreatedIndex:  n.CreatedIndex,
+		}
+		node.Expiration, node.TTL = n.expirationAndTTL(clock)
+
+		if !recursive {
+			return node
+		}
+
+		children, _ := n.List()
+		node.Nodes = make(NodeExterns, len(children))
+
+		// we do not use the index in the children slice directly
+		// we need to skip the hidden one
+		i := 0
+
+		for _, child := range children {
+
+			if child.IsHidden() { // get will not list hidden node
+				continue
+			}
+
+			node.Nodes[i] = child.Repr(recursive, sorted, clock)
+
+			i++
+		}
+
+		// eliminate hidden nodes
+		node.Nodes = node.Nodes[:i]
+		if sorted {
+			sort.Sort(node.Nodes)
+		}
+
+		return node
+	}
+
+	// since n.Value could be changed later, so we need to copy the value out
+	value := n.Value
+	node := &NodeExtern{
+		Key:           n.Path,
+		Value:         &value,
+		ModifiedIndex: n.ModifiedIndex,
+		CreatedIndex:  n.CreatedIndex,
+	}
+	node.Expiration, node.TTL = n.expirationAndTTL(clock)
+	return node
+}
+
+func (n *node) UpdateTTL(expireTime time.Time) {
+	if !n.IsPermanent() {
+		if expireTime.IsZero() {
+			// from ttl to permanent
+			n.ExpireTime = expireTime
+			// remove from ttl heap
+			n.store.ttlKeyHeap.remove(n)
+			return
+		}
+
+		// update ttl
+		n.ExpireTime = expireTime
+		// update ttl heap
+		n.store.ttlKeyHeap.update(n)
+		return
+	}
+
+	if expireTime.IsZero() {
+		return
+	}
+
+	// from permanent to ttl
+	n.ExpireTime = expireTime
+	// push into ttl heap
+	n.store.ttlKeyHeap.push(n)
+}
+
+// Compare function compares node index and value with provided ones.
+// second result value explains result and equals to one of Compare.. constants
+func (n *node) Compare(prevValue string, prevIndex uint64) (ok bool, which int) {
+	indexMatch := prevIndex == 0 || n.ModifiedIndex == prevIndex
+	valueMatch := prevValue == "" || n.Value == prevValue
+	ok = valueMatch && indexMatch
+	switch {
+	case valueMatch && indexMatch:
+		which = CompareMatch
+	case indexMatch && !valueMatch:
+		which = CompareValueNotMatch
+	case valueMatch && !indexMatch:
+		which = CompareIndexNotMatch
+	default:
+		which = CompareNotMatch
+	}
+	return ok, which
+}
+
+// Clone function clone the node recursively and return the new node.
+// If the node is a directory, it will clone all the content under this directory.
+// If the node is a key-value pair, it will clone the pair.
+func (n *node) Clone() *node {
+	if !n.IsDir() {
+		newkv := newKV(n.store, n.Path, n.Value, n.CreatedIndex, n.Parent, n.ExpireTime)
+		newkv.ModifiedIndex = n.ModifiedIndex
+		return newkv
+	}
+
+	clone := newDir(n.store, n.Path, n.CreatedIndex, n.Parent, n.ExpireTime)
+	clone.ModifiedIndex = n.ModifiedIndex
+
+	for key, child := range n.Children {
+		clone.Children[key] = child.Clone()
+	}
+
+	return clone
+}
+
+// recoverAndclean function help to do recovery.
+// Two things need to be done: 1. recovery structure; 2. delete expired nodes
+//
+// If the node is a directory, it will help recover children's parent pointer and recursively
+// call this function on its children.
+// We check the expire last since we need to recover the whole structure first and add all the
+// notifications into the event history.
+func (n *node) recoverAndclean() {
+	if n.IsDir() {
+		for _, child := range n.Children {
+			child.Parent = n
+			child.store = n.store
+			child.recoverAndclean()
+		}
+	}
+
+	if !n.ExpireTime.IsZero() {
+		n.store.ttlKeyHeap.push(n)
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2store/node_extern.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/node_extern.go
new file mode 100644
index 0000000..b3bf5f3
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/node_extern.go
@@ -0,0 +1,116 @@
+// Copyright 2015 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 v2store
+
+import (
+	"sort"
+	"time"
+
+	"github.com/jonboulle/clockwork"
+)
+
+// NodeExtern is the external representation of the
+// internal node with additional fields
+// PrevValue is the previous value of the node
+// TTL is time to live in second
+type NodeExtern struct {
+	Key           string      `json:"key,omitempty"`
+	Value         *string     `json:"value,omitempty"`
+	Dir           bool        `json:"dir,omitempty"`
+	Expiration    *time.Time  `json:"expiration,omitempty"`
+	TTL           int64       `json:"ttl,omitempty"`
+	Nodes         NodeExterns `json:"nodes,omitempty"`
+	ModifiedIndex uint64      `json:"modifiedIndex,omitempty"`
+	CreatedIndex  uint64      `json:"createdIndex,omitempty"`
+}
+
+func (eNode *NodeExtern) loadInternalNode(n *node, recursive, sorted bool, clock clockwork.Clock) {
+	if n.IsDir() { // node is a directory
+		eNode.Dir = true
+
+		children, _ := n.List()
+		eNode.Nodes = make(NodeExterns, len(children))
+
+		// we do not use the index in the children slice directly
+		// we need to skip the hidden one
+		i := 0
+
+		for _, child := range children {
+			if child.IsHidden() { // get will not return hidden nodes
+				continue
+			}
+
+			eNode.Nodes[i] = child.Repr(recursive, sorted, clock)
+			i++
+		}
+
+		// eliminate hidden nodes
+		eNode.Nodes = eNode.Nodes[:i]
+
+		if sorted {
+			sort.Sort(eNode.Nodes)
+		}
+
+	} else { // node is a file
+		value, _ := n.Read()
+		eNode.Value = &value
+	}
+
+	eNode.Expiration, eNode.TTL = n.expirationAndTTL(clock)
+}
+
+func (eNode *NodeExtern) Clone() *NodeExtern {
+	if eNode == nil {
+		return nil
+	}
+	nn := &NodeExtern{
+		Key:           eNode.Key,
+		Dir:           eNode.Dir,
+		TTL:           eNode.TTL,
+		ModifiedIndex: eNode.ModifiedIndex,
+		CreatedIndex:  eNode.CreatedIndex,
+	}
+	if eNode.Value != nil {
+		s := *eNode.Value
+		nn.Value = &s
+	}
+	if eNode.Expiration != nil {
+		t := *eNode.Expiration
+		nn.Expiration = &t
+	}
+	if eNode.Nodes != nil {
+		nn.Nodes = make(NodeExterns, len(eNode.Nodes))
+		for i, n := range eNode.Nodes {
+			nn.Nodes[i] = n.Clone()
+		}
+	}
+	return nn
+}
+
+type NodeExterns []*NodeExtern
+
+// interfaces for sorting
+
+func (ns NodeExterns) Len() int {
+	return len(ns)
+}
+
+func (ns NodeExterns) Less(i, j int) bool {
+	return ns[i].Key < ns[j].Key
+}
+
+func (ns NodeExterns) Swap(i, j int) {
+	ns[i], ns[j] = ns[j], ns[i]
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2store/stats.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/stats.go
new file mode 100644
index 0000000..45bc97f
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/stats.go
@@ -0,0 +1,145 @@
+// Copyright 2015 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 v2store
+
+import (
+	"encoding/json"
+	"sync/atomic"
+)
+
+const (
+	SetSuccess = iota
+	SetFail
+	DeleteSuccess
+	DeleteFail
+	CreateSuccess
+	CreateFail
+	UpdateSuccess
+	UpdateFail
+	CompareAndSwapSuccess
+	CompareAndSwapFail
+	GetSuccess
+	GetFail
+	ExpireCount
+	CompareAndDeleteSuccess
+	CompareAndDeleteFail
+)
+
+type Stats struct {
+	// Number of get requests
+
+	GetSuccess uint64 `json:"getsSuccess"`
+	GetFail    uint64 `json:"getsFail"`
+
+	// Number of sets requests
+
+	SetSuccess uint64 `json:"setsSuccess"`
+	SetFail    uint64 `json:"setsFail"`
+
+	// Number of delete requests
+
+	DeleteSuccess uint64 `json:"deleteSuccess"`
+	DeleteFail    uint64 `json:"deleteFail"`
+
+	// Number of update requests
+
+	UpdateSuccess uint64 `json:"updateSuccess"`
+	UpdateFail    uint64 `json:"updateFail"`
+
+	// Number of create requests
+
+	CreateSuccess uint64 `json:"createSuccess"`
+	CreateFail    uint64 `json:"createFail"`
+
+	// Number of testAndSet requests
+
+	CompareAndSwapSuccess uint64 `json:"compareAndSwapSuccess"`
+	CompareAndSwapFail    uint64 `json:"compareAndSwapFail"`
+
+	// Number of compareAndDelete requests
+
+	CompareAndDeleteSuccess uint64 `json:"compareAndDeleteSuccess"`
+	CompareAndDeleteFail    uint64 `json:"compareAndDeleteFail"`
+
+	ExpireCount uint64 `json:"expireCount"`
+
+	Watchers uint64 `json:"watchers"`
+}
+
+func newStats() *Stats {
+	s := new(Stats)
+	return s
+}
+
+func (s *Stats) clone() *Stats {
+	return &Stats{
+		GetSuccess:              s.GetSuccess,
+		GetFail:                 s.GetFail,
+		SetSuccess:              s.SetSuccess,
+		SetFail:                 s.SetFail,
+		DeleteSuccess:           s.DeleteSuccess,
+		DeleteFail:              s.DeleteFail,
+		UpdateSuccess:           s.UpdateSuccess,
+		UpdateFail:              s.UpdateFail,
+		CreateSuccess:           s.CreateSuccess,
+		CreateFail:              s.CreateFail,
+		CompareAndSwapSuccess:   s.CompareAndSwapSuccess,
+		CompareAndSwapFail:      s.CompareAndSwapFail,
+		CompareAndDeleteSuccess: s.CompareAndDeleteSuccess,
+		CompareAndDeleteFail:    s.CompareAndDeleteFail,
+		ExpireCount:             s.ExpireCount,
+		Watchers:                s.Watchers,
+	}
+}
+
+func (s *Stats) toJson() []byte {
+	b, _ := json.Marshal(s)
+	return b
+}
+
+func (s *Stats) Inc(field int) {
+	switch field {
+	case SetSuccess:
+		atomic.AddUint64(&s.SetSuccess, 1)
+	case SetFail:
+		atomic.AddUint64(&s.SetFail, 1)
+	case CreateSuccess:
+		atomic.AddUint64(&s.CreateSuccess, 1)
+	case CreateFail:
+		atomic.AddUint64(&s.CreateFail, 1)
+	case DeleteSuccess:
+		atomic.AddUint64(&s.DeleteSuccess, 1)
+	case DeleteFail:
+		atomic.AddUint64(&s.DeleteFail, 1)
+	case GetSuccess:
+		atomic.AddUint64(&s.GetSuccess, 1)
+	case GetFail:
+		atomic.AddUint64(&s.GetFail, 1)
+	case UpdateSuccess:
+		atomic.AddUint64(&s.UpdateSuccess, 1)
+	case UpdateFail:
+		atomic.AddUint64(&s.UpdateFail, 1)
+	case CompareAndSwapSuccess:
+		atomic.AddUint64(&s.CompareAndSwapSuccess, 1)
+	case CompareAndSwapFail:
+		atomic.AddUint64(&s.CompareAndSwapFail, 1)
+	case CompareAndDeleteSuccess:
+		atomic.AddUint64(&s.CompareAndDeleteSuccess, 1)
+	case CompareAndDeleteFail:
+		atomic.AddUint64(&s.CompareAndDeleteFail, 1)
+	case ExpireCount:
+		atomic.AddUint64(&s.ExpireCount, 1)
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2store/store.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/store.go
new file mode 100644
index 0000000..ce94043
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/store.go
@@ -0,0 +1,791 @@
+// Copyright 2015 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 v2store
+
+import (
+	"encoding/json"
+	"fmt"
+	"path"
+	"strconv"
+	"strings"
+	"sync"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver/api/v2error"
+	"go.etcd.io/etcd/pkg/types"
+
+	"github.com/jonboulle/clockwork"
+)
+
+// The default version to set when the store is first initialized.
+const defaultVersion = 2
+
+var minExpireTime time.Time
+
+func init() {
+	minExpireTime, _ = time.Parse(time.RFC3339, "2000-01-01T00:00:00Z")
+}
+
+type Store interface {
+	Version() int
+	Index() uint64
+
+	Get(nodePath string, recursive, sorted bool) (*Event, error)
+	Set(nodePath string, dir bool, value string, expireOpts TTLOptionSet) (*Event, error)
+	Update(nodePath string, newValue string, expireOpts TTLOptionSet) (*Event, error)
+	Create(nodePath string, dir bool, value string, unique bool,
+		expireOpts TTLOptionSet) (*Event, error)
+	CompareAndSwap(nodePath string, prevValue string, prevIndex uint64,
+		value string, expireOpts TTLOptionSet) (*Event, error)
+	Delete(nodePath string, dir, recursive bool) (*Event, error)
+	CompareAndDelete(nodePath string, prevValue string, prevIndex uint64) (*Event, error)
+
+	Watch(prefix string, recursive, stream bool, sinceIndex uint64) (Watcher, error)
+
+	Save() ([]byte, error)
+	Recovery(state []byte) error
+
+	Clone() Store
+	SaveNoCopy() ([]byte, error)
+
+	JsonStats() []byte
+	DeleteExpiredKeys(cutoff time.Time)
+
+	HasTTLKeys() bool
+}
+
+type TTLOptionSet struct {
+	ExpireTime time.Time
+	Refresh    bool
+}
+
+type store struct {
+	Root           *node
+	WatcherHub     *watcherHub
+	CurrentIndex   uint64
+	Stats          *Stats
+	CurrentVersion int
+	ttlKeyHeap     *ttlKeyHeap  // need to recovery manually
+	worldLock      sync.RWMutex // stop the world lock
+	clock          clockwork.Clock
+	readonlySet    types.Set
+}
+
+// New creates a store where the given namespaces will be created as initial directories.
+func New(namespaces ...string) Store {
+	s := newStore(namespaces...)
+	s.clock = clockwork.NewRealClock()
+	return s
+}
+
+func newStore(namespaces ...string) *store {
+	s := new(store)
+	s.CurrentVersion = defaultVersion
+	s.Root = newDir(s, "/", s.CurrentIndex, nil, Permanent)
+	for _, namespace := range namespaces {
+		s.Root.Add(newDir(s, namespace, s.CurrentIndex, s.Root, Permanent))
+	}
+	s.Stats = newStats()
+	s.WatcherHub = newWatchHub(1000)
+	s.ttlKeyHeap = newTtlKeyHeap()
+	s.readonlySet = types.NewUnsafeSet(append(namespaces, "/")...)
+	return s
+}
+
+// Version retrieves current version of the store.
+func (s *store) Version() int {
+	return s.CurrentVersion
+}
+
+// Index retrieves the current index of the store.
+func (s *store) Index() uint64 {
+	s.worldLock.RLock()
+	defer s.worldLock.RUnlock()
+	return s.CurrentIndex
+}
+
+// Get returns a get event.
+// If recursive is true, it will return all the content under the node path.
+// If sorted is true, it will sort the content by keys.
+func (s *store) Get(nodePath string, recursive, sorted bool) (*Event, error) {
+	var err *v2error.Error
+
+	s.worldLock.RLock()
+	defer s.worldLock.RUnlock()
+
+	defer func() {
+		if err == nil {
+			s.Stats.Inc(GetSuccess)
+			if recursive {
+				reportReadSuccess(GetRecursive)
+			} else {
+				reportReadSuccess(Get)
+			}
+			return
+		}
+
+		s.Stats.Inc(GetFail)
+		if recursive {
+			reportReadFailure(GetRecursive)
+		} else {
+			reportReadFailure(Get)
+		}
+	}()
+
+	n, err := s.internalGet(nodePath)
+	if err != nil {
+		return nil, err
+	}
+
+	e := newEvent(Get, nodePath, n.ModifiedIndex, n.CreatedIndex)
+	e.EtcdIndex = s.CurrentIndex
+	e.Node.loadInternalNode(n, recursive, sorted, s.clock)
+
+	return e, nil
+}
+
+// Create creates the node at nodePath. Create will help to create intermediate directories with no ttl.
+// If the node has already existed, create will fail.
+// If any node on the path is a file, create will fail.
+func (s *store) Create(nodePath string, dir bool, value string, unique bool, expireOpts TTLOptionSet) (*Event, error) {
+	var err *v2error.Error
+
+	s.worldLock.Lock()
+	defer s.worldLock.Unlock()
+
+	defer func() {
+		if err == nil {
+			s.Stats.Inc(CreateSuccess)
+			reportWriteSuccess(Create)
+			return
+		}
+
+		s.Stats.Inc(CreateFail)
+		reportWriteFailure(Create)
+	}()
+
+	e, err := s.internalCreate(nodePath, dir, value, unique, false, expireOpts.ExpireTime, Create)
+	if err != nil {
+		return nil, err
+	}
+
+	e.EtcdIndex = s.CurrentIndex
+	s.WatcherHub.notify(e)
+
+	return e, nil
+}
+
+// Set creates or replace the node at nodePath.
+func (s *store) Set(nodePath string, dir bool, value string, expireOpts TTLOptionSet) (*Event, error) {
+	var err *v2error.Error
+
+	s.worldLock.Lock()
+	defer s.worldLock.Unlock()
+
+	defer func() {
+		if err == nil {
+			s.Stats.Inc(SetSuccess)
+			reportWriteSuccess(Set)
+			return
+		}
+
+		s.Stats.Inc(SetFail)
+		reportWriteFailure(Set)
+	}()
+
+	// Get prevNode value
+	n, getErr := s.internalGet(nodePath)
+	if getErr != nil && getErr.ErrorCode != v2error.EcodeKeyNotFound {
+		err = getErr
+		return nil, err
+	}
+
+	if expireOpts.Refresh {
+		if getErr != nil {
+			err = getErr
+			return nil, err
+		}
+		value = n.Value
+	}
+
+	// Set new value
+	e, err := s.internalCreate(nodePath, dir, value, false, true, expireOpts.ExpireTime, Set)
+	if err != nil {
+		return nil, err
+	}
+	e.EtcdIndex = s.CurrentIndex
+
+	// Put prevNode into event
+	if getErr == nil {
+		prev := newEvent(Get, nodePath, n.ModifiedIndex, n.CreatedIndex)
+		prev.Node.loadInternalNode(n, false, false, s.clock)
+		e.PrevNode = prev.Node
+	}
+
+	if !expireOpts.Refresh {
+		s.WatcherHub.notify(e)
+	} else {
+		e.SetRefresh()
+		s.WatcherHub.add(e)
+	}
+
+	return e, nil
+}
+
+// returns user-readable cause of failed comparison
+func getCompareFailCause(n *node, which int, prevValue string, prevIndex uint64) string {
+	switch which {
+	case CompareIndexNotMatch:
+		return fmt.Sprintf("[%v != %v]", prevIndex, n.ModifiedIndex)
+	case CompareValueNotMatch:
+		return fmt.Sprintf("[%v != %v]", prevValue, n.Value)
+	default:
+		return fmt.Sprintf("[%v != %v] [%v != %v]", prevValue, n.Value, prevIndex, n.ModifiedIndex)
+	}
+}
+
+func (s *store) CompareAndSwap(nodePath string, prevValue string, prevIndex uint64,
+	value string, expireOpts TTLOptionSet) (*Event, error) {
+
+	var err *v2error.Error
+
+	s.worldLock.Lock()
+	defer s.worldLock.Unlock()
+
+	defer func() {
+		if err == nil {
+			s.Stats.Inc(CompareAndSwapSuccess)
+			reportWriteSuccess(CompareAndSwap)
+			return
+		}
+
+		s.Stats.Inc(CompareAndSwapFail)
+		reportWriteFailure(CompareAndSwap)
+	}()
+
+	nodePath = path.Clean(path.Join("/", nodePath))
+	// we do not allow the user to change "/"
+	if s.readonlySet.Contains(nodePath) {
+		return nil, v2error.NewError(v2error.EcodeRootROnly, "/", s.CurrentIndex)
+	}
+
+	n, err := s.internalGet(nodePath)
+	if err != nil {
+		return nil, err
+	}
+	if n.IsDir() { // can only compare and swap file
+		err = v2error.NewError(v2error.EcodeNotFile, nodePath, s.CurrentIndex)
+		return nil, err
+	}
+
+	// If both of the prevValue and prevIndex are given, we will test both of them.
+	// Command will be executed, only if both of the tests are successful.
+	if ok, which := n.Compare(prevValue, prevIndex); !ok {
+		cause := getCompareFailCause(n, which, prevValue, prevIndex)
+		err = v2error.NewError(v2error.EcodeTestFailed, cause, s.CurrentIndex)
+		return nil, err
+	}
+
+	if expireOpts.Refresh {
+		value = n.Value
+	}
+
+	// update etcd index
+	s.CurrentIndex++
+
+	e := newEvent(CompareAndSwap, nodePath, s.CurrentIndex, n.CreatedIndex)
+	e.EtcdIndex = s.CurrentIndex
+	e.PrevNode = n.Repr(false, false, s.clock)
+	eNode := e.Node
+
+	// if test succeed, write the value
+	n.Write(value, s.CurrentIndex)
+	n.UpdateTTL(expireOpts.ExpireTime)
+
+	// copy the value for safety
+	valueCopy := value
+	eNode.Value = &valueCopy
+	eNode.Expiration, eNode.TTL = n.expirationAndTTL(s.clock)
+
+	if !expireOpts.Refresh {
+		s.WatcherHub.notify(e)
+	} else {
+		e.SetRefresh()
+		s.WatcherHub.add(e)
+	}
+
+	return e, nil
+}
+
+// Delete deletes the node at the given path.
+// If the node is a directory, recursive must be true to delete it.
+func (s *store) Delete(nodePath string, dir, recursive bool) (*Event, error) {
+	var err *v2error.Error
+
+	s.worldLock.Lock()
+	defer s.worldLock.Unlock()
+
+	defer func() {
+		if err == nil {
+			s.Stats.Inc(DeleteSuccess)
+			reportWriteSuccess(Delete)
+			return
+		}
+
+		s.Stats.Inc(DeleteFail)
+		reportWriteFailure(Delete)
+	}()
+
+	nodePath = path.Clean(path.Join("/", nodePath))
+	// we do not allow the user to change "/"
+	if s.readonlySet.Contains(nodePath) {
+		return nil, v2error.NewError(v2error.EcodeRootROnly, "/", s.CurrentIndex)
+	}
+
+	// recursive implies dir
+	if recursive {
+		dir = true
+	}
+
+	n, err := s.internalGet(nodePath)
+	if err != nil { // if the node does not exist, return error
+		return nil, err
+	}
+
+	nextIndex := s.CurrentIndex + 1
+	e := newEvent(Delete, nodePath, nextIndex, n.CreatedIndex)
+	e.EtcdIndex = nextIndex
+	e.PrevNode = n.Repr(false, false, s.clock)
+	eNode := e.Node
+
+	if n.IsDir() {
+		eNode.Dir = true
+	}
+
+	callback := func(path string) { // notify function
+		// notify the watchers with deleted set true
+		s.WatcherHub.notifyWatchers(e, path, true)
+	}
+
+	err = n.Remove(dir, recursive, callback)
+	if err != nil {
+		return nil, err
+	}
+
+	// update etcd index
+	s.CurrentIndex++
+
+	s.WatcherHub.notify(e)
+
+	return e, nil
+}
+
+func (s *store) CompareAndDelete(nodePath string, prevValue string, prevIndex uint64) (*Event, error) {
+	var err *v2error.Error
+
+	s.worldLock.Lock()
+	defer s.worldLock.Unlock()
+
+	defer func() {
+		if err == nil {
+			s.Stats.Inc(CompareAndDeleteSuccess)
+			reportWriteSuccess(CompareAndDelete)
+			return
+		}
+
+		s.Stats.Inc(CompareAndDeleteFail)
+		reportWriteFailure(CompareAndDelete)
+	}()
+
+	nodePath = path.Clean(path.Join("/", nodePath))
+
+	n, err := s.internalGet(nodePath)
+	if err != nil { // if the node does not exist, return error
+		return nil, err
+	}
+	if n.IsDir() { // can only compare and delete file
+		return nil, v2error.NewError(v2error.EcodeNotFile, nodePath, s.CurrentIndex)
+	}
+
+	// If both of the prevValue and prevIndex are given, we will test both of them.
+	// Command will be executed, only if both of the tests are successful.
+	if ok, which := n.Compare(prevValue, prevIndex); !ok {
+		cause := getCompareFailCause(n, which, prevValue, prevIndex)
+		return nil, v2error.NewError(v2error.EcodeTestFailed, cause, s.CurrentIndex)
+	}
+
+	// update etcd index
+	s.CurrentIndex++
+
+	e := newEvent(CompareAndDelete, nodePath, s.CurrentIndex, n.CreatedIndex)
+	e.EtcdIndex = s.CurrentIndex
+	e.PrevNode = n.Repr(false, false, s.clock)
+
+	callback := func(path string) { // notify function
+		// notify the watchers with deleted set true
+		s.WatcherHub.notifyWatchers(e, path, true)
+	}
+
+	err = n.Remove(false, false, callback)
+	if err != nil {
+		return nil, err
+	}
+
+	s.WatcherHub.notify(e)
+
+	return e, nil
+}
+
+func (s *store) Watch(key string, recursive, stream bool, sinceIndex uint64) (Watcher, error) {
+	s.worldLock.RLock()
+	defer s.worldLock.RUnlock()
+
+	key = path.Clean(path.Join("/", key))
+	if sinceIndex == 0 {
+		sinceIndex = s.CurrentIndex + 1
+	}
+	// WatcherHub does not know about the current index, so we need to pass it in
+	w, err := s.WatcherHub.watch(key, recursive, stream, sinceIndex, s.CurrentIndex)
+	if err != nil {
+		return nil, err
+	}
+
+	return w, nil
+}
+
+// walk walks all the nodePath and apply the walkFunc on each directory
+func (s *store) walk(nodePath string, walkFunc func(prev *node, component string) (*node, *v2error.Error)) (*node, *v2error.Error) {
+	components := strings.Split(nodePath, "/")
+
+	curr := s.Root
+	var err *v2error.Error
+
+	for i := 1; i < len(components); i++ {
+		if len(components[i]) == 0 { // ignore empty string
+			return curr, nil
+		}
+
+		curr, err = walkFunc(curr, components[i])
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	return curr, nil
+}
+
+// Update updates the value/ttl of the node.
+// If the node is a file, the value and the ttl can be updated.
+// If the node is a directory, only the ttl can be updated.
+func (s *store) Update(nodePath string, newValue string, expireOpts TTLOptionSet) (*Event, error) {
+	var err *v2error.Error
+
+	s.worldLock.Lock()
+	defer s.worldLock.Unlock()
+
+	defer func() {
+		if err == nil {
+			s.Stats.Inc(UpdateSuccess)
+			reportWriteSuccess(Update)
+			return
+		}
+
+		s.Stats.Inc(UpdateFail)
+		reportWriteFailure(Update)
+	}()
+
+	nodePath = path.Clean(path.Join("/", nodePath))
+	// we do not allow the user to change "/"
+	if s.readonlySet.Contains(nodePath) {
+		return nil, v2error.NewError(v2error.EcodeRootROnly, "/", s.CurrentIndex)
+	}
+
+	currIndex, nextIndex := s.CurrentIndex, s.CurrentIndex+1
+
+	n, err := s.internalGet(nodePath)
+	if err != nil { // if the node does not exist, return error
+		return nil, err
+	}
+	if n.IsDir() && len(newValue) != 0 {
+		// if the node is a directory, we cannot update value to non-empty
+		return nil, v2error.NewError(v2error.EcodeNotFile, nodePath, currIndex)
+	}
+
+	if expireOpts.Refresh {
+		newValue = n.Value
+	}
+
+	e := newEvent(Update, nodePath, nextIndex, n.CreatedIndex)
+	e.EtcdIndex = nextIndex
+	e.PrevNode = n.Repr(false, false, s.clock)
+	eNode := e.Node
+
+	n.Write(newValue, nextIndex)
+
+	if n.IsDir() {
+		eNode.Dir = true
+	} else {
+		// copy the value for safety
+		newValueCopy := newValue
+		eNode.Value = &newValueCopy
+	}
+
+	// update ttl
+	n.UpdateTTL(expireOpts.ExpireTime)
+
+	eNode.Expiration, eNode.TTL = n.expirationAndTTL(s.clock)
+
+	if !expireOpts.Refresh {
+		s.WatcherHub.notify(e)
+	} else {
+		e.SetRefresh()
+		s.WatcherHub.add(e)
+	}
+
+	s.CurrentIndex = nextIndex
+
+	return e, nil
+}
+
+func (s *store) internalCreate(nodePath string, dir bool, value string, unique, replace bool,
+	expireTime time.Time, action string) (*Event, *v2error.Error) {
+
+	currIndex, nextIndex := s.CurrentIndex, s.CurrentIndex+1
+
+	if unique { // append unique item under the node path
+		nodePath += "/" + fmt.Sprintf("%020s", strconv.FormatUint(nextIndex, 10))
+	}
+
+	nodePath = path.Clean(path.Join("/", nodePath))
+
+	// we do not allow the user to change "/"
+	if s.readonlySet.Contains(nodePath) {
+		return nil, v2error.NewError(v2error.EcodeRootROnly, "/", currIndex)
+	}
+
+	// Assume expire times that are way in the past are
+	// This can occur when the time is serialized to JS
+	if expireTime.Before(minExpireTime) {
+		expireTime = Permanent
+	}
+
+	dirName, nodeName := path.Split(nodePath)
+
+	// walk through the nodePath, create dirs and get the last directory node
+	d, err := s.walk(dirName, s.checkDir)
+
+	if err != nil {
+		s.Stats.Inc(SetFail)
+		reportWriteFailure(action)
+		err.Index = currIndex
+		return nil, err
+	}
+
+	e := newEvent(action, nodePath, nextIndex, nextIndex)
+	eNode := e.Node
+
+	n, _ := d.GetChild(nodeName)
+
+	// force will try to replace an existing file
+	if n != nil {
+		if replace {
+			if n.IsDir() {
+				return nil, v2error.NewError(v2error.EcodeNotFile, nodePath, currIndex)
+			}
+			e.PrevNode = n.Repr(false, false, s.clock)
+
+			n.Remove(false, false, nil)
+		} else {
+			return nil, v2error.NewError(v2error.EcodeNodeExist, nodePath, currIndex)
+		}
+	}
+
+	if !dir { // create file
+		// copy the value for safety
+		valueCopy := value
+		eNode.Value = &valueCopy
+
+		n = newKV(s, nodePath, value, nextIndex, d, expireTime)
+
+	} else { // create directory
+		eNode.Dir = true
+
+		n = newDir(s, nodePath, nextIndex, d, expireTime)
+	}
+
+	// we are sure d is a directory and does not have the children with name n.Name
+	d.Add(n)
+
+	// node with TTL
+	if !n.IsPermanent() {
+		s.ttlKeyHeap.push(n)
+
+		eNode.Expiration, eNode.TTL = n.expirationAndTTL(s.clock)
+	}
+
+	s.CurrentIndex = nextIndex
+
+	return e, nil
+}
+
+// InternalGet gets the node of the given nodePath.
+func (s *store) internalGet(nodePath string) (*node, *v2error.Error) {
+	nodePath = path.Clean(path.Join("/", nodePath))
+
+	walkFunc := func(parent *node, name string) (*node, *v2error.Error) {
+
+		if !parent.IsDir() {
+			err := v2error.NewError(v2error.EcodeNotDir, parent.Path, s.CurrentIndex)
+			return nil, err
+		}
+
+		child, ok := parent.Children[name]
+		if ok {
+			return child, nil
+		}
+
+		return nil, v2error.NewError(v2error.EcodeKeyNotFound, path.Join(parent.Path, name), s.CurrentIndex)
+	}
+
+	f, err := s.walk(nodePath, walkFunc)
+
+	if err != nil {
+		return nil, err
+	}
+	return f, nil
+}
+
+// DeleteExpiredKeys will delete all expired keys
+func (s *store) DeleteExpiredKeys(cutoff time.Time) {
+	s.worldLock.Lock()
+	defer s.worldLock.Unlock()
+
+	for {
+		node := s.ttlKeyHeap.top()
+		if node == nil || node.ExpireTime.After(cutoff) {
+			break
+		}
+
+		s.CurrentIndex++
+		e := newEvent(Expire, node.Path, s.CurrentIndex, node.CreatedIndex)
+		e.EtcdIndex = s.CurrentIndex
+		e.PrevNode = node.Repr(false, false, s.clock)
+		if node.IsDir() {
+			e.Node.Dir = true
+		}
+
+		callback := func(path string) { // notify function
+			// notify the watchers with deleted set true
+			s.WatcherHub.notifyWatchers(e, path, true)
+		}
+
+		s.ttlKeyHeap.pop()
+		node.Remove(true, true, callback)
+
+		reportExpiredKey()
+		s.Stats.Inc(ExpireCount)
+
+		s.WatcherHub.notify(e)
+	}
+
+}
+
+// checkDir will check whether the component is a directory under parent node.
+// If it is a directory, this function will return the pointer to that node.
+// If it does not exist, this function will create a new directory and return the pointer to that node.
+// If it is a file, this function will return error.
+func (s *store) checkDir(parent *node, dirName string) (*node, *v2error.Error) {
+	node, ok := parent.Children[dirName]
+
+	if ok {
+		if node.IsDir() {
+			return node, nil
+		}
+
+		return nil, v2error.NewError(v2error.EcodeNotDir, node.Path, s.CurrentIndex)
+	}
+
+	n := newDir(s, path.Join(parent.Path, dirName), s.CurrentIndex+1, parent, Permanent)
+
+	parent.Children[dirName] = n
+
+	return n, nil
+}
+
+// Save saves the static state of the store system.
+// It will not be able to save the state of watchers.
+// It will not save the parent field of the node. Or there will
+// be cyclic dependencies issue for the json package.
+func (s *store) Save() ([]byte, error) {
+	b, err := json.Marshal(s.Clone())
+	if err != nil {
+		return nil, err
+	}
+
+	return b, nil
+}
+
+func (s *store) SaveNoCopy() ([]byte, error) {
+	b, err := json.Marshal(s)
+	if err != nil {
+		return nil, err
+	}
+
+	return b, nil
+}
+
+func (s *store) Clone() Store {
+	s.worldLock.Lock()
+
+	clonedStore := newStore()
+	clonedStore.CurrentIndex = s.CurrentIndex
+	clonedStore.Root = s.Root.Clone()
+	clonedStore.WatcherHub = s.WatcherHub.clone()
+	clonedStore.Stats = s.Stats.clone()
+	clonedStore.CurrentVersion = s.CurrentVersion
+
+	s.worldLock.Unlock()
+	return clonedStore
+}
+
+// Recovery recovers the store system from a static state
+// It needs to recover the parent field of the nodes.
+// It needs to delete the expired nodes since the saved time and also
+// needs to create monitoring go routines.
+func (s *store) Recovery(state []byte) error {
+	s.worldLock.Lock()
+	defer s.worldLock.Unlock()
+	err := json.Unmarshal(state, s)
+
+	if err != nil {
+		return err
+	}
+
+	s.ttlKeyHeap = newTtlKeyHeap()
+
+	s.Root.recoverAndclean()
+	return nil
+}
+
+func (s *store) JsonStats() []byte {
+	s.Stats.Watchers = uint64(s.WatcherHub.count)
+	return s.Stats.toJson()
+}
+
+func (s *store) HasTTLKeys() bool {
+	s.worldLock.RLock()
+	defer s.worldLock.RUnlock()
+	return s.ttlKeyHeap.Len() != 0
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2store/ttl_key_heap.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/ttl_key_heap.go
new file mode 100644
index 0000000..477d2b9
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/ttl_key_heap.go
@@ -0,0 +1,97 @@
+// Copyright 2015 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 v2store
+
+import "container/heap"
+
+// An TTLKeyHeap is a min-heap of TTLKeys order by expiration time
+type ttlKeyHeap struct {
+	array  []*node
+	keyMap map[*node]int
+}
+
+func newTtlKeyHeap() *ttlKeyHeap {
+	h := &ttlKeyHeap{keyMap: make(map[*node]int)}
+	heap.Init(h)
+	return h
+}
+
+func (h ttlKeyHeap) Len() int {
+	return len(h.array)
+}
+
+func (h ttlKeyHeap) Less(i, j int) bool {
+	return h.array[i].ExpireTime.Before(h.array[j].ExpireTime)
+}
+
+func (h ttlKeyHeap) Swap(i, j int) {
+	// swap node
+	h.array[i], h.array[j] = h.array[j], h.array[i]
+
+	// update map
+	h.keyMap[h.array[i]] = i
+	h.keyMap[h.array[j]] = j
+}
+
+func (h *ttlKeyHeap) Push(x interface{}) {
+	n, _ := x.(*node)
+	h.keyMap[n] = len(h.array)
+	h.array = append(h.array, n)
+}
+
+func (h *ttlKeyHeap) Pop() interface{} {
+	old := h.array
+	n := len(old)
+	x := old[n-1]
+	// Set slice element to nil, so GC can recycle the node.
+	// This is due to golang GC doesn't support partial recycling:
+	// https://github.com/golang/go/issues/9618
+	old[n-1] = nil
+	h.array = old[0 : n-1]
+	delete(h.keyMap, x)
+	return x
+}
+
+func (h *ttlKeyHeap) top() *node {
+	if h.Len() != 0 {
+		return h.array[0]
+	}
+	return nil
+}
+
+func (h *ttlKeyHeap) pop() *node {
+	x := heap.Pop(h)
+	n, _ := x.(*node)
+	return n
+}
+
+func (h *ttlKeyHeap) push(x interface{}) {
+	heap.Push(h, x)
+}
+
+func (h *ttlKeyHeap) update(n *node) {
+	index, ok := h.keyMap[n]
+	if ok {
+		heap.Remove(h, index)
+		heap.Push(h, n)
+	}
+}
+
+func (h *ttlKeyHeap) remove(n *node) {
+	index, ok := h.keyMap[n]
+	if ok {
+		heap.Remove(h, index)
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2store/watcher.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/watcher.go
new file mode 100644
index 0000000..4b1e846
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/watcher.go
@@ -0,0 +1,95 @@
+// Copyright 2015 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 v2store
+
+type Watcher interface {
+	EventChan() chan *Event
+	StartIndex() uint64 // The EtcdIndex at which the Watcher was created
+	Remove()
+}
+
+type watcher struct {
+	eventChan  chan *Event
+	stream     bool
+	recursive  bool
+	sinceIndex uint64
+	startIndex uint64
+	hub        *watcherHub
+	removed    bool
+	remove     func()
+}
+
+func (w *watcher) EventChan() chan *Event {
+	return w.eventChan
+}
+
+func (w *watcher) StartIndex() uint64 {
+	return w.startIndex
+}
+
+// notify function notifies the watcher. If the watcher interests in the given path,
+// the function will return true.
+func (w *watcher) notify(e *Event, originalPath bool, deleted bool) bool {
+	// watcher is interested the path in three cases and under one condition
+	// the condition is that the event happens after the watcher's sinceIndex
+
+	// 1. the path at which the event happens is the path the watcher is watching at.
+	// For example if the watcher is watching at "/foo" and the event happens at "/foo",
+	// the watcher must be interested in that event.
+
+	// 2. the watcher is a recursive watcher, it interests in the event happens after
+	// its watching path. For example if watcher A watches at "/foo" and it is a recursive
+	// one, it will interest in the event happens at "/foo/bar".
+
+	// 3. when we delete a directory, we need to force notify all the watchers who watches
+	// at the file we need to delete.
+	// For example a watcher is watching at "/foo/bar". And we deletes "/foo". The watcher
+	// should get notified even if "/foo" is not the path it is watching.
+	if (w.recursive || originalPath || deleted) && e.Index() >= w.sinceIndex {
+		// We cannot block here if the eventChan capacity is full, otherwise
+		// etcd will hang. eventChan capacity is full when the rate of
+		// notifications are higher than our send rate.
+		// If this happens, we close the channel.
+		select {
+		case w.eventChan <- e:
+		default:
+			// We have missed a notification. Remove the watcher.
+			// Removing the watcher also closes the eventChan.
+			w.remove()
+		}
+		return true
+	}
+	return false
+}
+
+// Remove removes the watcher from watcherHub
+// The actual remove function is guaranteed to only be executed once
+func (w *watcher) Remove() {
+	w.hub.mutex.Lock()
+	defer w.hub.mutex.Unlock()
+
+	close(w.eventChan)
+	if w.remove != nil {
+		w.remove()
+	}
+}
+
+// nopWatcher is a watcher that receives nothing, always blocking.
+type nopWatcher struct{}
+
+func NewNopWatcher() Watcher                 { return &nopWatcher{} }
+func (w *nopWatcher) EventChan() chan *Event { return nil }
+func (w *nopWatcher) StartIndex() uint64     { return 0 }
+func (w *nopWatcher) Remove()                {}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2store/watcher_hub.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/watcher_hub.go
new file mode 100644
index 0000000..a452e7e
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2store/watcher_hub.go
@@ -0,0 +1,200 @@
+// Copyright 2015 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 v2store
+
+import (
+	"container/list"
+	"path"
+	"strings"
+	"sync"
+	"sync/atomic"
+
+	"go.etcd.io/etcd/etcdserver/api/v2error"
+)
+
+// A watcherHub contains all subscribed watchers
+// watchers is a map with watched path as key and watcher as value
+// EventHistory keeps the old events for watcherHub. It is used to help
+// watcher to get a continuous event history. Or a watcher might miss the
+// event happens between the end of the first watch command and the start
+// of the second command.
+type watcherHub struct {
+	// count must be the first element to keep 64-bit alignment for atomic
+	// access
+
+	count int64 // current number of watchers.
+
+	mutex        sync.Mutex
+	watchers     map[string]*list.List
+	EventHistory *EventHistory
+}
+
+// newWatchHub creates a watcherHub. The capacity determines how many events we will
+// keep in the eventHistory.
+// Typically, we only need to keep a small size of history[smaller than 20K].
+// Ideally, it should smaller than 20K/s[max throughput] * 2 * 50ms[RTT] = 2000
+func newWatchHub(capacity int) *watcherHub {
+	return &watcherHub{
+		watchers:     make(map[string]*list.List),
+		EventHistory: newEventHistory(capacity),
+	}
+}
+
+// Watch function returns a Watcher.
+// If recursive is true, the first change after index under key will be sent to the event channel of the watcher.
+// If recursive is false, the first change after index at key will be sent to the event channel of the watcher.
+// If index is zero, watch will start from the current index + 1.
+func (wh *watcherHub) watch(key string, recursive, stream bool, index, storeIndex uint64) (Watcher, *v2error.Error) {
+	reportWatchRequest()
+	event, err := wh.EventHistory.scan(key, recursive, index)
+
+	if err != nil {
+		err.Index = storeIndex
+		return nil, err
+	}
+
+	w := &watcher{
+		eventChan:  make(chan *Event, 100), // use a buffered channel
+		recursive:  recursive,
+		stream:     stream,
+		sinceIndex: index,
+		startIndex: storeIndex,
+		hub:        wh,
+	}
+
+	wh.mutex.Lock()
+	defer wh.mutex.Unlock()
+	// If the event exists in the known history, append the EtcdIndex and return immediately
+	if event != nil {
+		ne := event.Clone()
+		ne.EtcdIndex = storeIndex
+		w.eventChan <- ne
+		return w, nil
+	}
+
+	l, ok := wh.watchers[key]
+
+	var elem *list.Element
+
+	if ok { // add the new watcher to the back of the list
+		elem = l.PushBack(w)
+	} else { // create a new list and add the new watcher
+		l = list.New()
+		elem = l.PushBack(w)
+		wh.watchers[key] = l
+	}
+
+	w.remove = func() {
+		if w.removed { // avoid removing it twice
+			return
+		}
+		w.removed = true
+		l.Remove(elem)
+		atomic.AddInt64(&wh.count, -1)
+		reportWatcherRemoved()
+		if l.Len() == 0 {
+			delete(wh.watchers, key)
+		}
+	}
+
+	atomic.AddInt64(&wh.count, 1)
+	reportWatcherAdded()
+
+	return w, nil
+}
+
+func (wh *watcherHub) add(e *Event) {
+	wh.EventHistory.addEvent(e)
+}
+
+// notify function accepts an event and notify to the watchers.
+func (wh *watcherHub) notify(e *Event) {
+	e = wh.EventHistory.addEvent(e) // add event into the eventHistory
+
+	segments := strings.Split(e.Node.Key, "/")
+
+	currPath := "/"
+
+	// walk through all the segments of the path and notify the watchers
+	// if the path is "/foo/bar", it will notify watchers with path "/",
+	// "/foo" and "/foo/bar"
+
+	for _, segment := range segments {
+		currPath = path.Join(currPath, segment)
+		// notify the watchers who interests in the changes of current path
+		wh.notifyWatchers(e, currPath, false)
+	}
+}
+
+func (wh *watcherHub) notifyWatchers(e *Event, nodePath string, deleted bool) {
+	wh.mutex.Lock()
+	defer wh.mutex.Unlock()
+
+	l, ok := wh.watchers[nodePath]
+	if ok {
+		curr := l.Front()
+
+		for curr != nil {
+			next := curr.Next() // save reference to the next one in the list
+
+			w, _ := curr.Value.(*watcher)
+
+			originalPath := e.Node.Key == nodePath
+			if (originalPath || !isHidden(nodePath, e.Node.Key)) && w.notify(e, originalPath, deleted) {
+				if !w.stream { // do not remove the stream watcher
+					// if we successfully notify a watcher
+					// we need to remove the watcher from the list
+					// and decrease the counter
+					w.removed = true
+					l.Remove(curr)
+					atomic.AddInt64(&wh.count, -1)
+					reportWatcherRemoved()
+				}
+			}
+
+			curr = next // update current to the next element in the list
+		}
+
+		if l.Len() == 0 {
+			// if we have notified all watcher in the list
+			// we can delete the list
+			delete(wh.watchers, nodePath)
+		}
+	}
+}
+
+// clone function clones the watcherHub and return the cloned one.
+// only clone the static content. do not clone the current watchers.
+func (wh *watcherHub) clone() *watcherHub {
+	clonedHistory := wh.EventHistory.clone()
+
+	return &watcherHub{
+		EventHistory: clonedHistory,
+	}
+}
+
+// isHidden checks to see if key path is considered hidden to watch path i.e. the
+// last element is hidden or it's within a hidden directory
+func isHidden(watchPath, keyPath string) bool {
+	// When deleting a directory, watchPath might be deeper than the actual keyPath
+	// For example, when deleting /foo we also need to notify watchers on /foo/bar.
+	if len(watchPath) > len(keyPath) {
+		return false
+	}
+	// if watch path is just a "/", after path will start without "/"
+	// add a "/" to deal with the special case when watchPath is "/"
+	afterPath := path.Clean("/" + keyPath[len(watchPath):])
+	return strings.Contains(afterPath, "/_")
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2v3/cluster.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2v3/cluster.go
new file mode 100644
index 0000000..a22e4af
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2v3/cluster.go
@@ -0,0 +1,31 @@
+// 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 v2v3
+
+import (
+	"go.etcd.io/etcd/etcdserver/api/membership"
+	"go.etcd.io/etcd/pkg/types"
+
+	"github.com/coreos/go-semver/semver"
+)
+
+func (s *v2v3Server) ID() types.ID {
+	// TODO: use an actual member ID
+	return types.ID(0xe7cd2f00d)
+}
+func (s *v2v3Server) ClientURLs() []string                  { panic("STUB") }
+func (s *v2v3Server) Members() []*membership.Member         { panic("STUB") }
+func (s *v2v3Server) Member(id types.ID) *membership.Member { panic("STUB") }
+func (s *v2v3Server) Version() *semver.Version              { panic("STUB") }
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2v3/doc.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2v3/doc.go
new file mode 100644
index 0000000..2ff372f
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2v3/doc.go
@@ -0,0 +1,16 @@
+// 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 v2v3 provides a ServerV2 implementation backed by clientv3.Client.
+package v2v3
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2v3/server.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2v3/server.go
new file mode 100644
index 0000000..5ff9b96
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2v3/server.go
@@ -0,0 +1,129 @@
+// 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 v2v3
+
+import (
+	"context"
+	"net/http"
+	"time"
+
+	"go.etcd.io/etcd/clientv3"
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/api"
+	"go.etcd.io/etcd/etcdserver/api/membership"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/pkg/types"
+
+	"github.com/coreos/go-semver/semver"
+	"go.uber.org/zap"
+)
+
+type fakeStats struct{}
+
+func (s *fakeStats) SelfStats() []byte   { return nil }
+func (s *fakeStats) LeaderStats() []byte { return nil }
+func (s *fakeStats) StoreStats() []byte  { return nil }
+
+type v2v3Server struct {
+	lg    *zap.Logger
+	c     *clientv3.Client
+	store *v2v3Store
+	fakeStats
+}
+
+func NewServer(lg *zap.Logger, c *clientv3.Client, pfx string) etcdserver.ServerPeer {
+	return &v2v3Server{lg: lg, c: c, store: newStore(c, pfx)}
+}
+
+func (s *v2v3Server) ClientCertAuthEnabled() bool { return false }
+
+func (s *v2v3Server) LeaseHandler() http.Handler { panic("STUB: lease handler") }
+func (s *v2v3Server) RaftHandler() http.Handler  { panic("STUB: raft handler") }
+
+func (s *v2v3Server) Leader() types.ID {
+	ctx, cancel := context.WithTimeout(context.TODO(), 5*time.Second)
+	defer cancel()
+	resp, err := s.c.Status(ctx, s.c.Endpoints()[0])
+	if err != nil {
+		return 0
+	}
+	return types.ID(resp.Leader)
+}
+
+func (s *v2v3Server) AddMember(ctx context.Context, memb membership.Member) ([]*membership.Member, error) {
+	// adding member as learner is not supported by V2 Server.
+	resp, err := s.c.MemberAdd(ctx, memb.PeerURLs)
+	if err != nil {
+		return nil, err
+	}
+	return v3MembersToMembership(resp.Members), nil
+}
+
+func (s *v2v3Server) RemoveMember(ctx context.Context, id uint64) ([]*membership.Member, error) {
+	resp, err := s.c.MemberRemove(ctx, id)
+	if err != nil {
+		return nil, err
+	}
+	return v3MembersToMembership(resp.Members), nil
+}
+
+func (s *v2v3Server) PromoteMember(ctx context.Context, id uint64) ([]*membership.Member, error) {
+	resp, err := s.c.MemberPromote(ctx, id)
+	if err != nil {
+		return nil, err
+	}
+	return v3MembersToMembership(resp.Members), nil
+}
+
+func (s *v2v3Server) UpdateMember(ctx context.Context, m membership.Member) ([]*membership.Member, error) {
+	resp, err := s.c.MemberUpdate(ctx, uint64(m.ID), m.PeerURLs)
+	if err != nil {
+		return nil, err
+	}
+	return v3MembersToMembership(resp.Members), nil
+}
+
+func v3MembersToMembership(v3membs []*pb.Member) []*membership.Member {
+	membs := make([]*membership.Member, len(v3membs))
+	for i, m := range v3membs {
+		membs[i] = &membership.Member{
+			ID: types.ID(m.ID),
+			RaftAttributes: membership.RaftAttributes{
+				PeerURLs:  m.PeerURLs,
+				IsLearner: m.IsLearner,
+			},
+			Attributes: membership.Attributes{
+				Name:       m.Name,
+				ClientURLs: m.ClientURLs,
+			},
+		}
+	}
+	return membs
+}
+
+func (s *v2v3Server) ClusterVersion() *semver.Version { return s.Version() }
+func (s *v2v3Server) Cluster() api.Cluster            { return s }
+func (s *v2v3Server) Alarms() []*pb.AlarmMember       { return nil }
+
+func (s *v2v3Server) Do(ctx context.Context, r pb.Request) (etcdserver.Response, error) {
+	applier := etcdserver.NewApplierV2(s.lg, s.store, nil)
+	reqHandler := etcdserver.NewStoreRequestV2Handler(s.store, applier)
+	req := (*etcdserver.RequestV2)(&r)
+	resp, err := req.Handle(ctx, reqHandler)
+	if resp.Err != nil {
+		return resp, resp.Err
+	}
+	return resp, err
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2v3/store.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2v3/store.go
new file mode 100644
index 0000000..f1c7ab3
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2v3/store.go
@@ -0,0 +1,638 @@
+// 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 v2v3
+
+import (
+	"context"
+	"fmt"
+	"path"
+	"sort"
+	"strings"
+	"time"
+
+	"go.etcd.io/etcd/clientv3"
+	"go.etcd.io/etcd/clientv3/concurrency"
+	"go.etcd.io/etcd/etcdserver/api/v2error"
+	"go.etcd.io/etcd/etcdserver/api/v2store"
+	"go.etcd.io/etcd/mvcc/mvccpb"
+)
+
+// store implements the Store interface for V2 using
+// a v3 client.
+type v2v3Store struct {
+	c *clientv3.Client
+	// pfx is the v3 prefix where keys should be stored.
+	pfx string
+	ctx context.Context
+}
+
+const maxPathDepth = 63
+
+var errUnsupported = fmt.Errorf("TTLs are unsupported")
+
+func NewStore(c *clientv3.Client, pfx string) v2store.Store { return newStore(c, pfx) }
+
+func newStore(c *clientv3.Client, pfx string) *v2v3Store { return &v2v3Store{c, pfx, c.Ctx()} }
+
+func (s *v2v3Store) Index() uint64 { panic("STUB") }
+
+func (s *v2v3Store) Get(nodePath string, recursive, sorted bool) (*v2store.Event, error) {
+	key := s.mkPath(nodePath)
+	resp, err := s.c.Txn(s.ctx).Then(
+		clientv3.OpGet(key+"/"),
+		clientv3.OpGet(key),
+	).Commit()
+	if err != nil {
+		return nil, err
+	}
+
+	if kvs := resp.Responses[0].GetResponseRange().Kvs; len(kvs) != 0 || isRoot(nodePath) {
+		nodes, err := s.getDir(nodePath, recursive, sorted, resp.Header.Revision)
+		if err != nil {
+			return nil, err
+		}
+		cidx, midx := uint64(0), uint64(0)
+		if len(kvs) > 0 {
+			cidx, midx = mkV2Rev(kvs[0].CreateRevision), mkV2Rev(kvs[0].ModRevision)
+		}
+		return &v2store.Event{
+			Action: v2store.Get,
+			Node: &v2store.NodeExtern{
+				Key:           nodePath,
+				Dir:           true,
+				Nodes:         nodes,
+				CreatedIndex:  cidx,
+				ModifiedIndex: midx,
+			},
+			EtcdIndex: mkV2Rev(resp.Header.Revision),
+		}, nil
+	}
+
+	kvs := resp.Responses[1].GetResponseRange().Kvs
+	if len(kvs) == 0 {
+		return nil, v2error.NewError(v2error.EcodeKeyNotFound, nodePath, mkV2Rev(resp.Header.Revision))
+	}
+
+	return &v2store.Event{
+		Action:    v2store.Get,
+		Node:      s.mkV2Node(kvs[0]),
+		EtcdIndex: mkV2Rev(resp.Header.Revision),
+	}, nil
+}
+
+func (s *v2v3Store) getDir(nodePath string, recursive, sorted bool, rev int64) ([]*v2store.NodeExtern, error) {
+	rootNodes, err := s.getDirDepth(nodePath, 1, rev)
+	if err != nil || !recursive {
+		if sorted {
+			sort.Sort(v2store.NodeExterns(rootNodes))
+		}
+		return rootNodes, err
+	}
+	nextNodes := rootNodes
+	nodes := make(map[string]*v2store.NodeExtern)
+	// Breadth walk the subdirectories
+	for i := 2; len(nextNodes) > 0; i++ {
+		for _, n := range nextNodes {
+			nodes[n.Key] = n
+			if parent := nodes[path.Dir(n.Key)]; parent != nil {
+				parent.Nodes = append(parent.Nodes, n)
+			}
+		}
+		if nextNodes, err = s.getDirDepth(nodePath, i, rev); err != nil {
+			return nil, err
+		}
+	}
+
+	if sorted {
+		sort.Sort(v2store.NodeExterns(rootNodes))
+	}
+	return rootNodes, nil
+}
+
+func (s *v2v3Store) getDirDepth(nodePath string, depth int, rev int64) ([]*v2store.NodeExtern, error) {
+	pd := s.mkPathDepth(nodePath, depth)
+	resp, err := s.c.Get(s.ctx, pd, clientv3.WithPrefix(), clientv3.WithRev(rev))
+	if err != nil {
+		return nil, err
+	}
+
+	nodes := make([]*v2store.NodeExtern, len(resp.Kvs))
+	for i, kv := range resp.Kvs {
+		nodes[i] = s.mkV2Node(kv)
+	}
+	return nodes, nil
+}
+
+func (s *v2v3Store) Set(
+	nodePath string,
+	dir bool,
+	value string,
+	expireOpts v2store.TTLOptionSet,
+) (*v2store.Event, error) {
+	if expireOpts.Refresh || !expireOpts.ExpireTime.IsZero() {
+		return nil, errUnsupported
+	}
+
+	if isRoot(nodePath) {
+		return nil, v2error.NewError(v2error.EcodeRootROnly, nodePath, 0)
+	}
+
+	ecode := 0
+	applyf := func(stm concurrency.STM) error {
+		// build path if any directories in path do not exist
+		dirs := []string{}
+		for p := path.Dir(nodePath); !isRoot(p); p = path.Dir(p) {
+			pp := s.mkPath(p)
+			if stm.Rev(pp) > 0 {
+				ecode = v2error.EcodeNotDir
+				return nil
+			}
+			if stm.Rev(pp+"/") == 0 {
+				dirs = append(dirs, pp+"/")
+			}
+		}
+		for _, d := range dirs {
+			stm.Put(d, "")
+		}
+
+		key := s.mkPath(nodePath)
+		if dir {
+			if stm.Rev(key) != 0 {
+				// exists as non-dir
+				ecode = v2error.EcodeNotDir
+				return nil
+			}
+			key = key + "/"
+		} else if stm.Rev(key+"/") != 0 {
+			ecode = v2error.EcodeNotFile
+			return nil
+		}
+		stm.Put(key, value, clientv3.WithPrevKV())
+		stm.Put(s.mkActionKey(), v2store.Set)
+		return nil
+	}
+
+	resp, err := s.newSTM(applyf)
+	if err != nil {
+		return nil, err
+	}
+	if ecode != 0 {
+		return nil, v2error.NewError(ecode, nodePath, mkV2Rev(resp.Header.Revision))
+	}
+
+	createRev := resp.Header.Revision
+	var pn *v2store.NodeExtern
+	if pkv := prevKeyFromPuts(resp); pkv != nil {
+		pn = s.mkV2Node(pkv)
+		createRev = pkv.CreateRevision
+	}
+
+	vp := &value
+	if dir {
+		vp = nil
+	}
+	return &v2store.Event{
+		Action: v2store.Set,
+		Node: &v2store.NodeExtern{
+			Key:           nodePath,
+			Value:         vp,
+			Dir:           dir,
+			ModifiedIndex: mkV2Rev(resp.Header.Revision),
+			CreatedIndex:  mkV2Rev(createRev),
+		},
+		PrevNode:  pn,
+		EtcdIndex: mkV2Rev(resp.Header.Revision),
+	}, nil
+}
+
+func (s *v2v3Store) Update(nodePath, newValue string, expireOpts v2store.TTLOptionSet) (*v2store.Event, error) {
+	if isRoot(nodePath) {
+		return nil, v2error.NewError(v2error.EcodeRootROnly, nodePath, 0)
+	}
+
+	if expireOpts.Refresh || !expireOpts.ExpireTime.IsZero() {
+		return nil, errUnsupported
+	}
+
+	key := s.mkPath(nodePath)
+	ecode := 0
+	applyf := func(stm concurrency.STM) error {
+		if rev := stm.Rev(key + "/"); rev != 0 {
+			ecode = v2error.EcodeNotFile
+			return nil
+		}
+		if rev := stm.Rev(key); rev == 0 {
+			ecode = v2error.EcodeKeyNotFound
+			return nil
+		}
+		stm.Put(key, newValue, clientv3.WithPrevKV())
+		stm.Put(s.mkActionKey(), v2store.Update)
+		return nil
+	}
+
+	resp, err := s.newSTM(applyf)
+	if err != nil {
+		return nil, err
+	}
+	if ecode != 0 {
+		return nil, v2error.NewError(v2error.EcodeNotFile, nodePath, mkV2Rev(resp.Header.Revision))
+	}
+
+	pkv := prevKeyFromPuts(resp)
+	return &v2store.Event{
+		Action: v2store.Update,
+		Node: &v2store.NodeExtern{
+			Key:           nodePath,
+			Value:         &newValue,
+			ModifiedIndex: mkV2Rev(resp.Header.Revision),
+			CreatedIndex:  mkV2Rev(pkv.CreateRevision),
+		},
+		PrevNode:  s.mkV2Node(pkv),
+		EtcdIndex: mkV2Rev(resp.Header.Revision),
+	}, nil
+}
+
+func (s *v2v3Store) Create(
+	nodePath string,
+	dir bool,
+	value string,
+	unique bool,
+	expireOpts v2store.TTLOptionSet,
+) (*v2store.Event, error) {
+	if isRoot(nodePath) {
+		return nil, v2error.NewError(v2error.EcodeRootROnly, nodePath, 0)
+	}
+	if expireOpts.Refresh || !expireOpts.ExpireTime.IsZero() {
+		return nil, errUnsupported
+	}
+	ecode := 0
+	applyf := func(stm concurrency.STM) error {
+		ecode = 0
+		key := s.mkPath(nodePath)
+		if unique {
+			// append unique item under the node path
+			for {
+				key = nodePath + "/" + fmt.Sprintf("%020s", time.Now())
+				key = path.Clean(path.Join("/", key))
+				key = s.mkPath(key)
+				if stm.Rev(key) == 0 {
+					break
+				}
+			}
+		}
+		if stm.Rev(key) > 0 || stm.Rev(key+"/") > 0 {
+			ecode = v2error.EcodeNodeExist
+			return nil
+		}
+		// build path if any directories in path do not exist
+		dirs := []string{}
+		for p := path.Dir(nodePath); !isRoot(p); p = path.Dir(p) {
+			pp := s.mkPath(p)
+			if stm.Rev(pp) > 0 {
+				ecode = v2error.EcodeNotDir
+				return nil
+			}
+			if stm.Rev(pp+"/") == 0 {
+				dirs = append(dirs, pp+"/")
+			}
+		}
+		for _, d := range dirs {
+			stm.Put(d, "")
+		}
+
+		if dir {
+			// directories marked with extra slash in key name
+			key += "/"
+		}
+		stm.Put(key, value)
+		stm.Put(s.mkActionKey(), v2store.Create)
+		return nil
+	}
+
+	resp, err := s.newSTM(applyf)
+	if err != nil {
+		return nil, err
+	}
+	if ecode != 0 {
+		return nil, v2error.NewError(ecode, nodePath, mkV2Rev(resp.Header.Revision))
+	}
+
+	var v *string
+	if !dir {
+		v = &value
+	}
+
+	return &v2store.Event{
+		Action: v2store.Create,
+		Node: &v2store.NodeExtern{
+			Key:           nodePath,
+			Value:         v,
+			Dir:           dir,
+			ModifiedIndex: mkV2Rev(resp.Header.Revision),
+			CreatedIndex:  mkV2Rev(resp.Header.Revision),
+		},
+		EtcdIndex: mkV2Rev(resp.Header.Revision),
+	}, nil
+}
+
+func (s *v2v3Store) CompareAndSwap(
+	nodePath string,
+	prevValue string,
+	prevIndex uint64,
+	value string,
+	expireOpts v2store.TTLOptionSet,
+) (*v2store.Event, error) {
+	if isRoot(nodePath) {
+		return nil, v2error.NewError(v2error.EcodeRootROnly, nodePath, 0)
+	}
+	if expireOpts.Refresh || !expireOpts.ExpireTime.IsZero() {
+		return nil, errUnsupported
+	}
+
+	key := s.mkPath(nodePath)
+	resp, err := s.c.Txn(s.ctx).If(
+		s.mkCompare(nodePath, prevValue, prevIndex)...,
+	).Then(
+		clientv3.OpPut(key, value, clientv3.WithPrevKV()),
+		clientv3.OpPut(s.mkActionKey(), v2store.CompareAndSwap),
+	).Else(
+		clientv3.OpGet(key),
+		clientv3.OpGet(key+"/"),
+	).Commit()
+
+	if err != nil {
+		return nil, err
+	}
+	if !resp.Succeeded {
+		return nil, compareFail(nodePath, prevValue, prevIndex, resp)
+	}
+
+	pkv := resp.Responses[0].GetResponsePut().PrevKv
+	return &v2store.Event{
+		Action: v2store.CompareAndSwap,
+		Node: &v2store.NodeExtern{
+			Key:           nodePath,
+			Value:         &value,
+			CreatedIndex:  mkV2Rev(pkv.CreateRevision),
+			ModifiedIndex: mkV2Rev(resp.Header.Revision),
+		},
+		PrevNode:  s.mkV2Node(pkv),
+		EtcdIndex: mkV2Rev(resp.Header.Revision),
+	}, nil
+}
+
+func (s *v2v3Store) Delete(nodePath string, dir, recursive bool) (*v2store.Event, error) {
+	if isRoot(nodePath) {
+		return nil, v2error.NewError(v2error.EcodeRootROnly, nodePath, 0)
+	}
+	if !dir && !recursive {
+		return s.deleteNode(nodePath)
+	}
+	if !recursive {
+		return s.deleteEmptyDir(nodePath)
+	}
+
+	dels := make([]clientv3.Op, maxPathDepth+1)
+	dels[0] = clientv3.OpDelete(s.mkPath(nodePath)+"/", clientv3.WithPrevKV())
+	for i := 1; i < maxPathDepth; i++ {
+		dels[i] = clientv3.OpDelete(s.mkPathDepth(nodePath, i), clientv3.WithPrefix())
+	}
+	dels[maxPathDepth] = clientv3.OpPut(s.mkActionKey(), v2store.Delete)
+
+	resp, err := s.c.Txn(s.ctx).If(
+		clientv3.Compare(clientv3.Version(s.mkPath(nodePath)+"/"), ">", 0),
+		clientv3.Compare(clientv3.Version(s.mkPathDepth(nodePath, maxPathDepth)+"/"), "=", 0),
+	).Then(
+		dels...,
+	).Commit()
+	if err != nil {
+		return nil, err
+	}
+	if !resp.Succeeded {
+		return nil, v2error.NewError(v2error.EcodeNodeExist, nodePath, mkV2Rev(resp.Header.Revision))
+	}
+	dresp := resp.Responses[0].GetResponseDeleteRange()
+	return &v2store.Event{
+		Action:    v2store.Delete,
+		PrevNode:  s.mkV2Node(dresp.PrevKvs[0]),
+		EtcdIndex: mkV2Rev(resp.Header.Revision),
+	}, nil
+}
+
+func (s *v2v3Store) deleteEmptyDir(nodePath string) (*v2store.Event, error) {
+	resp, err := s.c.Txn(s.ctx).If(
+		clientv3.Compare(clientv3.Version(s.mkPathDepth(nodePath, 1)), "=", 0).WithPrefix(),
+	).Then(
+		clientv3.OpDelete(s.mkPath(nodePath)+"/", clientv3.WithPrevKV()),
+		clientv3.OpPut(s.mkActionKey(), v2store.Delete),
+	).Commit()
+	if err != nil {
+		return nil, err
+	}
+	if !resp.Succeeded {
+		return nil, v2error.NewError(v2error.EcodeDirNotEmpty, nodePath, mkV2Rev(resp.Header.Revision))
+	}
+	dresp := resp.Responses[0].GetResponseDeleteRange()
+	if len(dresp.PrevKvs) == 0 {
+		return nil, v2error.NewError(v2error.EcodeNodeExist, nodePath, mkV2Rev(resp.Header.Revision))
+	}
+	return &v2store.Event{
+		Action:    v2store.Delete,
+		PrevNode:  s.mkV2Node(dresp.PrevKvs[0]),
+		EtcdIndex: mkV2Rev(resp.Header.Revision),
+	}, nil
+}
+
+func (s *v2v3Store) deleteNode(nodePath string) (*v2store.Event, error) {
+	resp, err := s.c.Txn(s.ctx).If(
+		clientv3.Compare(clientv3.Version(s.mkPath(nodePath)+"/"), "=", 0),
+	).Then(
+		clientv3.OpDelete(s.mkPath(nodePath), clientv3.WithPrevKV()),
+		clientv3.OpPut(s.mkActionKey(), v2store.Delete),
+	).Commit()
+	if err != nil {
+		return nil, err
+	}
+	if !resp.Succeeded {
+		return nil, v2error.NewError(v2error.EcodeNotFile, nodePath, mkV2Rev(resp.Header.Revision))
+	}
+	pkvs := resp.Responses[0].GetResponseDeleteRange().PrevKvs
+	if len(pkvs) == 0 {
+		return nil, v2error.NewError(v2error.EcodeKeyNotFound, nodePath, mkV2Rev(resp.Header.Revision))
+	}
+	pkv := pkvs[0]
+	return &v2store.Event{
+		Action: v2store.Delete,
+		Node: &v2store.NodeExtern{
+			Key:           nodePath,
+			CreatedIndex:  mkV2Rev(pkv.CreateRevision),
+			ModifiedIndex: mkV2Rev(resp.Header.Revision),
+		},
+		PrevNode:  s.mkV2Node(pkv),
+		EtcdIndex: mkV2Rev(resp.Header.Revision),
+	}, nil
+}
+
+func (s *v2v3Store) CompareAndDelete(nodePath, prevValue string, prevIndex uint64) (*v2store.Event, error) {
+	if isRoot(nodePath) {
+		return nil, v2error.NewError(v2error.EcodeRootROnly, nodePath, 0)
+	}
+
+	key := s.mkPath(nodePath)
+	resp, err := s.c.Txn(s.ctx).If(
+		s.mkCompare(nodePath, prevValue, prevIndex)...,
+	).Then(
+		clientv3.OpDelete(key, clientv3.WithPrevKV()),
+		clientv3.OpPut(s.mkActionKey(), v2store.CompareAndDelete),
+	).Else(
+		clientv3.OpGet(key),
+		clientv3.OpGet(key+"/"),
+	).Commit()
+
+	if err != nil {
+		return nil, err
+	}
+	if !resp.Succeeded {
+		return nil, compareFail(nodePath, prevValue, prevIndex, resp)
+	}
+
+	// len(pkvs) > 1 since txn only succeeds when key exists
+	pkv := resp.Responses[0].GetResponseDeleteRange().PrevKvs[0]
+	return &v2store.Event{
+		Action: v2store.CompareAndDelete,
+		Node: &v2store.NodeExtern{
+			Key:           nodePath,
+			CreatedIndex:  mkV2Rev(pkv.CreateRevision),
+			ModifiedIndex: mkV2Rev(resp.Header.Revision),
+		},
+		PrevNode:  s.mkV2Node(pkv),
+		EtcdIndex: mkV2Rev(resp.Header.Revision),
+	}, nil
+}
+
+func compareFail(nodePath, prevValue string, prevIndex uint64, resp *clientv3.TxnResponse) error {
+	if dkvs := resp.Responses[1].GetResponseRange().Kvs; len(dkvs) > 0 {
+		return v2error.NewError(v2error.EcodeNotFile, nodePath, mkV2Rev(resp.Header.Revision))
+	}
+	kvs := resp.Responses[0].GetResponseRange().Kvs
+	if len(kvs) == 0 {
+		return v2error.NewError(v2error.EcodeKeyNotFound, nodePath, mkV2Rev(resp.Header.Revision))
+	}
+	kv := kvs[0]
+	indexMatch := prevIndex == 0 || kv.ModRevision == int64(prevIndex)
+	valueMatch := prevValue == "" || string(kv.Value) == prevValue
+	var cause string
+	switch {
+	case indexMatch && !valueMatch:
+		cause = fmt.Sprintf("[%v != %v]", prevValue, string(kv.Value))
+	case valueMatch && !indexMatch:
+		cause = fmt.Sprintf("[%v != %v]", prevIndex, kv.ModRevision)
+	default:
+		cause = fmt.Sprintf("[%v != %v] [%v != %v]", prevValue, string(kv.Value), prevIndex, kv.ModRevision)
+	}
+	return v2error.NewError(v2error.EcodeTestFailed, cause, mkV2Rev(resp.Header.Revision))
+}
+
+func (s *v2v3Store) mkCompare(nodePath, prevValue string, prevIndex uint64) []clientv3.Cmp {
+	key := s.mkPath(nodePath)
+	cmps := []clientv3.Cmp{clientv3.Compare(clientv3.Version(key), ">", 0)}
+	if prevIndex != 0 {
+		cmps = append(cmps, clientv3.Compare(clientv3.ModRevision(key), "=", mkV3Rev(prevIndex)))
+	}
+	if prevValue != "" {
+		cmps = append(cmps, clientv3.Compare(clientv3.Value(key), "=", prevValue))
+	}
+	return cmps
+}
+
+func (s *v2v3Store) JsonStats() []byte                  { panic("STUB") }
+func (s *v2v3Store) DeleteExpiredKeys(cutoff time.Time) { panic("STUB") }
+
+func (s *v2v3Store) Version() int { return 2 }
+
+// TODO: move this out of the Store interface?
+
+func (s *v2v3Store) Save() ([]byte, error)       { panic("STUB") }
+func (s *v2v3Store) Recovery(state []byte) error { panic("STUB") }
+func (s *v2v3Store) Clone() v2store.Store        { panic("STUB") }
+func (s *v2v3Store) SaveNoCopy() ([]byte, error) { panic("STUB") }
+func (s *v2v3Store) HasTTLKeys() bool            { panic("STUB") }
+
+func (s *v2v3Store) mkPath(nodePath string) string { return s.mkPathDepth(nodePath, 0) }
+
+func (s *v2v3Store) mkNodePath(p string) string {
+	return path.Clean(p[len(s.pfx)+len("/k/000/"):])
+}
+
+// mkPathDepth makes a path to a key that encodes its directory depth
+// for fast directory listing. If a depth is provided, it is added
+// to the computed depth.
+func (s *v2v3Store) mkPathDepth(nodePath string, depth int) string {
+	normalForm := path.Clean(path.Join("/", nodePath))
+	n := strings.Count(normalForm, "/") + depth
+	return fmt.Sprintf("%s/%03d/k/%s", s.pfx, n, normalForm)
+}
+
+func (s *v2v3Store) mkActionKey() string { return s.pfx + "/act" }
+
+func isRoot(s string) bool { return len(s) == 0 || s == "/" || s == "/0" || s == "/1" }
+
+func mkV2Rev(v3Rev int64) uint64 {
+	if v3Rev == 0 {
+		return 0
+	}
+	return uint64(v3Rev - 1)
+}
+
+func mkV3Rev(v2Rev uint64) int64 {
+	if v2Rev == 0 {
+		return 0
+	}
+	return int64(v2Rev + 1)
+}
+
+// mkV2Node creates a V2 NodeExtern from a V3 KeyValue
+func (s *v2v3Store) mkV2Node(kv *mvccpb.KeyValue) *v2store.NodeExtern {
+	if kv == nil {
+		return nil
+	}
+	n := &v2store.NodeExtern{
+		Key:           s.mkNodePath(string(kv.Key)),
+		Dir:           kv.Key[len(kv.Key)-1] == '/',
+		CreatedIndex:  mkV2Rev(kv.CreateRevision),
+		ModifiedIndex: mkV2Rev(kv.ModRevision),
+	}
+	if !n.Dir {
+		v := string(kv.Value)
+		n.Value = &v
+	}
+	return n
+}
+
+// prevKeyFromPuts gets the prev key that is being put; ignores
+// the put action response.
+func prevKeyFromPuts(resp *clientv3.TxnResponse) *mvccpb.KeyValue {
+	for _, r := range resp.Responses {
+		pkv := r.GetResponsePut().PrevKv
+		if pkv != nil && pkv.CreateRevision > 0 {
+			return pkv
+		}
+	}
+	return nil
+}
+
+func (s *v2v3Store) newSTM(applyf func(concurrency.STM) error) (*clientv3.TxnResponse, error) {
+	return concurrency.NewSTM(s.c, applyf, concurrency.WithIsolation(concurrency.Serializable))
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v2v3/watcher.go b/vendor/go.etcd.io/etcd/etcdserver/api/v2v3/watcher.go
new file mode 100644
index 0000000..e8a3557
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v2v3/watcher.go
@@ -0,0 +1,140 @@
+// 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 v2v3
+
+import (
+	"context"
+	"strings"
+
+	"go.etcd.io/etcd/clientv3"
+	"go.etcd.io/etcd/etcdserver/api/v2error"
+	"go.etcd.io/etcd/etcdserver/api/v2store"
+)
+
+func (s *v2v3Store) Watch(prefix string, recursive, stream bool, sinceIndex uint64) (v2store.Watcher, error) {
+	ctx, cancel := context.WithCancel(s.ctx)
+	wch := s.c.Watch(
+		ctx,
+		// TODO: very pricey; use a single store-wide watch in future
+		s.pfx,
+		clientv3.WithPrefix(),
+		clientv3.WithRev(int64(sinceIndex)),
+		clientv3.WithCreatedNotify(),
+		clientv3.WithPrevKV())
+	resp, ok := <-wch
+	if err := resp.Err(); err != nil || !ok {
+		cancel()
+		return nil, v2error.NewError(v2error.EcodeRaftInternal, prefix, 0)
+	}
+
+	evc, donec := make(chan *v2store.Event), make(chan struct{})
+	go func() {
+		defer func() {
+			close(evc)
+			close(donec)
+		}()
+		for resp := range wch {
+			for _, ev := range s.mkV2Events(resp) {
+				k := ev.Node.Key
+				if recursive {
+					if !strings.HasPrefix(k, prefix) {
+						continue
+					}
+					// accept events on hidden keys given in prefix
+					k = strings.Replace(k, prefix, "/", 1)
+					// ignore hidden keys deeper than prefix
+					if strings.Contains(k, "/_") {
+						continue
+					}
+				}
+				if !recursive && k != prefix {
+					continue
+				}
+				select {
+				case evc <- ev:
+				case <-ctx.Done():
+					return
+				}
+				if !stream {
+					return
+				}
+			}
+		}
+	}()
+
+	return &v2v3Watcher{
+		startRev: resp.Header.Revision,
+		evc:      evc,
+		donec:    donec,
+		cancel:   cancel,
+	}, nil
+}
+
+func (s *v2v3Store) mkV2Events(wr clientv3.WatchResponse) (evs []*v2store.Event) {
+	ak := s.mkActionKey()
+	for _, rev := range mkRevs(wr) {
+		var act, key *clientv3.Event
+		for _, ev := range rev {
+			if string(ev.Kv.Key) == ak {
+				act = ev
+			} else if key != nil && len(key.Kv.Key) < len(ev.Kv.Key) {
+				// use longest key to ignore intermediate new
+				// directories from Create.
+				key = ev
+			} else if key == nil {
+				key = ev
+			}
+		}
+		v2ev := &v2store.Event{
+			Action:    string(act.Kv.Value),
+			Node:      s.mkV2Node(key.Kv),
+			PrevNode:  s.mkV2Node(key.PrevKv),
+			EtcdIndex: mkV2Rev(wr.Header.Revision),
+		}
+		evs = append(evs, v2ev)
+	}
+	return evs
+}
+
+func mkRevs(wr clientv3.WatchResponse) (revs [][]*clientv3.Event) {
+	var curRev []*clientv3.Event
+	for _, ev := range wr.Events {
+		if curRev != nil && ev.Kv.ModRevision != curRev[0].Kv.ModRevision {
+			revs = append(revs, curRev)
+			curRev = nil
+		}
+		curRev = append(curRev, ev)
+	}
+	if curRev != nil {
+		revs = append(revs, curRev)
+	}
+	return revs
+}
+
+type v2v3Watcher struct {
+	startRev int64
+	evc      chan *v2store.Event
+	donec    chan struct{}
+	cancel   context.CancelFunc
+}
+
+func (w *v2v3Watcher) StartIndex() uint64 { return mkV2Rev(w.startRev) }
+
+func (w *v2v3Watcher) Remove() {
+	w.cancel()
+	<-w.donec
+}
+
+func (w *v2v3Watcher) EventChan() chan *v2store.Event { return w.evc }
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3alarm/alarms.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3alarm/alarms.go
new file mode 100644
index 0000000..2b085a8
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3alarm/alarms.go
@@ -0,0 +1,153 @@
+// Copyright 2016 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 v3alarm manages health status alarms in etcd.
+package v3alarm
+
+import (
+	"sync"
+
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/mvcc/backend"
+	"go.etcd.io/etcd/pkg/types"
+
+	"github.com/coreos/pkg/capnslog"
+)
+
+var (
+	alarmBucketName = []byte("alarm")
+	plog            = capnslog.NewPackageLogger("go.etcd.io/etcd", "alarm")
+)
+
+type BackendGetter interface {
+	Backend() backend.Backend
+}
+
+type alarmSet map[types.ID]*pb.AlarmMember
+
+// AlarmStore persists alarms to the backend.
+type AlarmStore struct {
+	mu    sync.Mutex
+	types map[pb.AlarmType]alarmSet
+
+	bg BackendGetter
+}
+
+func NewAlarmStore(bg BackendGetter) (*AlarmStore, error) {
+	ret := &AlarmStore{types: make(map[pb.AlarmType]alarmSet), bg: bg}
+	err := ret.restore()
+	return ret, err
+}
+
+func (a *AlarmStore) Activate(id types.ID, at pb.AlarmType) *pb.AlarmMember {
+	a.mu.Lock()
+	defer a.mu.Unlock()
+
+	newAlarm := &pb.AlarmMember{MemberID: uint64(id), Alarm: at}
+	if m := a.addToMap(newAlarm); m != newAlarm {
+		return m
+	}
+
+	v, err := newAlarm.Marshal()
+	if err != nil {
+		plog.Panicf("failed to marshal alarm member")
+	}
+
+	b := a.bg.Backend()
+	b.BatchTx().Lock()
+	b.BatchTx().UnsafePut(alarmBucketName, v, nil)
+	b.BatchTx().Unlock()
+
+	return newAlarm
+}
+
+func (a *AlarmStore) Deactivate(id types.ID, at pb.AlarmType) *pb.AlarmMember {
+	a.mu.Lock()
+	defer a.mu.Unlock()
+
+	t := a.types[at]
+	if t == nil {
+		t = make(alarmSet)
+		a.types[at] = t
+	}
+	m := t[id]
+	if m == nil {
+		return nil
+	}
+
+	delete(t, id)
+
+	v, err := m.Marshal()
+	if err != nil {
+		plog.Panicf("failed to marshal alarm member")
+	}
+
+	b := a.bg.Backend()
+	b.BatchTx().Lock()
+	b.BatchTx().UnsafeDelete(alarmBucketName, v)
+	b.BatchTx().Unlock()
+
+	return m
+}
+
+func (a *AlarmStore) Get(at pb.AlarmType) (ret []*pb.AlarmMember) {
+	a.mu.Lock()
+	defer a.mu.Unlock()
+	if at == pb.AlarmType_NONE {
+		for _, t := range a.types {
+			for _, m := range t {
+				ret = append(ret, m)
+			}
+		}
+		return ret
+	}
+	for _, m := range a.types[at] {
+		ret = append(ret, m)
+	}
+	return ret
+}
+
+func (a *AlarmStore) restore() error {
+	b := a.bg.Backend()
+	tx := b.BatchTx()
+
+	tx.Lock()
+	tx.UnsafeCreateBucket(alarmBucketName)
+	err := tx.UnsafeForEach(alarmBucketName, func(k, v []byte) error {
+		var m pb.AlarmMember
+		if err := m.Unmarshal(k); err != nil {
+			return err
+		}
+		a.addToMap(&m)
+		return nil
+	})
+	tx.Unlock()
+
+	b.ForceCommit()
+	return err
+}
+
+func (a *AlarmStore) addToMap(newAlarm *pb.AlarmMember) *pb.AlarmMember {
+	t := a.types[newAlarm.Alarm]
+	if t == nil {
+		t = make(alarmSet)
+		a.types[newAlarm.Alarm] = t
+	}
+	m := t[types.ID(newAlarm.MemberID)]
+	if m != nil {
+		return m
+	}
+	t[types.ID(newAlarm.MemberID)] = newAlarm
+	return newAlarm
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3client/doc.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3client/doc.go
new file mode 100644
index 0000000..47922c4
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3client/doc.go
@@ -0,0 +1,45 @@
+// 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 v3client provides clientv3 interfaces from an etcdserver.
+//
+// Use v3client by creating an EtcdServer instance, then wrapping it with v3client.New:
+//
+//	import (
+//		"context"
+//
+//		"go.etcd.io/etcd/embed"
+//		"go.etcd.io/etcd/etcdserver/api/v3client"
+//	)
+//
+//	...
+//
+//	// create an embedded EtcdServer from the default configuration
+//	cfg := embed.NewConfig()
+//	cfg.Dir = "default.etcd"
+//	e, err := embed.StartEtcd(cfg)
+//	if err != nil {
+//		// handle error!
+//	}
+//
+//	// wrap the EtcdServer with v3client
+//	cli := v3client.New(e.Server)
+//
+//	// use like an ordinary clientv3
+//	resp, err := cli.Put(context.TODO(), "some-key", "it works!")
+//	if err != nil {
+//		// handle error!
+//	}
+//
+package v3client
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3client/v3client.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3client/v3client.go
new file mode 100644
index 0000000..d203121
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3client/v3client.go
@@ -0,0 +1,66 @@
+// 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 v3client
+
+import (
+	"context"
+	"time"
+
+	"go.etcd.io/etcd/clientv3"
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/api/v3rpc"
+	"go.etcd.io/etcd/proxy/grpcproxy/adapter"
+)
+
+// New creates a clientv3 client that wraps an in-process EtcdServer. Instead
+// of making gRPC calls through sockets, the client makes direct function calls
+// to the etcd server through its api/v3rpc function interfaces.
+func New(s *etcdserver.EtcdServer) *clientv3.Client {
+	c := clientv3.NewCtxClient(context.Background())
+
+	kvc := adapter.KvServerToKvClient(v3rpc.NewQuotaKVServer(s))
+	c.KV = clientv3.NewKVFromKVClient(kvc, c)
+
+	lc := adapter.LeaseServerToLeaseClient(v3rpc.NewQuotaLeaseServer(s))
+	c.Lease = clientv3.NewLeaseFromLeaseClient(lc, c, time.Second)
+
+	wc := adapter.WatchServerToWatchClient(v3rpc.NewWatchServer(s))
+	c.Watcher = &watchWrapper{clientv3.NewWatchFromWatchClient(wc, c)}
+
+	mc := adapter.MaintenanceServerToMaintenanceClient(v3rpc.NewMaintenanceServer(s))
+	c.Maintenance = clientv3.NewMaintenanceFromMaintenanceClient(mc, c)
+
+	clc := adapter.ClusterServerToClusterClient(v3rpc.NewClusterServer(s))
+	c.Cluster = clientv3.NewClusterFromClusterClient(clc, c)
+
+	// TODO: implement clientv3.Auth interface?
+
+	return c
+}
+
+// BlankContext implements Stringer on a context so the ctx string doesn't
+// depend on the context's WithValue data, which tends to be unsynchronized
+// (e.g., x/net/trace), causing ctx.String() to throw data races.
+type blankContext struct{ context.Context }
+
+func (*blankContext) String() string { return "(blankCtx)" }
+
+// watchWrapper wraps clientv3 watch calls to blank out the context
+// to avoid races on trace data.
+type watchWrapper struct{ clientv3.Watcher }
+
+func (ww *watchWrapper) Watch(ctx context.Context, key string, opts ...clientv3.OpOption) clientv3.WatchChan {
+	return ww.Watcher.Watch(&blankContext{ctx}, key, opts...)
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3compactor/compactor.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3compactor/compactor.go
new file mode 100644
index 0000000..73a9684
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3compactor/compactor.go
@@ -0,0 +1,75 @@
+// Copyright 2016 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 v3compactor
+
+import (
+	"context"
+	"fmt"
+	"time"
+
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+
+	"github.com/coreos/pkg/capnslog"
+	"github.com/jonboulle/clockwork"
+	"go.uber.org/zap"
+)
+
+var (
+	plog = capnslog.NewPackageLogger("go.etcd.io/etcd", "compactor")
+)
+
+const (
+	ModePeriodic = "periodic"
+	ModeRevision = "revision"
+)
+
+// Compactor purges old log from the storage periodically.
+type Compactor interface {
+	// Run starts the main loop of the compactor in background.
+	// Use Stop() to halt the loop and release the resource.
+	Run()
+	// Stop halts the main loop of the compactor.
+	Stop()
+	// Pause temporally suspend the compactor not to run compaction. Resume() to unpose.
+	Pause()
+	// Resume restarts the compactor suspended by Pause().
+	Resume()
+}
+
+type Compactable interface {
+	Compact(ctx context.Context, r *pb.CompactionRequest) (*pb.CompactionResponse, error)
+}
+
+type RevGetter interface {
+	Rev() int64
+}
+
+// New returns a new Compactor based on given "mode".
+func New(
+	lg *zap.Logger,
+	mode string,
+	retention time.Duration,
+	rg RevGetter,
+	c Compactable,
+) (Compactor, error) {
+	switch mode {
+	case ModePeriodic:
+		return newPeriodic(lg, clockwork.NewRealClock(), retention, rg, c), nil
+	case ModeRevision:
+		return newRevision(lg, clockwork.NewRealClock(), int64(retention), rg, c), nil
+	default:
+		return nil, fmt.Errorf("unsupported compaction mode %s", mode)
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3compactor/doc.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3compactor/doc.go
new file mode 100644
index 0000000..bb28046
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3compactor/doc.go
@@ -0,0 +1,16 @@
+// Copyright 2016 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 v3compactor implements automated policies for compacting etcd's mvcc storage.
+package v3compactor
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3compactor/periodic.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3compactor/periodic.go
new file mode 100644
index 0000000..ab64cb7
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3compactor/periodic.go
@@ -0,0 +1,217 @@
+// Copyright 2017 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package v3compactor
+
+import (
+	"context"
+	"sync"
+	"time"
+
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/mvcc"
+
+	"github.com/jonboulle/clockwork"
+	"go.uber.org/zap"
+)
+
+// Periodic compacts the log by purging revisions older than
+// the configured retention time.
+type Periodic struct {
+	lg     *zap.Logger
+	clock  clockwork.Clock
+	period time.Duration
+
+	rg RevGetter
+	c  Compactable
+
+	revs   []int64
+	ctx    context.Context
+	cancel context.CancelFunc
+
+	// mu protects paused
+	mu     sync.RWMutex
+	paused bool
+}
+
+// newPeriodic creates a new instance of Periodic compactor that purges
+// the log older than h Duration.
+func newPeriodic(lg *zap.Logger, clock clockwork.Clock, h time.Duration, rg RevGetter, c Compactable) *Periodic {
+	pc := &Periodic{
+		lg:     lg,
+		clock:  clock,
+		period: h,
+		rg:     rg,
+		c:      c,
+		revs:   make([]int64, 0),
+	}
+	pc.ctx, pc.cancel = context.WithCancel(context.Background())
+	return pc
+}
+
+/*
+Compaction period 1-hour:
+  1. compute compaction period, which is 1-hour
+  2. record revisions for every 1/10 of 1-hour (6-minute)
+  3. keep recording revisions with no compaction for first 1-hour
+  4. do compact with revs[0]
+	- success? contiue on for-loop and move sliding window; revs = revs[1:]
+	- failure? update revs, and retry after 1/10 of 1-hour (6-minute)
+
+Compaction period 24-hour:
+  1. compute compaction period, which is 1-hour
+  2. record revisions for every 1/10 of 1-hour (6-minute)
+  3. keep recording revisions with no compaction for first 24-hour
+  4. do compact with revs[0]
+	- success? contiue on for-loop and move sliding window; revs = revs[1:]
+	- failure? update revs, and retry after 1/10 of 1-hour (6-minute)
+
+Compaction period 59-min:
+  1. compute compaction period, which is 59-min
+  2. record revisions for every 1/10 of 59-min (5.9-min)
+  3. keep recording revisions with no compaction for first 59-min
+  4. do compact with revs[0]
+	- success? contiue on for-loop and move sliding window; revs = revs[1:]
+	- failure? update revs, and retry after 1/10 of 59-min (5.9-min)
+
+Compaction period 5-sec:
+  1. compute compaction period, which is 5-sec
+  2. record revisions for every 1/10 of 5-sec (0.5-sec)
+  3. keep recording revisions with no compaction for first 5-sec
+  4. do compact with revs[0]
+	- success? contiue on for-loop and move sliding window; revs = revs[1:]
+	- failure? update revs, and retry after 1/10 of 5-sec (0.5-sec)
+*/
+
+// Run runs periodic compactor.
+func (pc *Periodic) Run() {
+	compactInterval := pc.getCompactInterval()
+	retryInterval := pc.getRetryInterval()
+	retentions := pc.getRetentions()
+
+	go func() {
+		lastSuccess := pc.clock.Now()
+		baseInterval := pc.period
+		for {
+			pc.revs = append(pc.revs, pc.rg.Rev())
+			if len(pc.revs) > retentions {
+				pc.revs = pc.revs[1:] // pc.revs[0] is always the rev at pc.period ago
+			}
+
+			select {
+			case <-pc.ctx.Done():
+				return
+			case <-pc.clock.After(retryInterval):
+				pc.mu.Lock()
+				p := pc.paused
+				pc.mu.Unlock()
+				if p {
+					continue
+				}
+			}
+
+			if pc.clock.Now().Sub(lastSuccess) < baseInterval {
+				continue
+			}
+
+			// wait up to initial given period
+			if baseInterval == pc.period {
+				baseInterval = compactInterval
+			}
+			rev := pc.revs[0]
+
+			if pc.lg != nil {
+				pc.lg.Info(
+					"starting auto periodic compaction",
+					zap.Int64("revision", rev),
+					zap.Duration("compact-period", pc.period),
+				)
+			} else {
+				plog.Noticef("Starting auto-compaction at revision %d (retention: %v)", rev, pc.period)
+			}
+			_, err := pc.c.Compact(pc.ctx, &pb.CompactionRequest{Revision: rev})
+			if err == nil || err == mvcc.ErrCompacted {
+				if pc.lg != nil {
+					pc.lg.Info(
+						"completed auto periodic compaction",
+						zap.Int64("revision", rev),
+						zap.Duration("compact-period", pc.period),
+						zap.Duration("took", time.Since(lastSuccess)),
+					)
+				} else {
+					plog.Noticef("Finished auto-compaction at revision %d", rev)
+				}
+				lastSuccess = pc.clock.Now()
+			} else {
+				if pc.lg != nil {
+					pc.lg.Warn(
+						"failed auto periodic compaction",
+						zap.Int64("revision", rev),
+						zap.Duration("compact-period", pc.period),
+						zap.Duration("retry-interval", retryInterval),
+						zap.Error(err),
+					)
+				} else {
+					plog.Noticef("Failed auto-compaction at revision %d (%v)", rev, err)
+					plog.Noticef("Retry after %v", retryInterval)
+				}
+			}
+		}
+	}()
+}
+
+// if given compaction period x is <1-hour, compact every x duration.
+// (e.g. --auto-compaction-mode 'periodic' --auto-compaction-retention='10m', then compact every 10-minute)
+// if given compaction period x is >1-hour, compact every hour.
+// (e.g. --auto-compaction-mode 'periodic' --auto-compaction-retention='2h', then compact every 1-hour)
+func (pc *Periodic) getCompactInterval() time.Duration {
+	itv := pc.period
+	if itv > time.Hour {
+		itv = time.Hour
+	}
+	return itv
+}
+
+func (pc *Periodic) getRetentions() int {
+	return int(pc.period/pc.getRetryInterval()) + 1
+}
+
+const retryDivisor = 10
+
+func (pc *Periodic) getRetryInterval() time.Duration {
+	itv := pc.period
+	if itv > time.Hour {
+		itv = time.Hour
+	}
+	return itv / retryDivisor
+}
+
+// Stop stops periodic compactor.
+func (pc *Periodic) Stop() {
+	pc.cancel()
+}
+
+// Pause pauses periodic compactor.
+func (pc *Periodic) Pause() {
+	pc.mu.Lock()
+	pc.paused = true
+	pc.mu.Unlock()
+}
+
+// Resume resumes periodic compactor.
+func (pc *Periodic) Resume() {
+	pc.mu.Lock()
+	pc.paused = false
+	pc.mu.Unlock()
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3compactor/revision.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3compactor/revision.go
new file mode 100644
index 0000000..cf8ac43
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3compactor/revision.go
@@ -0,0 +1,143 @@
+// 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 v3compactor
+
+import (
+	"context"
+	"sync"
+	"time"
+
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/mvcc"
+
+	"github.com/jonboulle/clockwork"
+	"go.uber.org/zap"
+)
+
+// Revision compacts the log by purging revisions older than
+// the configured reivison number. Compaction happens every 5 minutes.
+type Revision struct {
+	lg *zap.Logger
+
+	clock     clockwork.Clock
+	retention int64
+
+	rg RevGetter
+	c  Compactable
+
+	ctx    context.Context
+	cancel context.CancelFunc
+
+	mu     sync.Mutex
+	paused bool
+}
+
+// newRevision creates a new instance of Revisonal compactor that purges
+// the log older than retention revisions from the current revision.
+func newRevision(lg *zap.Logger, clock clockwork.Clock, retention int64, rg RevGetter, c Compactable) *Revision {
+	rc := &Revision{
+		lg:        lg,
+		clock:     clock,
+		retention: retention,
+		rg:        rg,
+		c:         c,
+	}
+	rc.ctx, rc.cancel = context.WithCancel(context.Background())
+	return rc
+}
+
+const revInterval = 5 * time.Minute
+
+// Run runs revision-based compactor.
+func (rc *Revision) Run() {
+	prev := int64(0)
+	go func() {
+		for {
+			select {
+			case <-rc.ctx.Done():
+				return
+			case <-rc.clock.After(revInterval):
+				rc.mu.Lock()
+				p := rc.paused
+				rc.mu.Unlock()
+				if p {
+					continue
+				}
+			}
+
+			rev := rc.rg.Rev() - rc.retention
+			if rev <= 0 || rev == prev {
+				continue
+			}
+
+			now := time.Now()
+			if rc.lg != nil {
+				rc.lg.Info(
+					"starting auto revision compaction",
+					zap.Int64("revision", rev),
+					zap.Int64("revision-compaction-retention", rc.retention),
+				)
+			} else {
+				plog.Noticef("Starting auto-compaction at revision %d (retention: %d revisions)", rev, rc.retention)
+			}
+			_, err := rc.c.Compact(rc.ctx, &pb.CompactionRequest{Revision: rev})
+			if err == nil || err == mvcc.ErrCompacted {
+				prev = rev
+				if rc.lg != nil {
+					rc.lg.Info(
+						"completed auto revision compaction",
+						zap.Int64("revision", rev),
+						zap.Int64("revision-compaction-retention", rc.retention),
+						zap.Duration("took", time.Since(now)),
+					)
+				} else {
+					plog.Noticef("Finished auto-compaction at revision %d", rev)
+				}
+			} else {
+				if rc.lg != nil {
+					rc.lg.Warn(
+						"failed auto revision compaction",
+						zap.Int64("revision", rev),
+						zap.Int64("revision-compaction-retention", rc.retention),
+						zap.Duration("retry-interval", revInterval),
+						zap.Error(err),
+					)
+				} else {
+					plog.Noticef("Failed auto-compaction at revision %d (%v)", rev, err)
+					plog.Noticef("Retry after %v", revInterval)
+				}
+			}
+		}
+	}()
+}
+
+// Stop stops revision-based compactor.
+func (rc *Revision) Stop() {
+	rc.cancel()
+}
+
+// Pause pauses revision-based compactor.
+func (rc *Revision) Pause() {
+	rc.mu.Lock()
+	rc.paused = true
+	rc.mu.Unlock()
+}
+
+// Resume resumes revision-based compactor.
+func (rc *Revision) Resume() {
+	rc.mu.Lock()
+	rc.paused = false
+	rc.mu.Unlock()
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3election/doc.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3election/doc.go
new file mode 100644
index 0000000..d6fefd7
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3election/doc.go
@@ -0,0 +1,16 @@
+// 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 v3election provides a v3 election service from an etcdserver.
+package v3election
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3election/election.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3election/election.go
new file mode 100644
index 0000000..f5a3be3
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3election/election.go
@@ -0,0 +1,134 @@
+// 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 v3election
+
+import (
+	"context"
+	"errors"
+
+	"go.etcd.io/etcd/clientv3"
+	"go.etcd.io/etcd/clientv3/concurrency"
+	epb "go.etcd.io/etcd/etcdserver/api/v3election/v3electionpb"
+)
+
+// ErrMissingLeaderKey is returned when election API request
+// is missing the "leader" field.
+var ErrMissingLeaderKey = errors.New(`"leader" field must be provided`)
+
+type electionServer struct {
+	c *clientv3.Client
+}
+
+func NewElectionServer(c *clientv3.Client) epb.ElectionServer {
+	return &electionServer{c}
+}
+
+func (es *electionServer) Campaign(ctx context.Context, req *epb.CampaignRequest) (*epb.CampaignResponse, error) {
+	s, err := es.session(ctx, req.Lease)
+	if err != nil {
+		return nil, err
+	}
+	e := concurrency.NewElection(s, string(req.Name))
+	if err = e.Campaign(ctx, string(req.Value)); err != nil {
+		return nil, err
+	}
+	return &epb.CampaignResponse{
+		Header: e.Header(),
+		Leader: &epb.LeaderKey{
+			Name:  req.Name,
+			Key:   []byte(e.Key()),
+			Rev:   e.Rev(),
+			Lease: int64(s.Lease()),
+		},
+	}, nil
+}
+
+func (es *electionServer) Proclaim(ctx context.Context, req *epb.ProclaimRequest) (*epb.ProclaimResponse, error) {
+	if req.Leader == nil {
+		return nil, ErrMissingLeaderKey
+	}
+	s, err := es.session(ctx, req.Leader.Lease)
+	if err != nil {
+		return nil, err
+	}
+	e := concurrency.ResumeElection(s, string(req.Leader.Name), string(req.Leader.Key), req.Leader.Rev)
+	if err := e.Proclaim(ctx, string(req.Value)); err != nil {
+		return nil, err
+	}
+	return &epb.ProclaimResponse{Header: e.Header()}, nil
+}
+
+func (es *electionServer) Observe(req *epb.LeaderRequest, stream epb.Election_ObserveServer) error {
+	s, err := es.session(stream.Context(), -1)
+	if err != nil {
+		return err
+	}
+	e := concurrency.NewElection(s, string(req.Name))
+	ch := e.Observe(stream.Context())
+	for stream.Context().Err() == nil {
+		select {
+		case <-stream.Context().Done():
+		case resp, ok := <-ch:
+			if !ok {
+				return nil
+			}
+			lresp := &epb.LeaderResponse{Header: resp.Header, Kv: resp.Kvs[0]}
+			if err := stream.Send(lresp); err != nil {
+				return err
+			}
+		}
+	}
+	return stream.Context().Err()
+}
+
+func (es *electionServer) Leader(ctx context.Context, req *epb.LeaderRequest) (*epb.LeaderResponse, error) {
+	s, err := es.session(ctx, -1)
+	if err != nil {
+		return nil, err
+	}
+	l, lerr := concurrency.NewElection(s, string(req.Name)).Leader(ctx)
+	if lerr != nil {
+		return nil, lerr
+	}
+	return &epb.LeaderResponse{Header: l.Header, Kv: l.Kvs[0]}, nil
+}
+
+func (es *electionServer) Resign(ctx context.Context, req *epb.ResignRequest) (*epb.ResignResponse, error) {
+	if req.Leader == nil {
+		return nil, ErrMissingLeaderKey
+	}
+	s, err := es.session(ctx, req.Leader.Lease)
+	if err != nil {
+		return nil, err
+	}
+	e := concurrency.ResumeElection(s, string(req.Leader.Name), string(req.Leader.Key), req.Leader.Rev)
+	if err := e.Resign(ctx); err != nil {
+		return nil, err
+	}
+	return &epb.ResignResponse{Header: e.Header()}, nil
+}
+
+func (es *electionServer) session(ctx context.Context, lease int64) (*concurrency.Session, error) {
+	s, err := concurrency.NewSession(
+		es.c,
+		concurrency.WithLease(clientv3.LeaseID(lease)),
+		concurrency.WithContext(ctx),
+	)
+	if err != nil {
+		return nil, err
+	}
+	s.Orphan()
+	return s, nil
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3election/v3electionpb/gw/v3election.pb.gw.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3election/v3electionpb/gw/v3election.pb.gw.go
new file mode 100644
index 0000000..23551b5
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3election/v3electionpb/gw/v3election.pb.gw.go
@@ -0,0 +1,313 @@
+// Code generated by protoc-gen-grpc-gateway. DO NOT EDIT.
+// source: etcdserver/api/v3election/v3electionpb/v3election.proto
+
+/*
+Package v3electionpb is a reverse proxy.
+
+It translates gRPC into RESTful JSON APIs.
+*/
+package gw
+
+import (
+	"go.etcd.io/etcd/etcdserver/api/v3election/v3electionpb"
+	"io"
+	"net/http"
+
+	"github.com/golang/protobuf/proto"
+	"github.com/grpc-ecosystem/grpc-gateway/runtime"
+	"github.com/grpc-ecosystem/grpc-gateway/utilities"
+	"golang.org/x/net/context"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/grpclog"
+	"google.golang.org/grpc/status"
+)
+
+var _ codes.Code
+var _ io.Reader
+var _ status.Status
+var _ = runtime.String
+var _ = utilities.NewDoubleArray
+
+func request_Election_Campaign_0(ctx context.Context, marshaler runtime.Marshaler, client v3electionpb.ElectionClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq v3electionpb.CampaignRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.Campaign(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Election_Proclaim_0(ctx context.Context, marshaler runtime.Marshaler, client v3electionpb.ElectionClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq v3electionpb.ProclaimRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.Proclaim(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Election_Leader_0(ctx context.Context, marshaler runtime.Marshaler, client v3electionpb.ElectionClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq v3electionpb.LeaderRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.Leader(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Election_Observe_0(ctx context.Context, marshaler runtime.Marshaler, client v3electionpb.ElectionClient, req *http.Request, pathParams map[string]string) (v3electionpb.Election_ObserveClient, runtime.ServerMetadata, error) {
+	var protoReq v3electionpb.LeaderRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	stream, err := client.Observe(ctx, &protoReq)
+	if err != nil {
+		return nil, metadata, err
+	}
+	header, err := stream.Header()
+	if err != nil {
+		return nil, metadata, err
+	}
+	metadata.HeaderMD = header
+	return stream, metadata, nil
+
+}
+
+func request_Election_Resign_0(ctx context.Context, marshaler runtime.Marshaler, client v3electionpb.ElectionClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq v3electionpb.ResignRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.Resign(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+// RegisterElectionHandlerFromEndpoint is same as RegisterElectionHandler but
+// automatically dials to "endpoint" and closes the connection when "ctx" gets done.
+func RegisterElectionHandlerFromEndpoint(ctx context.Context, mux *runtime.ServeMux, endpoint string, opts []grpc.DialOption) (err error) {
+	conn, err := grpc.Dial(endpoint, opts...)
+	if err != nil {
+		return err
+	}
+	defer func() {
+		if err != nil {
+			if cerr := conn.Close(); cerr != nil {
+				grpclog.Printf("Failed to close conn to %s: %v", endpoint, cerr)
+			}
+			return
+		}
+		go func() {
+			<-ctx.Done()
+			if cerr := conn.Close(); cerr != nil {
+				grpclog.Printf("Failed to close conn to %s: %v", endpoint, cerr)
+			}
+		}()
+	}()
+
+	return RegisterElectionHandler(ctx, mux, conn)
+}
+
+// RegisterElectionHandler registers the http handlers for service Election to "mux".
+// The handlers forward requests to the grpc endpoint over "conn".
+func RegisterElectionHandler(ctx context.Context, mux *runtime.ServeMux, conn *grpc.ClientConn) error {
+	return RegisterElectionHandlerClient(ctx, mux, v3electionpb.NewElectionClient(conn))
+}
+
+// RegisterElectionHandler registers the http handlers for service Election to "mux".
+// The handlers forward requests to the grpc endpoint over the given implementation of "ElectionClient".
+// Note: the gRPC framework executes interceptors within the gRPC handler. If the passed in "ElectionClient"
+// doesn't go through the normal gRPC flow (creating a gRPC client etc.) then it will be up to the passed in
+// "ElectionClient" to call the correct interceptors.
+func RegisterElectionHandlerClient(ctx context.Context, mux *runtime.ServeMux, client v3electionpb.ElectionClient) error {
+
+	mux.Handle("POST", pattern_Election_Campaign_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Election_Campaign_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Election_Campaign_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Election_Proclaim_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Election_Proclaim_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Election_Proclaim_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Election_Leader_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Election_Leader_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Election_Leader_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Election_Observe_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Election_Observe_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Election_Observe_0(ctx, mux, outboundMarshaler, w, req, func() (proto.Message, error) { return resp.Recv() }, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Election_Resign_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Election_Resign_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Election_Resign_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	return nil
+}
+
+var (
+	pattern_Election_Campaign_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "election", "campaign"}, ""))
+
+	pattern_Election_Proclaim_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "election", "proclaim"}, ""))
+
+	pattern_Election_Leader_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "election", "leader"}, ""))
+
+	pattern_Election_Observe_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "election", "observe"}, ""))
+
+	pattern_Election_Resign_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "election", "resign"}, ""))
+)
+
+var (
+	forward_Election_Campaign_0 = runtime.ForwardResponseMessage
+
+	forward_Election_Proclaim_0 = runtime.ForwardResponseMessage
+
+	forward_Election_Leader_0 = runtime.ForwardResponseMessage
+
+	forward_Election_Observe_0 = runtime.ForwardResponseStream
+
+	forward_Election_Resign_0 = runtime.ForwardResponseMessage
+)
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3election/v3electionpb/v3election.pb.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3election/v3electionpb/v3election.pb.go
new file mode 100644
index 0000000..1fc1bce
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3election/v3electionpb/v3election.pb.go
@@ -0,0 +1,2079 @@
+// Code generated by protoc-gen-gogo. DO NOT EDIT.
+// source: v3election.proto
+
+/*
+	Package v3electionpb is a generated protocol buffer package.
+
+	It is generated from these files:
+		v3election.proto
+
+	It has these top-level messages:
+		CampaignRequest
+		CampaignResponse
+		LeaderKey
+		LeaderRequest
+		LeaderResponse
+		ResignRequest
+		ResignResponse
+		ProclaimRequest
+		ProclaimResponse
+*/
+package v3electionpb
+
+import (
+	"fmt"
+
+	proto "github.com/golang/protobuf/proto"
+
+	math "math"
+
+	_ "github.com/gogo/protobuf/gogoproto"
+
+	etcdserverpb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+
+	mvccpb "go.etcd.io/etcd/mvcc/mvccpb"
+
+	context "golang.org/x/net/context"
+
+	grpc "google.golang.org/grpc"
+
+	io "io"
+)
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
+
+type CampaignRequest struct {
+	// name is the election's identifier for the campaign.
+	Name []byte `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	// lease is the ID of the lease attached to leadership of the election. If the
+	// lease expires or is revoked before resigning leadership, then the
+	// leadership is transferred to the next campaigner, if any.
+	Lease int64 `protobuf:"varint,2,opt,name=lease,proto3" json:"lease,omitempty"`
+	// value is the initial proclaimed value set when the campaigner wins the
+	// election.
+	Value []byte `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"`
+}
+
+func (m *CampaignRequest) Reset()                    { *m = CampaignRequest{} }
+func (m *CampaignRequest) String() string            { return proto.CompactTextString(m) }
+func (*CampaignRequest) ProtoMessage()               {}
+func (*CampaignRequest) Descriptor() ([]byte, []int) { return fileDescriptorV3Election, []int{0} }
+
+func (m *CampaignRequest) GetName() []byte {
+	if m != nil {
+		return m.Name
+	}
+	return nil
+}
+
+func (m *CampaignRequest) GetLease() int64 {
+	if m != nil {
+		return m.Lease
+	}
+	return 0
+}
+
+func (m *CampaignRequest) GetValue() []byte {
+	if m != nil {
+		return m.Value
+	}
+	return nil
+}
+
+type CampaignResponse struct {
+	Header *etcdserverpb.ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
+	// leader describes the resources used for holding leadereship of the election.
+	Leader *LeaderKey `protobuf:"bytes,2,opt,name=leader" json:"leader,omitempty"`
+}
+
+func (m *CampaignResponse) Reset()                    { *m = CampaignResponse{} }
+func (m *CampaignResponse) String() string            { return proto.CompactTextString(m) }
+func (*CampaignResponse) ProtoMessage()               {}
+func (*CampaignResponse) Descriptor() ([]byte, []int) { return fileDescriptorV3Election, []int{1} }
+
+func (m *CampaignResponse) GetHeader() *etcdserverpb.ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *CampaignResponse) GetLeader() *LeaderKey {
+	if m != nil {
+		return m.Leader
+	}
+	return nil
+}
+
+type LeaderKey struct {
+	// name is the election identifier that correponds to the leadership key.
+	Name []byte `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	// key is an opaque key representing the ownership of the election. If the key
+	// is deleted, then leadership is lost.
+	Key []byte `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"`
+	// rev is the creation revision of the key. It can be used to test for ownership
+	// of an election during transactions by testing the key's creation revision
+	// matches rev.
+	Rev int64 `protobuf:"varint,3,opt,name=rev,proto3" json:"rev,omitempty"`
+	// lease is the lease ID of the election leader.
+	Lease int64 `protobuf:"varint,4,opt,name=lease,proto3" json:"lease,omitempty"`
+}
+
+func (m *LeaderKey) Reset()                    { *m = LeaderKey{} }
+func (m *LeaderKey) String() string            { return proto.CompactTextString(m) }
+func (*LeaderKey) ProtoMessage()               {}
+func (*LeaderKey) Descriptor() ([]byte, []int) { return fileDescriptorV3Election, []int{2} }
+
+func (m *LeaderKey) GetName() []byte {
+	if m != nil {
+		return m.Name
+	}
+	return nil
+}
+
+func (m *LeaderKey) GetKey() []byte {
+	if m != nil {
+		return m.Key
+	}
+	return nil
+}
+
+func (m *LeaderKey) GetRev() int64 {
+	if m != nil {
+		return m.Rev
+	}
+	return 0
+}
+
+func (m *LeaderKey) GetLease() int64 {
+	if m != nil {
+		return m.Lease
+	}
+	return 0
+}
+
+type LeaderRequest struct {
+	// name is the election identifier for the leadership information.
+	Name []byte `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+}
+
+func (m *LeaderRequest) Reset()                    { *m = LeaderRequest{} }
+func (m *LeaderRequest) String() string            { return proto.CompactTextString(m) }
+func (*LeaderRequest) ProtoMessage()               {}
+func (*LeaderRequest) Descriptor() ([]byte, []int) { return fileDescriptorV3Election, []int{3} }
+
+func (m *LeaderRequest) GetName() []byte {
+	if m != nil {
+		return m.Name
+	}
+	return nil
+}
+
+type LeaderResponse struct {
+	Header *etcdserverpb.ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
+	// kv is the key-value pair representing the latest leader update.
+	Kv *mvccpb.KeyValue `protobuf:"bytes,2,opt,name=kv" json:"kv,omitempty"`
+}
+
+func (m *LeaderResponse) Reset()                    { *m = LeaderResponse{} }
+func (m *LeaderResponse) String() string            { return proto.CompactTextString(m) }
+func (*LeaderResponse) ProtoMessage()               {}
+func (*LeaderResponse) Descriptor() ([]byte, []int) { return fileDescriptorV3Election, []int{4} }
+
+func (m *LeaderResponse) GetHeader() *etcdserverpb.ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *LeaderResponse) GetKv() *mvccpb.KeyValue {
+	if m != nil {
+		return m.Kv
+	}
+	return nil
+}
+
+type ResignRequest struct {
+	// leader is the leadership to relinquish by resignation.
+	Leader *LeaderKey `protobuf:"bytes,1,opt,name=leader" json:"leader,omitempty"`
+}
+
+func (m *ResignRequest) Reset()                    { *m = ResignRequest{} }
+func (m *ResignRequest) String() string            { return proto.CompactTextString(m) }
+func (*ResignRequest) ProtoMessage()               {}
+func (*ResignRequest) Descriptor() ([]byte, []int) { return fileDescriptorV3Election, []int{5} }
+
+func (m *ResignRequest) GetLeader() *LeaderKey {
+	if m != nil {
+		return m.Leader
+	}
+	return nil
+}
+
+type ResignResponse struct {
+	Header *etcdserverpb.ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
+}
+
+func (m *ResignResponse) Reset()                    { *m = ResignResponse{} }
+func (m *ResignResponse) String() string            { return proto.CompactTextString(m) }
+func (*ResignResponse) ProtoMessage()               {}
+func (*ResignResponse) Descriptor() ([]byte, []int) { return fileDescriptorV3Election, []int{6} }
+
+func (m *ResignResponse) GetHeader() *etcdserverpb.ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+type ProclaimRequest struct {
+	// leader is the leadership hold on the election.
+	Leader *LeaderKey `protobuf:"bytes,1,opt,name=leader" json:"leader,omitempty"`
+	// value is an update meant to overwrite the leader's current value.
+	Value []byte `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
+}
+
+func (m *ProclaimRequest) Reset()                    { *m = ProclaimRequest{} }
+func (m *ProclaimRequest) String() string            { return proto.CompactTextString(m) }
+func (*ProclaimRequest) ProtoMessage()               {}
+func (*ProclaimRequest) Descriptor() ([]byte, []int) { return fileDescriptorV3Election, []int{7} }
+
+func (m *ProclaimRequest) GetLeader() *LeaderKey {
+	if m != nil {
+		return m.Leader
+	}
+	return nil
+}
+
+func (m *ProclaimRequest) GetValue() []byte {
+	if m != nil {
+		return m.Value
+	}
+	return nil
+}
+
+type ProclaimResponse struct {
+	Header *etcdserverpb.ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
+}
+
+func (m *ProclaimResponse) Reset()                    { *m = ProclaimResponse{} }
+func (m *ProclaimResponse) String() string            { return proto.CompactTextString(m) }
+func (*ProclaimResponse) ProtoMessage()               {}
+func (*ProclaimResponse) Descriptor() ([]byte, []int) { return fileDescriptorV3Election, []int{8} }
+
+func (m *ProclaimResponse) GetHeader() *etcdserverpb.ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func init() {
+	proto.RegisterType((*CampaignRequest)(nil), "v3electionpb.CampaignRequest")
+	proto.RegisterType((*CampaignResponse)(nil), "v3electionpb.CampaignResponse")
+	proto.RegisterType((*LeaderKey)(nil), "v3electionpb.LeaderKey")
+	proto.RegisterType((*LeaderRequest)(nil), "v3electionpb.LeaderRequest")
+	proto.RegisterType((*LeaderResponse)(nil), "v3electionpb.LeaderResponse")
+	proto.RegisterType((*ResignRequest)(nil), "v3electionpb.ResignRequest")
+	proto.RegisterType((*ResignResponse)(nil), "v3electionpb.ResignResponse")
+	proto.RegisterType((*ProclaimRequest)(nil), "v3electionpb.ProclaimRequest")
+	proto.RegisterType((*ProclaimResponse)(nil), "v3electionpb.ProclaimResponse")
+}
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ context.Context
+var _ grpc.ClientConn
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the grpc package it is being compiled against.
+const _ = grpc.SupportPackageIsVersion4
+
+// Client API for Election service
+
+type ElectionClient interface {
+	// Campaign waits to acquire leadership in an election, returning a LeaderKey
+	// representing the leadership if successful. The LeaderKey can then be used
+	// to issue new values on the election, transactionally guard API requests on
+	// leadership still being held, and resign from the election.
+	Campaign(ctx context.Context, in *CampaignRequest, opts ...grpc.CallOption) (*CampaignResponse, error)
+	// Proclaim updates the leader's posted value with a new value.
+	Proclaim(ctx context.Context, in *ProclaimRequest, opts ...grpc.CallOption) (*ProclaimResponse, error)
+	// Leader returns the current election proclamation, if any.
+	Leader(ctx context.Context, in *LeaderRequest, opts ...grpc.CallOption) (*LeaderResponse, error)
+	// Observe streams election proclamations in-order as made by the election's
+	// elected leaders.
+	Observe(ctx context.Context, in *LeaderRequest, opts ...grpc.CallOption) (Election_ObserveClient, error)
+	// Resign releases election leadership so other campaigners may acquire
+	// leadership on the election.
+	Resign(ctx context.Context, in *ResignRequest, opts ...grpc.CallOption) (*ResignResponse, error)
+}
+
+type electionClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewElectionClient(cc *grpc.ClientConn) ElectionClient {
+	return &electionClient{cc}
+}
+
+func (c *electionClient) Campaign(ctx context.Context, in *CampaignRequest, opts ...grpc.CallOption) (*CampaignResponse, error) {
+	out := new(CampaignResponse)
+	err := grpc.Invoke(ctx, "/v3electionpb.Election/Campaign", in, out, c.cc, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *electionClient) Proclaim(ctx context.Context, in *ProclaimRequest, opts ...grpc.CallOption) (*ProclaimResponse, error) {
+	out := new(ProclaimResponse)
+	err := grpc.Invoke(ctx, "/v3electionpb.Election/Proclaim", in, out, c.cc, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *electionClient) Leader(ctx context.Context, in *LeaderRequest, opts ...grpc.CallOption) (*LeaderResponse, error) {
+	out := new(LeaderResponse)
+	err := grpc.Invoke(ctx, "/v3electionpb.Election/Leader", in, out, c.cc, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *electionClient) Observe(ctx context.Context, in *LeaderRequest, opts ...grpc.CallOption) (Election_ObserveClient, error) {
+	stream, err := grpc.NewClientStream(ctx, &_Election_serviceDesc.Streams[0], c.cc, "/v3electionpb.Election/Observe", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &electionObserveClient{stream}
+	if err := x.ClientStream.SendMsg(in); err != nil {
+		return nil, err
+	}
+	if err := x.ClientStream.CloseSend(); err != nil {
+		return nil, err
+	}
+	return x, nil
+}
+
+type Election_ObserveClient interface {
+	Recv() (*LeaderResponse, error)
+	grpc.ClientStream
+}
+
+type electionObserveClient struct {
+	grpc.ClientStream
+}
+
+func (x *electionObserveClient) Recv() (*LeaderResponse, error) {
+	m := new(LeaderResponse)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+func (c *electionClient) Resign(ctx context.Context, in *ResignRequest, opts ...grpc.CallOption) (*ResignResponse, error) {
+	out := new(ResignResponse)
+	err := grpc.Invoke(ctx, "/v3electionpb.Election/Resign", in, out, c.cc, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+// Server API for Election service
+
+type ElectionServer interface {
+	// Campaign waits to acquire leadership in an election, returning a LeaderKey
+	// representing the leadership if successful. The LeaderKey can then be used
+	// to issue new values on the election, transactionally guard API requests on
+	// leadership still being held, and resign from the election.
+	Campaign(context.Context, *CampaignRequest) (*CampaignResponse, error)
+	// Proclaim updates the leader's posted value with a new value.
+	Proclaim(context.Context, *ProclaimRequest) (*ProclaimResponse, error)
+	// Leader returns the current election proclamation, if any.
+	Leader(context.Context, *LeaderRequest) (*LeaderResponse, error)
+	// Observe streams election proclamations in-order as made by the election's
+	// elected leaders.
+	Observe(*LeaderRequest, Election_ObserveServer) error
+	// Resign releases election leadership so other campaigners may acquire
+	// leadership on the election.
+	Resign(context.Context, *ResignRequest) (*ResignResponse, error)
+}
+
+func RegisterElectionServer(s *grpc.Server, srv ElectionServer) {
+	s.RegisterService(&_Election_serviceDesc, srv)
+}
+
+func _Election_Campaign_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(CampaignRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(ElectionServer).Campaign(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/v3electionpb.Election/Campaign",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(ElectionServer).Campaign(ctx, req.(*CampaignRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Election_Proclaim_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(ProclaimRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(ElectionServer).Proclaim(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/v3electionpb.Election/Proclaim",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(ElectionServer).Proclaim(ctx, req.(*ProclaimRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Election_Leader_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(LeaderRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(ElectionServer).Leader(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/v3electionpb.Election/Leader",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(ElectionServer).Leader(ctx, req.(*LeaderRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Election_Observe_Handler(srv interface{}, stream grpc.ServerStream) error {
+	m := new(LeaderRequest)
+	if err := stream.RecvMsg(m); err != nil {
+		return err
+	}
+	return srv.(ElectionServer).Observe(m, &electionObserveServer{stream})
+}
+
+type Election_ObserveServer interface {
+	Send(*LeaderResponse) error
+	grpc.ServerStream
+}
+
+type electionObserveServer struct {
+	grpc.ServerStream
+}
+
+func (x *electionObserveServer) Send(m *LeaderResponse) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+func _Election_Resign_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(ResignRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(ElectionServer).Resign(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/v3electionpb.Election/Resign",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(ElectionServer).Resign(ctx, req.(*ResignRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+var _Election_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "v3electionpb.Election",
+	HandlerType: (*ElectionServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "Campaign",
+			Handler:    _Election_Campaign_Handler,
+		},
+		{
+			MethodName: "Proclaim",
+			Handler:    _Election_Proclaim_Handler,
+		},
+		{
+			MethodName: "Leader",
+			Handler:    _Election_Leader_Handler,
+		},
+		{
+			MethodName: "Resign",
+			Handler:    _Election_Resign_Handler,
+		},
+	},
+	Streams: []grpc.StreamDesc{
+		{
+			StreamName:    "Observe",
+			Handler:       _Election_Observe_Handler,
+			ServerStreams: true,
+		},
+	},
+	Metadata: "v3election.proto",
+}
+
+func (m *CampaignRequest) 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 *CampaignRequest) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if len(m.Name) > 0 {
+		dAtA[i] = 0xa
+		i++
+		i = encodeVarintV3Election(dAtA, i, uint64(len(m.Name)))
+		i += copy(dAtA[i:], m.Name)
+	}
+	if m.Lease != 0 {
+		dAtA[i] = 0x10
+		i++
+		i = encodeVarintV3Election(dAtA, i, uint64(m.Lease))
+	}
+	if len(m.Value) > 0 {
+		dAtA[i] = 0x1a
+		i++
+		i = encodeVarintV3Election(dAtA, i, uint64(len(m.Value)))
+		i += copy(dAtA[i:], m.Value)
+	}
+	return i, nil
+}
+
+func (m *CampaignResponse) 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 *CampaignResponse) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if m.Header != nil {
+		dAtA[i] = 0xa
+		i++
+		i = encodeVarintV3Election(dAtA, i, uint64(m.Header.Size()))
+		n1, err := m.Header.MarshalTo(dAtA[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n1
+	}
+	if m.Leader != nil {
+		dAtA[i] = 0x12
+		i++
+		i = encodeVarintV3Election(dAtA, i, uint64(m.Leader.Size()))
+		n2, err := m.Leader.MarshalTo(dAtA[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n2
+	}
+	return i, nil
+}
+
+func (m *LeaderKey) 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 *LeaderKey) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if len(m.Name) > 0 {
+		dAtA[i] = 0xa
+		i++
+		i = encodeVarintV3Election(dAtA, i, uint64(len(m.Name)))
+		i += copy(dAtA[i:], m.Name)
+	}
+	if len(m.Key) > 0 {
+		dAtA[i] = 0x12
+		i++
+		i = encodeVarintV3Election(dAtA, i, uint64(len(m.Key)))
+		i += copy(dAtA[i:], m.Key)
+	}
+	if m.Rev != 0 {
+		dAtA[i] = 0x18
+		i++
+		i = encodeVarintV3Election(dAtA, i, uint64(m.Rev))
+	}
+	if m.Lease != 0 {
+		dAtA[i] = 0x20
+		i++
+		i = encodeVarintV3Election(dAtA, i, uint64(m.Lease))
+	}
+	return i, nil
+}
+
+func (m *LeaderRequest) 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 *LeaderRequest) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if len(m.Name) > 0 {
+		dAtA[i] = 0xa
+		i++
+		i = encodeVarintV3Election(dAtA, i, uint64(len(m.Name)))
+		i += copy(dAtA[i:], m.Name)
+	}
+	return i, nil
+}
+
+func (m *LeaderResponse) 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 *LeaderResponse) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if m.Header != nil {
+		dAtA[i] = 0xa
+		i++
+		i = encodeVarintV3Election(dAtA, i, uint64(m.Header.Size()))
+		n3, err := m.Header.MarshalTo(dAtA[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n3
+	}
+	if m.Kv != nil {
+		dAtA[i] = 0x12
+		i++
+		i = encodeVarintV3Election(dAtA, i, uint64(m.Kv.Size()))
+		n4, err := m.Kv.MarshalTo(dAtA[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n4
+	}
+	return i, nil
+}
+
+func (m *ResignRequest) 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 *ResignRequest) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if m.Leader != nil {
+		dAtA[i] = 0xa
+		i++
+		i = encodeVarintV3Election(dAtA, i, uint64(m.Leader.Size()))
+		n5, err := m.Leader.MarshalTo(dAtA[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n5
+	}
+	return i, nil
+}
+
+func (m *ResignResponse) 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 *ResignResponse) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if m.Header != nil {
+		dAtA[i] = 0xa
+		i++
+		i = encodeVarintV3Election(dAtA, i, uint64(m.Header.Size()))
+		n6, err := m.Header.MarshalTo(dAtA[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n6
+	}
+	return i, nil
+}
+
+func (m *ProclaimRequest) 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 *ProclaimRequest) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if m.Leader != nil {
+		dAtA[i] = 0xa
+		i++
+		i = encodeVarintV3Election(dAtA, i, uint64(m.Leader.Size()))
+		n7, err := m.Leader.MarshalTo(dAtA[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n7
+	}
+	if len(m.Value) > 0 {
+		dAtA[i] = 0x12
+		i++
+		i = encodeVarintV3Election(dAtA, i, uint64(len(m.Value)))
+		i += copy(dAtA[i:], m.Value)
+	}
+	return i, nil
+}
+
+func (m *ProclaimResponse) 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 *ProclaimResponse) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if m.Header != nil {
+		dAtA[i] = 0xa
+		i++
+		i = encodeVarintV3Election(dAtA, i, uint64(m.Header.Size()))
+		n8, err := m.Header.MarshalTo(dAtA[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n8
+	}
+	return i, nil
+}
+
+func encodeVarintV3Election(dAtA []byte, offset int, v uint64) int {
+	for v >= 1<<7 {
+		dAtA[offset] = uint8(v&0x7f | 0x80)
+		v >>= 7
+		offset++
+	}
+	dAtA[offset] = uint8(v)
+	return offset + 1
+}
+func (m *CampaignRequest) Size() (n int) {
+	var l int
+	_ = l
+	l = len(m.Name)
+	if l > 0 {
+		n += 1 + l + sovV3Election(uint64(l))
+	}
+	if m.Lease != 0 {
+		n += 1 + sovV3Election(uint64(m.Lease))
+	}
+	l = len(m.Value)
+	if l > 0 {
+		n += 1 + l + sovV3Election(uint64(l))
+	}
+	return n
+}
+
+func (m *CampaignResponse) Size() (n int) {
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovV3Election(uint64(l))
+	}
+	if m.Leader != nil {
+		l = m.Leader.Size()
+		n += 1 + l + sovV3Election(uint64(l))
+	}
+	return n
+}
+
+func (m *LeaderKey) Size() (n int) {
+	var l int
+	_ = l
+	l = len(m.Name)
+	if l > 0 {
+		n += 1 + l + sovV3Election(uint64(l))
+	}
+	l = len(m.Key)
+	if l > 0 {
+		n += 1 + l + sovV3Election(uint64(l))
+	}
+	if m.Rev != 0 {
+		n += 1 + sovV3Election(uint64(m.Rev))
+	}
+	if m.Lease != 0 {
+		n += 1 + sovV3Election(uint64(m.Lease))
+	}
+	return n
+}
+
+func (m *LeaderRequest) Size() (n int) {
+	var l int
+	_ = l
+	l = len(m.Name)
+	if l > 0 {
+		n += 1 + l + sovV3Election(uint64(l))
+	}
+	return n
+}
+
+func (m *LeaderResponse) Size() (n int) {
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovV3Election(uint64(l))
+	}
+	if m.Kv != nil {
+		l = m.Kv.Size()
+		n += 1 + l + sovV3Election(uint64(l))
+	}
+	return n
+}
+
+func (m *ResignRequest) Size() (n int) {
+	var l int
+	_ = l
+	if m.Leader != nil {
+		l = m.Leader.Size()
+		n += 1 + l + sovV3Election(uint64(l))
+	}
+	return n
+}
+
+func (m *ResignResponse) Size() (n int) {
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovV3Election(uint64(l))
+	}
+	return n
+}
+
+func (m *ProclaimRequest) Size() (n int) {
+	var l int
+	_ = l
+	if m.Leader != nil {
+		l = m.Leader.Size()
+		n += 1 + l + sovV3Election(uint64(l))
+	}
+	l = len(m.Value)
+	if l > 0 {
+		n += 1 + l + sovV3Election(uint64(l))
+	}
+	return n
+}
+
+func (m *ProclaimResponse) Size() (n int) {
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovV3Election(uint64(l))
+	}
+	return n
+}
+
+func sovV3Election(x uint64) (n int) {
+	for {
+		n++
+		x >>= 7
+		if x == 0 {
+			break
+		}
+	}
+	return n
+}
+func sozV3Election(x uint64) (n int) {
+	return sovV3Election(uint64((x << 1) ^ uint64((int64(x) >> 63))))
+}
+func (m *CampaignRequest) 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 ErrIntOverflowV3Election
+			}
+			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: CampaignRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: CampaignRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Name = append(m.Name[:0], dAtA[iNdEx:postIndex]...)
+			if m.Name == nil {
+				m.Name = []byte{}
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Lease", wireType)
+			}
+			m.Lease = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Lease |= (int64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Value = append(m.Value[:0], dAtA[iNdEx:postIndex]...)
+			if m.Value == nil {
+				m.Value = []byte{}
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipV3Election(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *CampaignResponse) 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 ErrIntOverflowV3Election
+			}
+			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: CampaignResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: CampaignResponse: 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 ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &etcdserverpb.ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Leader", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Leader == nil {
+				m.Leader = &LeaderKey{}
+			}
+			if err := m.Leader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipV3Election(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *LeaderKey) 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 ErrIntOverflowV3Election
+			}
+			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: LeaderKey: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LeaderKey: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Name = append(m.Name[:0], dAtA[iNdEx:postIndex]...)
+			if m.Name == nil {
+				m.Name = []byte{}
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...)
+			if m.Key == nil {
+				m.Key = []byte{}
+			}
+			iNdEx = postIndex
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Rev", wireType)
+			}
+			m.Rev = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Rev |= (int64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 4:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Lease", wireType)
+			}
+			m.Lease = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Lease |= (int64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipV3Election(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *LeaderRequest) 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 ErrIntOverflowV3Election
+			}
+			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: LeaderRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LeaderRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Name = append(m.Name[:0], dAtA[iNdEx:postIndex]...)
+			if m.Name == nil {
+				m.Name = []byte{}
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipV3Election(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *LeaderResponse) 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 ErrIntOverflowV3Election
+			}
+			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: LeaderResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LeaderResponse: 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 ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &etcdserverpb.ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Kv", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Kv == nil {
+				m.Kv = &mvccpb.KeyValue{}
+			}
+			if err := m.Kv.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipV3Election(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *ResignRequest) 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 ErrIntOverflowV3Election
+			}
+			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: ResignRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: ResignRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Leader", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Leader == nil {
+				m.Leader = &LeaderKey{}
+			}
+			if err := m.Leader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipV3Election(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *ResignResponse) 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 ErrIntOverflowV3Election
+			}
+			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: ResignResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: ResignResponse: 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 ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &etcdserverpb.ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipV3Election(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *ProclaimRequest) 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 ErrIntOverflowV3Election
+			}
+			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: ProclaimRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: ProclaimRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Leader", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Leader == nil {
+				m.Leader = &LeaderKey{}
+			}
+			if err := m.Leader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Value = append(m.Value[:0], dAtA[iNdEx:postIndex]...)
+			if m.Value == nil {
+				m.Value = []byte{}
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipV3Election(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *ProclaimResponse) 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 ErrIntOverflowV3Election
+			}
+			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: ProclaimResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: ProclaimResponse: 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 ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &etcdserverpb.ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipV3Election(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthV3Election
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func skipV3Election(dAtA []byte) (n int, err error) {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return 0, ErrIntOverflowV3Election
+			}
+			if iNdEx >= l {
+				return 0, io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= (uint64(b) & 0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		wireType := int(wire & 0x7)
+		switch wireType {
+		case 0:
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return 0, ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return 0, io.ErrUnexpectedEOF
+				}
+				iNdEx++
+				if dAtA[iNdEx-1] < 0x80 {
+					break
+				}
+			}
+			return iNdEx, nil
+		case 1:
+			iNdEx += 8
+			return iNdEx, nil
+		case 2:
+			var length int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return 0, ErrIntOverflowV3Election
+				}
+				if iNdEx >= l {
+					return 0, io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				length |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			iNdEx += length
+			if length < 0 {
+				return 0, ErrInvalidLengthV3Election
+			}
+			return iNdEx, nil
+		case 3:
+			for {
+				var innerWire uint64
+				var start int = iNdEx
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return 0, ErrIntOverflowV3Election
+					}
+					if iNdEx >= l {
+						return 0, io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					innerWire |= (uint64(b) & 0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				innerWireType := int(innerWire & 0x7)
+				if innerWireType == 4 {
+					break
+				}
+				next, err := skipV3Election(dAtA[start:])
+				if err != nil {
+					return 0, err
+				}
+				iNdEx = start + next
+			}
+			return iNdEx, nil
+		case 4:
+			return iNdEx, nil
+		case 5:
+			iNdEx += 4
+			return iNdEx, nil
+		default:
+			return 0, fmt.Errorf("proto: illegal wireType %d", wireType)
+		}
+	}
+	panic("unreachable")
+}
+
+var (
+	ErrInvalidLengthV3Election = fmt.Errorf("proto: negative length found during unmarshaling")
+	ErrIntOverflowV3Election   = fmt.Errorf("proto: integer overflow")
+)
+
+func init() { proto.RegisterFile("v3election.proto", fileDescriptorV3Election) }
+
+var fileDescriptorV3Election = []byte{
+	// 535 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x94, 0xcf, 0x6e, 0xd3, 0x40,
+	0x10, 0xc6, 0x59, 0x27, 0x84, 0x32, 0xa4, 0xad, 0x65, 0x82, 0x48, 0x43, 0x30, 0xd1, 0x22, 0xa1,
+	0x2a, 0x07, 0x2f, 0x6a, 0x38, 0xe5, 0x84, 0x40, 0xa0, 0x4a, 0x45, 0x02, 0x7c, 0x40, 0x70, 0xdc,
+	0xb8, 0x23, 0x37, 0x8a, 0xe3, 0x35, 0xb6, 0x6b, 0x29, 0x57, 0x5e, 0x81, 0x03, 0x3c, 0x12, 0x47,
+	0x24, 0x5e, 0x00, 0x05, 0x1e, 0x04, 0xed, 0xae, 0x8d, 0xff, 0x28, 0x41, 0xa8, 0xb9, 0x58, 0xe3,
+	0x9d, 0xcf, 0xf3, 0x9b, 0x6f, 0x76, 0x12, 0x30, 0xb3, 0x09, 0x06, 0xe8, 0xa5, 0x73, 0x11, 0x3a,
+	0x51, 0x2c, 0x52, 0x61, 0x75, 0xcb, 0x93, 0x68, 0x36, 0xe8, 0xf9, 0xc2, 0x17, 0x2a, 0xc1, 0x64,
+	0xa4, 0x35, 0x83, 0x47, 0x98, 0x7a, 0xe7, 0x4c, 0x3e, 0x12, 0x8c, 0x33, 0x8c, 0x2b, 0x61, 0x34,
+	0x63, 0x71, 0xe4, 0xe5, 0xba, 0x23, 0xa5, 0x5b, 0x66, 0x9e, 0xa7, 0x1e, 0xd1, 0x8c, 0x2d, 0xb2,
+	0x3c, 0x35, 0xf4, 0x85, 0xf0, 0x03, 0x64, 0x3c, 0x9a, 0x33, 0x1e, 0x86, 0x22, 0xe5, 0x92, 0x98,
+	0xe8, 0x2c, 0x7d, 0x0b, 0x87, 0xcf, 0xf9, 0x32, 0xe2, 0x73, 0x3f, 0x74, 0xf1, 0xe3, 0x25, 0x26,
+	0xa9, 0x65, 0x41, 0x3b, 0xe4, 0x4b, 0xec, 0x93, 0x11, 0x39, 0xee, 0xba, 0x2a, 0xb6, 0x7a, 0x70,
+	0x3d, 0x40, 0x9e, 0x60, 0xdf, 0x18, 0x91, 0xe3, 0x96, 0xab, 0x5f, 0xe4, 0x69, 0xc6, 0x83, 0x4b,
+	0xec, 0xb7, 0x94, 0x54, 0xbf, 0xd0, 0x15, 0x98, 0x65, 0xc9, 0x24, 0x12, 0x61, 0x82, 0xd6, 0x13,
+	0xe8, 0x5c, 0x20, 0x3f, 0xc7, 0x58, 0x55, 0xbd, 0x75, 0x32, 0x74, 0xaa, 0x46, 0x9c, 0x42, 0x77,
+	0xaa, 0x34, 0x6e, 0xae, 0xb5, 0x18, 0x74, 0x02, 0xfd, 0x95, 0xa1, 0xbe, 0xba, 0xeb, 0x54, 0x47,
+	0xe6, 0xbc, 0x52, 0xb9, 0x33, 0x5c, 0xb9, 0xb9, 0x8c, 0x7e, 0x80, 0x9b, 0x7f, 0x0f, 0x37, 0xfa,
+	0x30, 0xa1, 0xb5, 0xc0, 0x95, 0x2a, 0xd7, 0x75, 0x65, 0x28, 0x4f, 0x62, 0xcc, 0x94, 0x83, 0x96,
+	0x2b, 0xc3, 0xd2, 0x6b, 0xbb, 0xe2, 0x95, 0x3e, 0x84, 0x7d, 0x5d, 0xfa, 0x1f, 0x63, 0xa2, 0x17,
+	0x70, 0x50, 0x88, 0x76, 0x32, 0x3e, 0x02, 0x63, 0x91, 0xe5, 0xa6, 0x4d, 0x47, 0xdf, 0xa8, 0x73,
+	0x86, 0xab, 0x77, 0x72, 0xc0, 0xae, 0xb1, 0xc8, 0xe8, 0x53, 0xd8, 0x77, 0x31, 0xa9, 0xdc, 0x5a,
+	0x39, 0x2b, 0xf2, 0x7f, 0xb3, 0x7a, 0x09, 0x07, 0x45, 0x85, 0x5d, 0x7a, 0xa5, 0xef, 0xe1, 0xf0,
+	0x4d, 0x2c, 0xbc, 0x80, 0xcf, 0x97, 0x57, 0xed, 0xa5, 0x5c, 0x24, 0xa3, 0xba, 0x48, 0xa7, 0x60,
+	0x96, 0x95, 0x77, 0xe9, 0xf1, 0xe4, 0x4b, 0x1b, 0xf6, 0x5e, 0xe4, 0x0d, 0x58, 0x0b, 0xd8, 0x2b,
+	0xf6, 0xd3, 0xba, 0x5f, 0xef, 0xac, 0xf1, 0x53, 0x18, 0xd8, 0xdb, 0xd2, 0x9a, 0x42, 0x47, 0x9f,
+	0x7e, 0xfc, 0xfe, 0x6c, 0x0c, 0xe8, 0x1d, 0x96, 0x4d, 0x58, 0x21, 0x64, 0x5e, 0x2e, 0x9b, 0x92,
+	0xb1, 0x84, 0x15, 0x1e, 0x9a, 0xb0, 0xc6, 0xd4, 0x9a, 0xb0, 0xa6, 0xf5, 0x2d, 0xb0, 0x28, 0x97,
+	0x49, 0x98, 0x07, 0x1d, 0x3d, 0x5b, 0xeb, 0xde, 0xa6, 0x89, 0x17, 0xa0, 0xe1, 0xe6, 0x64, 0x8e,
+	0xb1, 0x15, 0xa6, 0x4f, 0x6f, 0xd7, 0x30, 0xfa, 0xa2, 0x24, 0xc4, 0x87, 0x1b, 0xaf, 0x67, 0x6a,
+	0xe0, 0xbb, 0x50, 0x1e, 0x28, 0xca, 0x11, 0xed, 0xd5, 0x28, 0x42, 0x17, 0x9e, 0x92, 0xf1, 0x63,
+	0x22, 0xdd, 0xe8, 0x05, 0x6d, 0x72, 0x6a, 0x8b, 0xdf, 0xe4, 0xd4, 0x77, 0x7a, 0x8b, 0x9b, 0x58,
+	0x89, 0xa6, 0x64, 0xfc, 0xcc, 0xfc, 0xb6, 0xb6, 0xc9, 0xf7, 0xb5, 0x4d, 0x7e, 0xae, 0x6d, 0xf2,
+	0xf5, 0x97, 0x7d, 0x6d, 0xd6, 0x51, 0x7f, 0x8c, 0x93, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x2f,
+	0x1d, 0xfa, 0x11, 0xb1, 0x05, 0x00, 0x00,
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3election/v3electionpb/v3election.proto b/vendor/go.etcd.io/etcd/etcdserver/api/v3election/v3electionpb/v3election.proto
new file mode 100644
index 0000000..918f39f
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3election/v3electionpb/v3election.proto
@@ -0,0 +1,119 @@
+syntax = "proto3";
+package v3electionpb;
+
+import "gogoproto/gogo.proto";
+import "etcd/etcdserver/etcdserverpb/rpc.proto";
+import "etcd/mvcc/mvccpb/kv.proto";
+
+// for grpc-gateway
+import "google/api/annotations.proto";
+
+option (gogoproto.marshaler_all) = true;
+option (gogoproto.unmarshaler_all) = true;
+
+// The election service exposes client-side election facilities as a gRPC interface.
+service Election {
+  // Campaign waits to acquire leadership in an election, returning a LeaderKey
+  // representing the leadership if successful. The LeaderKey can then be used
+  // to issue new values on the election, transactionally guard API requests on
+  // leadership still being held, and resign from the election.
+  rpc Campaign(CampaignRequest) returns (CampaignResponse) {
+      option (google.api.http) = {
+        post: "/v3/election/campaign"
+        body: "*"
+    };
+  }
+  // Proclaim updates the leader's posted value with a new value.
+  rpc Proclaim(ProclaimRequest) returns (ProclaimResponse) {
+      option (google.api.http) = {
+        post: "/v3/election/proclaim"
+        body: "*"
+    };
+  }
+  // Leader returns the current election proclamation, if any.
+  rpc Leader(LeaderRequest) returns (LeaderResponse) {
+      option (google.api.http) = {
+        post: "/v3/election/leader"
+        body: "*"
+    };
+  }
+  // Observe streams election proclamations in-order as made by the election's
+  // elected leaders.
+  rpc Observe(LeaderRequest) returns (stream LeaderResponse) {
+      option (google.api.http) = {
+        post: "/v3/election/observe"
+        body: "*"
+    };
+  }
+  // Resign releases election leadership so other campaigners may acquire
+  // leadership on the election.
+  rpc Resign(ResignRequest) returns (ResignResponse) {
+      option (google.api.http) = {
+        post: "/v3/election/resign"
+        body: "*"
+    };
+  }
+}
+
+message CampaignRequest {
+  // name is the election's identifier for the campaign.
+  bytes name = 1;
+  // lease is the ID of the lease attached to leadership of the election. If the
+  // lease expires or is revoked before resigning leadership, then the
+  // leadership is transferred to the next campaigner, if any.
+  int64 lease = 2;
+  // value is the initial proclaimed value set when the campaigner wins the
+  // election.
+  bytes value = 3;
+}
+
+message CampaignResponse {
+  etcdserverpb.ResponseHeader header = 1;
+  // leader describes the resources used for holding leadereship of the election.
+  LeaderKey leader = 2;
+}
+
+message LeaderKey {
+  // name is the election identifier that correponds to the leadership key.
+  bytes name = 1;
+  // key is an opaque key representing the ownership of the election. If the key
+  // is deleted, then leadership is lost.
+  bytes key = 2;
+  // rev is the creation revision of the key. It can be used to test for ownership
+  // of an election during transactions by testing the key's creation revision
+  // matches rev.
+  int64 rev = 3;
+  // lease is the lease ID of the election leader.
+  int64 lease = 4;
+}
+
+message LeaderRequest {
+  // name is the election identifier for the leadership information.
+  bytes name = 1;
+}
+
+message LeaderResponse {
+  etcdserverpb.ResponseHeader header = 1;
+  // kv is the key-value pair representing the latest leader update.
+  mvccpb.KeyValue kv = 2;
+}
+
+message ResignRequest {
+  // leader is the leadership to relinquish by resignation.
+  LeaderKey leader = 1;
+}
+
+message ResignResponse {
+  etcdserverpb.ResponseHeader header = 1;
+}
+
+message ProclaimRequest {
+  // leader is the leadership hold on the election.
+  LeaderKey leader = 1;
+  // value is an update meant to overwrite the leader's current value.
+  bytes value = 2;
+}
+
+message ProclaimResponse {
+  etcdserverpb.ResponseHeader header = 1;
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3lock/doc.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3lock/doc.go
new file mode 100644
index 0000000..e0a1008
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3lock/doc.go
@@ -0,0 +1,16 @@
+// 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 v3lock provides a v3 locking service from an etcdserver.
+package v3lock
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3lock/lock.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3lock/lock.go
new file mode 100644
index 0000000..5a17c86
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3lock/lock.go
@@ -0,0 +1,56 @@
+// 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 v3lock
+
+import (
+	"context"
+
+	"go.etcd.io/etcd/clientv3"
+	"go.etcd.io/etcd/clientv3/concurrency"
+	"go.etcd.io/etcd/etcdserver/api/v3lock/v3lockpb"
+)
+
+type lockServer struct {
+	c *clientv3.Client
+}
+
+func NewLockServer(c *clientv3.Client) v3lockpb.LockServer {
+	return &lockServer{c}
+}
+
+func (ls *lockServer) Lock(ctx context.Context, req *v3lockpb.LockRequest) (*v3lockpb.LockResponse, error) {
+	s, err := concurrency.NewSession(
+		ls.c,
+		concurrency.WithLease(clientv3.LeaseID(req.Lease)),
+		concurrency.WithContext(ctx),
+	)
+	if err != nil {
+		return nil, err
+	}
+	s.Orphan()
+	m := concurrency.NewMutex(s, string(req.Name))
+	if err = m.Lock(ctx); err != nil {
+		return nil, err
+	}
+	return &v3lockpb.LockResponse{Header: m.Header(), Key: []byte(m.Key())}, nil
+}
+
+func (ls *lockServer) Unlock(ctx context.Context, req *v3lockpb.UnlockRequest) (*v3lockpb.UnlockResponse, error) {
+	resp, err := ls.c.Delete(ctx, string(req.Key))
+	if err != nil {
+		return nil, err
+	}
+	return &v3lockpb.UnlockResponse{Header: resp.Header}, nil
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3lock/v3lockpb/gw/v3lock.pb.gw.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3lock/v3lockpb/gw/v3lock.pb.gw.go
new file mode 100644
index 0000000..1eeeff1
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3lock/v3lockpb/gw/v3lock.pb.gw.go
@@ -0,0 +1,167 @@
+// Code generated by protoc-gen-grpc-gateway. DO NOT EDIT.
+// source: etcdserver/api/v3lock/v3lockpb/v3lock.proto
+
+/*
+Package v3lockpb is a reverse proxy.
+
+It translates gRPC into RESTful JSON APIs.
+*/
+package gw
+
+import (
+	"go.etcd.io/etcd/etcdserver/api/v3lock/v3lockpb"
+	"io"
+	"net/http"
+
+	"github.com/golang/protobuf/proto"
+	"github.com/grpc-ecosystem/grpc-gateway/runtime"
+	"github.com/grpc-ecosystem/grpc-gateway/utilities"
+	"golang.org/x/net/context"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/grpclog"
+	"google.golang.org/grpc/status"
+)
+
+var _ codes.Code
+var _ io.Reader
+var _ status.Status
+var _ = runtime.String
+var _ = utilities.NewDoubleArray
+
+func request_Lock_Lock_0(ctx context.Context, marshaler runtime.Marshaler, client v3lockpb.LockClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq v3lockpb.LockRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.Lock(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Lock_Unlock_0(ctx context.Context, marshaler runtime.Marshaler, client v3lockpb.LockClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq v3lockpb.UnlockRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.Unlock(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+// RegisterLockHandlerFromEndpoint is same as RegisterLockHandler but
+// automatically dials to "endpoint" and closes the connection when "ctx" gets done.
+func RegisterLockHandlerFromEndpoint(ctx context.Context, mux *runtime.ServeMux, endpoint string, opts []grpc.DialOption) (err error) {
+	conn, err := grpc.Dial(endpoint, opts...)
+	if err != nil {
+		return err
+	}
+	defer func() {
+		if err != nil {
+			if cerr := conn.Close(); cerr != nil {
+				grpclog.Printf("Failed to close conn to %s: %v", endpoint, cerr)
+			}
+			return
+		}
+		go func() {
+			<-ctx.Done()
+			if cerr := conn.Close(); cerr != nil {
+				grpclog.Printf("Failed to close conn to %s: %v", endpoint, cerr)
+			}
+		}()
+	}()
+
+	return RegisterLockHandler(ctx, mux, conn)
+}
+
+// RegisterLockHandler registers the http handlers for service Lock to "mux".
+// The handlers forward requests to the grpc endpoint over "conn".
+func RegisterLockHandler(ctx context.Context, mux *runtime.ServeMux, conn *grpc.ClientConn) error {
+	return RegisterLockHandlerClient(ctx, mux, v3lockpb.NewLockClient(conn))
+}
+
+// RegisterLockHandler registers the http handlers for service Lock to "mux".
+// The handlers forward requests to the grpc endpoint over the given implementation of "LockClient".
+// Note: the gRPC framework executes interceptors within the gRPC handler. If the passed in "LockClient"
+// doesn't go through the normal gRPC flow (creating a gRPC client etc.) then it will be up to the passed in
+// "LockClient" to call the correct interceptors.
+func RegisterLockHandlerClient(ctx context.Context, mux *runtime.ServeMux, client v3lockpb.LockClient) error {
+
+	mux.Handle("POST", pattern_Lock_Lock_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Lock_Lock_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Lock_Lock_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Lock_Unlock_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Lock_Unlock_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Lock_Unlock_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	return nil
+}
+
+var (
+	pattern_Lock_Lock_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 1}, []string{"v3", "lock"}, ""))
+
+	pattern_Lock_Unlock_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "lock", "unlock"}, ""))
+)
+
+var (
+	forward_Lock_Lock_0 = runtime.ForwardResponseMessage
+
+	forward_Lock_Unlock_0 = runtime.ForwardResponseMessage
+)
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3lock/v3lockpb/v3lock.pb.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3lock/v3lockpb/v3lock.pb.go
new file mode 100644
index 0000000..36ebdd9
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3lock/v3lockpb/v3lock.pb.go
@@ -0,0 +1,959 @@
+// Code generated by protoc-gen-gogo. DO NOT EDIT.
+// source: v3lock.proto
+
+/*
+	Package v3lockpb is a generated protocol buffer package.
+
+	It is generated from these files:
+		v3lock.proto
+
+	It has these top-level messages:
+		LockRequest
+		LockResponse
+		UnlockRequest
+		UnlockResponse
+*/
+package v3lockpb
+
+import (
+	"fmt"
+
+	proto "github.com/golang/protobuf/proto"
+
+	math "math"
+
+	_ "github.com/gogo/protobuf/gogoproto"
+
+	etcdserverpb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+
+	context "golang.org/x/net/context"
+
+	grpc "google.golang.org/grpc"
+
+	io "io"
+)
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
+
+type LockRequest struct {
+	// name is the identifier for the distributed shared lock to be acquired.
+	Name []byte `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	// lease is the ID of the lease that will be attached to ownership of the
+	// lock. If the lease expires or is revoked and currently holds the lock,
+	// the lock is automatically released. Calls to Lock with the same lease will
+	// be treated as a single acquisition; locking twice with the same lease is a
+	// no-op.
+	Lease int64 `protobuf:"varint,2,opt,name=lease,proto3" json:"lease,omitempty"`
+}
+
+func (m *LockRequest) Reset()                    { *m = LockRequest{} }
+func (m *LockRequest) String() string            { return proto.CompactTextString(m) }
+func (*LockRequest) ProtoMessage()               {}
+func (*LockRequest) Descriptor() ([]byte, []int) { return fileDescriptorV3Lock, []int{0} }
+
+func (m *LockRequest) GetName() []byte {
+	if m != nil {
+		return m.Name
+	}
+	return nil
+}
+
+func (m *LockRequest) GetLease() int64 {
+	if m != nil {
+		return m.Lease
+	}
+	return 0
+}
+
+type LockResponse struct {
+	Header *etcdserverpb.ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
+	// key is a key that will exist on etcd for the duration that the Lock caller
+	// owns the lock. Users should not modify this key or the lock may exhibit
+	// undefined behavior.
+	Key []byte `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"`
+}
+
+func (m *LockResponse) Reset()                    { *m = LockResponse{} }
+func (m *LockResponse) String() string            { return proto.CompactTextString(m) }
+func (*LockResponse) ProtoMessage()               {}
+func (*LockResponse) Descriptor() ([]byte, []int) { return fileDescriptorV3Lock, []int{1} }
+
+func (m *LockResponse) GetHeader() *etcdserverpb.ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func (m *LockResponse) GetKey() []byte {
+	if m != nil {
+		return m.Key
+	}
+	return nil
+}
+
+type UnlockRequest struct {
+	// key is the lock ownership key granted by Lock.
+	Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
+}
+
+func (m *UnlockRequest) Reset()                    { *m = UnlockRequest{} }
+func (m *UnlockRequest) String() string            { return proto.CompactTextString(m) }
+func (*UnlockRequest) ProtoMessage()               {}
+func (*UnlockRequest) Descriptor() ([]byte, []int) { return fileDescriptorV3Lock, []int{2} }
+
+func (m *UnlockRequest) GetKey() []byte {
+	if m != nil {
+		return m.Key
+	}
+	return nil
+}
+
+type UnlockResponse struct {
+	Header *etcdserverpb.ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
+}
+
+func (m *UnlockResponse) Reset()                    { *m = UnlockResponse{} }
+func (m *UnlockResponse) String() string            { return proto.CompactTextString(m) }
+func (*UnlockResponse) ProtoMessage()               {}
+func (*UnlockResponse) Descriptor() ([]byte, []int) { return fileDescriptorV3Lock, []int{3} }
+
+func (m *UnlockResponse) GetHeader() *etcdserverpb.ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
+func init() {
+	proto.RegisterType((*LockRequest)(nil), "v3lockpb.LockRequest")
+	proto.RegisterType((*LockResponse)(nil), "v3lockpb.LockResponse")
+	proto.RegisterType((*UnlockRequest)(nil), "v3lockpb.UnlockRequest")
+	proto.RegisterType((*UnlockResponse)(nil), "v3lockpb.UnlockResponse")
+}
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ context.Context
+var _ grpc.ClientConn
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the grpc package it is being compiled against.
+const _ = grpc.SupportPackageIsVersion4
+
+// Client API for Lock service
+
+type LockClient interface {
+	// Lock acquires a distributed shared lock on a given named lock.
+	// On success, it will return a unique key that exists so long as the
+	// lock is held by the caller. This key can be used in conjunction with
+	// transactions to safely ensure updates to etcd only occur while holding
+	// lock ownership. The lock is held until Unlock is called on the key or the
+	// lease associate with the owner expires.
+	Lock(ctx context.Context, in *LockRequest, opts ...grpc.CallOption) (*LockResponse, error)
+	// Unlock takes a key returned by Lock and releases the hold on lock. The
+	// next Lock caller waiting for the lock will then be woken up and given
+	// ownership of the lock.
+	Unlock(ctx context.Context, in *UnlockRequest, opts ...grpc.CallOption) (*UnlockResponse, error)
+}
+
+type lockClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewLockClient(cc *grpc.ClientConn) LockClient {
+	return &lockClient{cc}
+}
+
+func (c *lockClient) Lock(ctx context.Context, in *LockRequest, opts ...grpc.CallOption) (*LockResponse, error) {
+	out := new(LockResponse)
+	err := grpc.Invoke(ctx, "/v3lockpb.Lock/Lock", in, out, c.cc, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *lockClient) Unlock(ctx context.Context, in *UnlockRequest, opts ...grpc.CallOption) (*UnlockResponse, error) {
+	out := new(UnlockResponse)
+	err := grpc.Invoke(ctx, "/v3lockpb.Lock/Unlock", in, out, c.cc, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+// Server API for Lock service
+
+type LockServer interface {
+	// Lock acquires a distributed shared lock on a given named lock.
+	// On success, it will return a unique key that exists so long as the
+	// lock is held by the caller. This key can be used in conjunction with
+	// transactions to safely ensure updates to etcd only occur while holding
+	// lock ownership. The lock is held until Unlock is called on the key or the
+	// lease associate with the owner expires.
+	Lock(context.Context, *LockRequest) (*LockResponse, error)
+	// Unlock takes a key returned by Lock and releases the hold on lock. The
+	// next Lock caller waiting for the lock will then be woken up and given
+	// ownership of the lock.
+	Unlock(context.Context, *UnlockRequest) (*UnlockResponse, error)
+}
+
+func RegisterLockServer(s *grpc.Server, srv LockServer) {
+	s.RegisterService(&_Lock_serviceDesc, srv)
+}
+
+func _Lock_Lock_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(LockRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(LockServer).Lock(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/v3lockpb.Lock/Lock",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(LockServer).Lock(ctx, req.(*LockRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Lock_Unlock_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(UnlockRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(LockServer).Unlock(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/v3lockpb.Lock/Unlock",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(LockServer).Unlock(ctx, req.(*UnlockRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+var _Lock_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "v3lockpb.Lock",
+	HandlerType: (*LockServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "Lock",
+			Handler:    _Lock_Lock_Handler,
+		},
+		{
+			MethodName: "Unlock",
+			Handler:    _Lock_Unlock_Handler,
+		},
+	},
+	Streams:  []grpc.StreamDesc{},
+	Metadata: "v3lock.proto",
+}
+
+func (m *LockRequest) 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 *LockRequest) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if len(m.Name) > 0 {
+		dAtA[i] = 0xa
+		i++
+		i = encodeVarintV3Lock(dAtA, i, uint64(len(m.Name)))
+		i += copy(dAtA[i:], m.Name)
+	}
+	if m.Lease != 0 {
+		dAtA[i] = 0x10
+		i++
+		i = encodeVarintV3Lock(dAtA, i, uint64(m.Lease))
+	}
+	return i, nil
+}
+
+func (m *LockResponse) 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 *LockResponse) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if m.Header != nil {
+		dAtA[i] = 0xa
+		i++
+		i = encodeVarintV3Lock(dAtA, i, uint64(m.Header.Size()))
+		n1, err := m.Header.MarshalTo(dAtA[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n1
+	}
+	if len(m.Key) > 0 {
+		dAtA[i] = 0x12
+		i++
+		i = encodeVarintV3Lock(dAtA, i, uint64(len(m.Key)))
+		i += copy(dAtA[i:], m.Key)
+	}
+	return i, nil
+}
+
+func (m *UnlockRequest) 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 *UnlockRequest) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if len(m.Key) > 0 {
+		dAtA[i] = 0xa
+		i++
+		i = encodeVarintV3Lock(dAtA, i, uint64(len(m.Key)))
+		i += copy(dAtA[i:], m.Key)
+	}
+	return i, nil
+}
+
+func (m *UnlockResponse) 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 *UnlockResponse) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if m.Header != nil {
+		dAtA[i] = 0xa
+		i++
+		i = encodeVarintV3Lock(dAtA, i, uint64(m.Header.Size()))
+		n2, err := m.Header.MarshalTo(dAtA[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n2
+	}
+	return i, nil
+}
+
+func encodeVarintV3Lock(dAtA []byte, offset int, v uint64) int {
+	for v >= 1<<7 {
+		dAtA[offset] = uint8(v&0x7f | 0x80)
+		v >>= 7
+		offset++
+	}
+	dAtA[offset] = uint8(v)
+	return offset + 1
+}
+func (m *LockRequest) Size() (n int) {
+	var l int
+	_ = l
+	l = len(m.Name)
+	if l > 0 {
+		n += 1 + l + sovV3Lock(uint64(l))
+	}
+	if m.Lease != 0 {
+		n += 1 + sovV3Lock(uint64(m.Lease))
+	}
+	return n
+}
+
+func (m *LockResponse) Size() (n int) {
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovV3Lock(uint64(l))
+	}
+	l = len(m.Key)
+	if l > 0 {
+		n += 1 + l + sovV3Lock(uint64(l))
+	}
+	return n
+}
+
+func (m *UnlockRequest) Size() (n int) {
+	var l int
+	_ = l
+	l = len(m.Key)
+	if l > 0 {
+		n += 1 + l + sovV3Lock(uint64(l))
+	}
+	return n
+}
+
+func (m *UnlockResponse) Size() (n int) {
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovV3Lock(uint64(l))
+	}
+	return n
+}
+
+func sovV3Lock(x uint64) (n int) {
+	for {
+		n++
+		x >>= 7
+		if x == 0 {
+			break
+		}
+	}
+	return n
+}
+func sozV3Lock(x uint64) (n int) {
+	return sovV3Lock(uint64((x << 1) ^ uint64((int64(x) >> 63))))
+}
+func (m *LockRequest) 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 ErrIntOverflowV3Lock
+			}
+			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: LockRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LockRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Name", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowV3Lock
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthV3Lock
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Name = append(m.Name[:0], dAtA[iNdEx:postIndex]...)
+			if m.Name == nil {
+				m.Name = []byte{}
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Lease", wireType)
+			}
+			m.Lease = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowV3Lock
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Lease |= (int64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipV3Lock(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthV3Lock
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *LockResponse) 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 ErrIntOverflowV3Lock
+			}
+			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: LockResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LockResponse: 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 ErrIntOverflowV3Lock
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthV3Lock
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &etcdserverpb.ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowV3Lock
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthV3Lock
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...)
+			if m.Key == nil {
+				m.Key = []byte{}
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipV3Lock(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthV3Lock
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *UnlockRequest) 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 ErrIntOverflowV3Lock
+			}
+			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: UnlockRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: UnlockRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowV3Lock
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthV3Lock
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...)
+			if m.Key == nil {
+				m.Key = []byte{}
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipV3Lock(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthV3Lock
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *UnlockResponse) 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 ErrIntOverflowV3Lock
+			}
+			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: UnlockResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: UnlockResponse: 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 ErrIntOverflowV3Lock
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthV3Lock
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &etcdserverpb.ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipV3Lock(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthV3Lock
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func skipV3Lock(dAtA []byte) (n int, err error) {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return 0, ErrIntOverflowV3Lock
+			}
+			if iNdEx >= l {
+				return 0, io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= (uint64(b) & 0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		wireType := int(wire & 0x7)
+		switch wireType {
+		case 0:
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return 0, ErrIntOverflowV3Lock
+				}
+				if iNdEx >= l {
+					return 0, io.ErrUnexpectedEOF
+				}
+				iNdEx++
+				if dAtA[iNdEx-1] < 0x80 {
+					break
+				}
+			}
+			return iNdEx, nil
+		case 1:
+			iNdEx += 8
+			return iNdEx, nil
+		case 2:
+			var length int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return 0, ErrIntOverflowV3Lock
+				}
+				if iNdEx >= l {
+					return 0, io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				length |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			iNdEx += length
+			if length < 0 {
+				return 0, ErrInvalidLengthV3Lock
+			}
+			return iNdEx, nil
+		case 3:
+			for {
+				var innerWire uint64
+				var start int = iNdEx
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return 0, ErrIntOverflowV3Lock
+					}
+					if iNdEx >= l {
+						return 0, io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					innerWire |= (uint64(b) & 0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				innerWireType := int(innerWire & 0x7)
+				if innerWireType == 4 {
+					break
+				}
+				next, err := skipV3Lock(dAtA[start:])
+				if err != nil {
+					return 0, err
+				}
+				iNdEx = start + next
+			}
+			return iNdEx, nil
+		case 4:
+			return iNdEx, nil
+		case 5:
+			iNdEx += 4
+			return iNdEx, nil
+		default:
+			return 0, fmt.Errorf("proto: illegal wireType %d", wireType)
+		}
+	}
+	panic("unreachable")
+}
+
+var (
+	ErrInvalidLengthV3Lock = fmt.Errorf("proto: negative length found during unmarshaling")
+	ErrIntOverflowV3Lock   = fmt.Errorf("proto: integer overflow")
+)
+
+func init() { proto.RegisterFile("v3lock.proto", fileDescriptorV3Lock) }
+
+var fileDescriptorV3Lock = []byte{
+	// 331 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0xe2, 0x29, 0x33, 0xce, 0xc9,
+	0x4f, 0xce, 0xd6, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0xe2, 0x80, 0xf0, 0x0a, 0x92, 0xa4, 0x44,
+	0xd2, 0xf3, 0xd3, 0xf3, 0xc1, 0x82, 0xfa, 0x20, 0x16, 0x44, 0x5e, 0x4a, 0x2d, 0xb5, 0x24, 0x39,
+	0x45, 0x1f, 0x44, 0x14, 0xa7, 0x16, 0x95, 0xa5, 0x16, 0x21, 0x31, 0x0b, 0x92, 0xf4, 0x8b, 0x0a,
+	0x92, 0xa1, 0xea, 0x64, 0xd2, 0xf3, 0xf3, 0xd3, 0x73, 0x52, 0xf5, 0x13, 0x0b, 0x32, 0xf5, 0x13,
+	0xf3, 0xf2, 0xf2, 0x4b, 0x12, 0x4b, 0x32, 0xf3, 0xf3, 0x8a, 0x21, 0xb2, 0x4a, 0xe6, 0x5c, 0xdc,
+	0x3e, 0xf9, 0xc9, 0xd9, 0x41, 0xa9, 0x85, 0xa5, 0xa9, 0xc5, 0x25, 0x42, 0x42, 0x5c, 0x2c, 0x79,
+	0x89, 0xb9, 0xa9, 0x12, 0x8c, 0x0a, 0x8c, 0x1a, 0x3c, 0x41, 0x60, 0xb6, 0x90, 0x08, 0x17, 0x6b,
+	0x4e, 0x6a, 0x62, 0x71, 0xaa, 0x04, 0x93, 0x02, 0xa3, 0x06, 0x73, 0x10, 0x84, 0xa3, 0x14, 0xc6,
+	0xc5, 0x03, 0xd1, 0x58, 0x5c, 0x90, 0x9f, 0x57, 0x9c, 0x2a, 0x64, 0xc2, 0xc5, 0x96, 0x91, 0x9a,
+	0x98, 0x92, 0x5a, 0x04, 0xd6, 0xcb, 0x6d, 0x24, 0xa3, 0x87, 0xec, 0x1e, 0x3d, 0x98, 0x3a, 0x0f,
+	0xb0, 0x9a, 0x20, 0xa8, 0x5a, 0x21, 0x01, 0x2e, 0xe6, 0xec, 0xd4, 0x4a, 0xb0, 0xc9, 0x3c, 0x41,
+	0x20, 0xa6, 0x92, 0x22, 0x17, 0x6f, 0x68, 0x5e, 0x0e, 0x92, 0x93, 0xa0, 0x4a, 0x18, 0x11, 0x4a,
+	0xdc, 0xb8, 0xf8, 0x60, 0x4a, 0x28, 0xb1, 0xdc, 0x68, 0x03, 0x23, 0x17, 0x0b, 0xc8, 0x0f, 0x42,
+	0xfe, 0x50, 0x5a, 0x54, 0x0f, 0x16, 0xe6, 0x7a, 0x48, 0x81, 0x22, 0x25, 0x86, 0x2e, 0x0c, 0x31,
+	0x4d, 0x49, 0xa2, 0xe9, 0xf2, 0x93, 0xc9, 0x4c, 0x42, 0x4a, 0xbc, 0xfa, 0x65, 0xc6, 0xfa, 0x20,
+	0x05, 0x60, 0xc2, 0x8a, 0x51, 0x4b, 0x28, 0x9c, 0x8b, 0x0d, 0xe2, 0x42, 0x21, 0x71, 0x84, 0x5e,
+	0x14, 0x6f, 0x49, 0x49, 0x60, 0x4a, 0x40, 0x8d, 0x95, 0x02, 0x1b, 0x2b, 0xa2, 0xc4, 0x0f, 0x37,
+	0xb6, 0x34, 0x0f, 0x6a, 0xb0, 0x93, 0xc0, 0x89, 0x47, 0x72, 0x8c, 0x17, 0x1e, 0xc9, 0x31, 0x3e,
+	0x78, 0x24, 0xc7, 0x38, 0xe3, 0xb1, 0x1c, 0x43, 0x12, 0x1b, 0x38, 0x1e, 0x8d, 0x01, 0x01, 0x00,
+	0x00, 0xff, 0xff, 0x65, 0xa8, 0x61, 0xb1, 0x3d, 0x02, 0x00, 0x00,
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3lock/v3lockpb/v3lock.proto b/vendor/go.etcd.io/etcd/etcdserver/api/v3lock/v3lockpb/v3lock.proto
new file mode 100644
index 0000000..7220c7f
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3lock/v3lockpb/v3lock.proto
@@ -0,0 +1,65 @@
+syntax = "proto3";
+package v3lockpb;
+
+import "gogoproto/gogo.proto";
+import "etcd/etcdserver/etcdserverpb/rpc.proto";
+
+// for grpc-gateway
+import "google/api/annotations.proto";
+
+option (gogoproto.marshaler_all) = true;
+option (gogoproto.unmarshaler_all) = true;
+
+// The lock service exposes client-side locking facilities as a gRPC interface.
+service Lock {
+  // Lock acquires a distributed shared lock on a given named lock.
+  // On success, it will return a unique key that exists so long as the
+  // lock is held by the caller. This key can be used in conjunction with
+  // transactions to safely ensure updates to etcd only occur while holding
+  // lock ownership. The lock is held until Unlock is called on the key or the
+  // lease associate with the owner expires.
+  rpc Lock(LockRequest) returns (LockResponse) {
+      option (google.api.http) = {
+        post: "/v3/lock/lock"
+        body: "*"
+    };
+  }
+
+  // Unlock takes a key returned by Lock and releases the hold on lock. The
+  // next Lock caller waiting for the lock will then be woken up and given
+  // ownership of the lock.
+  rpc Unlock(UnlockRequest) returns (UnlockResponse) {
+      option (google.api.http) = {
+        post: "/v3/lock/unlock"
+        body: "*"
+    };
+  }
+}
+
+message LockRequest {
+  // name is the identifier for the distributed shared lock to be acquired.
+  bytes name = 1;
+  // lease is the ID of the lease that will be attached to ownership of the
+  // lock. If the lease expires or is revoked and currently holds the lock,
+  // the lock is automatically released. Calls to Lock with the same lease will
+  // be treated as a single acquisition; locking twice with the same lease is a
+  // no-op.
+  int64 lease = 2;
+}
+
+message LockResponse {
+  etcdserverpb.ResponseHeader header = 1;
+  // key is a key that will exist on etcd for the duration that the Lock caller
+  // owns the lock. Users should not modify this key or the lock may exhibit
+  // undefined behavior.
+  bytes key = 2;
+}
+
+message UnlockRequest {
+  // key is the lock ownership key granted by Lock.
+  bytes key = 1;
+}
+
+message UnlockResponse {
+  etcdserverpb.ResponseHeader header = 1;
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/auth.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/auth.go
new file mode 100644
index 0000000..62ce757
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/auth.go
@@ -0,0 +1,158 @@
+// Copyright 2016 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 v3rpc
+
+import (
+	"context"
+
+	"go.etcd.io/etcd/etcdserver"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+)
+
+type AuthServer struct {
+	authenticator etcdserver.Authenticator
+}
+
+func NewAuthServer(s *etcdserver.EtcdServer) *AuthServer {
+	return &AuthServer{authenticator: s}
+}
+
+func (as *AuthServer) AuthEnable(ctx context.Context, r *pb.AuthEnableRequest) (*pb.AuthEnableResponse, error) {
+	resp, err := as.authenticator.AuthEnable(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return resp, nil
+}
+
+func (as *AuthServer) AuthDisable(ctx context.Context, r *pb.AuthDisableRequest) (*pb.AuthDisableResponse, error) {
+	resp, err := as.authenticator.AuthDisable(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return resp, nil
+}
+
+func (as *AuthServer) Authenticate(ctx context.Context, r *pb.AuthenticateRequest) (*pb.AuthenticateResponse, error) {
+	resp, err := as.authenticator.Authenticate(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return resp, nil
+}
+
+func (as *AuthServer) RoleAdd(ctx context.Context, r *pb.AuthRoleAddRequest) (*pb.AuthRoleAddResponse, error) {
+	resp, err := as.authenticator.RoleAdd(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return resp, nil
+}
+
+func (as *AuthServer) RoleDelete(ctx context.Context, r *pb.AuthRoleDeleteRequest) (*pb.AuthRoleDeleteResponse, error) {
+	resp, err := as.authenticator.RoleDelete(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return resp, nil
+}
+
+func (as *AuthServer) RoleGet(ctx context.Context, r *pb.AuthRoleGetRequest) (*pb.AuthRoleGetResponse, error) {
+	resp, err := as.authenticator.RoleGet(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return resp, nil
+}
+
+func (as *AuthServer) RoleList(ctx context.Context, r *pb.AuthRoleListRequest) (*pb.AuthRoleListResponse, error) {
+	resp, err := as.authenticator.RoleList(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return resp, nil
+}
+
+func (as *AuthServer) RoleRevokePermission(ctx context.Context, r *pb.AuthRoleRevokePermissionRequest) (*pb.AuthRoleRevokePermissionResponse, error) {
+	resp, err := as.authenticator.RoleRevokePermission(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return resp, nil
+}
+
+func (as *AuthServer) RoleGrantPermission(ctx context.Context, r *pb.AuthRoleGrantPermissionRequest) (*pb.AuthRoleGrantPermissionResponse, error) {
+	resp, err := as.authenticator.RoleGrantPermission(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return resp, nil
+}
+
+func (as *AuthServer) UserAdd(ctx context.Context, r *pb.AuthUserAddRequest) (*pb.AuthUserAddResponse, error) {
+	resp, err := as.authenticator.UserAdd(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return resp, nil
+}
+
+func (as *AuthServer) UserDelete(ctx context.Context, r *pb.AuthUserDeleteRequest) (*pb.AuthUserDeleteResponse, error) {
+	resp, err := as.authenticator.UserDelete(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return resp, nil
+}
+
+func (as *AuthServer) UserGet(ctx context.Context, r *pb.AuthUserGetRequest) (*pb.AuthUserGetResponse, error) {
+	resp, err := as.authenticator.UserGet(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return resp, nil
+}
+
+func (as *AuthServer) UserList(ctx context.Context, r *pb.AuthUserListRequest) (*pb.AuthUserListResponse, error) {
+	resp, err := as.authenticator.UserList(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return resp, nil
+}
+
+func (as *AuthServer) UserGrantRole(ctx context.Context, r *pb.AuthUserGrantRoleRequest) (*pb.AuthUserGrantRoleResponse, error) {
+	resp, err := as.authenticator.UserGrantRole(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return resp, nil
+}
+
+func (as *AuthServer) UserRevokeRole(ctx context.Context, r *pb.AuthUserRevokeRoleRequest) (*pb.AuthUserRevokeRoleResponse, error) {
+	resp, err := as.authenticator.UserRevokeRole(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return resp, nil
+}
+
+func (as *AuthServer) UserChangePassword(ctx context.Context, r *pb.AuthUserChangePasswordRequest) (*pb.AuthUserChangePasswordResponse, error) {
+	resp, err := as.authenticator.UserChangePassword(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return resp, nil
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/codec.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/codec.go
new file mode 100644
index 0000000..17a2c87
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/codec.go
@@ -0,0 +1,34 @@
+// Copyright 2016 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 v3rpc
+
+import "github.com/gogo/protobuf/proto"
+
+type codec struct{}
+
+func (c *codec) Marshal(v interface{}) ([]byte, error) {
+	b, err := proto.Marshal(v.(proto.Message))
+	sentBytes.Add(float64(len(b)))
+	return b, err
+}
+
+func (c *codec) Unmarshal(data []byte, v interface{}) error {
+	receivedBytes.Add(float64(len(data)))
+	return proto.Unmarshal(data, v.(proto.Message))
+}
+
+func (c *codec) String() string {
+	return "proto"
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/grpc.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/grpc.go
new file mode 100644
index 0000000..3332016
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/grpc.go
@@ -0,0 +1,77 @@
+// Copyright 2016 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 v3rpc
+
+import (
+	"crypto/tls"
+	"math"
+
+	"go.etcd.io/etcd/etcdserver"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+
+	grpc_middleware "github.com/grpc-ecosystem/go-grpc-middleware"
+	grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus"
+	"go.etcd.io/etcd/clientv3/credentials"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/health"
+	healthpb "google.golang.org/grpc/health/grpc_health_v1"
+)
+
+const (
+	grpcOverheadBytes = 512 * 1024
+	maxStreams        = math.MaxUint32
+	maxSendBytes      = math.MaxInt32
+)
+
+func Server(s *etcdserver.EtcdServer, tls *tls.Config, gopts ...grpc.ServerOption) *grpc.Server {
+	var opts []grpc.ServerOption
+	opts = append(opts, grpc.CustomCodec(&codec{}))
+	if tls != nil {
+		bundle := credentials.NewBundle(credentials.Config{TLSConfig: tls})
+		opts = append(opts, grpc.Creds(bundle.TransportCredentials()))
+	}
+	opts = append(opts, grpc.UnaryInterceptor(grpc_middleware.ChainUnaryServer(
+		newLogUnaryInterceptor(s),
+		newUnaryInterceptor(s),
+		grpc_prometheus.UnaryServerInterceptor,
+	)))
+	opts = append(opts, grpc.StreamInterceptor(grpc_middleware.ChainStreamServer(
+		newStreamInterceptor(s),
+		grpc_prometheus.StreamServerInterceptor,
+	)))
+	opts = append(opts, grpc.MaxRecvMsgSize(int(s.Cfg.MaxRequestBytes+grpcOverheadBytes)))
+	opts = append(opts, grpc.MaxSendMsgSize(maxSendBytes))
+	opts = append(opts, grpc.MaxConcurrentStreams(maxStreams))
+	grpcServer := grpc.NewServer(append(opts, gopts...)...)
+
+	pb.RegisterKVServer(grpcServer, NewQuotaKVServer(s))
+	pb.RegisterWatchServer(grpcServer, NewWatchServer(s))
+	pb.RegisterLeaseServer(grpcServer, NewQuotaLeaseServer(s))
+	pb.RegisterClusterServer(grpcServer, NewClusterServer(s))
+	pb.RegisterAuthServer(grpcServer, NewAuthServer(s))
+	pb.RegisterMaintenanceServer(grpcServer, NewMaintenanceServer(s))
+
+	// server should register all the services manually
+	// use empty service name for all etcd services' health status,
+	// see https://github.com/grpc/grpc/blob/master/doc/health-checking.md for more
+	hsrv := health.NewServer()
+	hsrv.SetServingStatus("", healthpb.HealthCheckResponse_SERVING)
+	healthpb.RegisterHealthServer(grpcServer, hsrv)
+
+	// set zero values for metrics registered for this grpc server
+	grpc_prometheus.Register(grpcServer)
+
+	return grpcServer
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/header.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/header.go
new file mode 100644
index 0000000..f23b6a7
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/header.go
@@ -0,0 +1,49 @@
+// Copyright 2016 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 v3rpc
+
+import (
+	"go.etcd.io/etcd/etcdserver"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+)
+
+type header struct {
+	clusterID int64
+	memberID  int64
+	sg        etcdserver.RaftStatusGetter
+	rev       func() int64
+}
+
+func newHeader(s *etcdserver.EtcdServer) header {
+	return header{
+		clusterID: int64(s.Cluster().ID()),
+		memberID:  int64(s.ID()),
+		sg:        s,
+		rev:       func() int64 { return s.KV().Rev() },
+	}
+}
+
+// fill populates pb.ResponseHeader using etcdserver information
+func (h *header) fill(rh *pb.ResponseHeader) {
+	if rh == nil {
+		plog.Panic("unexpected nil resp.Header")
+	}
+	rh.ClusterId = uint64(h.clusterID)
+	rh.MemberId = uint64(h.memberID)
+	rh.RaftTerm = h.sg.Term()
+	if rh.Revision == 0 {
+		rh.Revision = h.rev()
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/interceptor.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/interceptor.go
new file mode 100644
index 0000000..ce9047e
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/interceptor.go
@@ -0,0 +1,276 @@
+// Copyright 2016 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 v3rpc
+
+import (
+	"context"
+	"sync"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/api"
+	"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
+	"go.etcd.io/etcd/pkg/types"
+	"go.etcd.io/etcd/raft"
+
+	"github.com/coreos/pkg/capnslog"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.uber.org/zap"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/metadata"
+	"google.golang.org/grpc/peer"
+)
+
+const (
+	maxNoLeaderCnt = 3
+)
+
+type streamsMap struct {
+	mu      sync.Mutex
+	streams map[grpc.ServerStream]struct{}
+}
+
+func newUnaryInterceptor(s *etcdserver.EtcdServer) grpc.UnaryServerInterceptor {
+	return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
+		if !api.IsCapabilityEnabled(api.V3rpcCapability) {
+			return nil, rpctypes.ErrGRPCNotCapable
+		}
+
+		if s.IsMemberExist(s.ID()) && s.IsLearner() && !isRPCSupportedForLearner(req) {
+			return nil, rpctypes.ErrGPRCNotSupportedForLearner
+		}
+
+		md, ok := metadata.FromIncomingContext(ctx)
+		if ok {
+			if ks := md[rpctypes.MetadataRequireLeaderKey]; len(ks) > 0 && ks[0] == rpctypes.MetadataHasLeader {
+				if s.Leader() == types.ID(raft.None) {
+					return nil, rpctypes.ErrGRPCNoLeader
+				}
+			}
+		}
+
+		return handler(ctx, req)
+	}
+}
+
+func newLogUnaryInterceptor(s *etcdserver.EtcdServer) grpc.UnaryServerInterceptor {
+	return func(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
+		startTime := time.Now()
+		resp, err := handler(ctx, req)
+		lg := s.Logger()
+		if (lg != nil && lg.Core().Enabled(zap.DebugLevel)) || // using zap logger and debug level is enabled
+			(lg == nil && plog.LevelAt(capnslog.DEBUG)) { // or, using capnslog and debug level is enabled
+			defer logUnaryRequestStats(ctx, lg, info, startTime, req, resp)
+		}
+		return resp, err
+	}
+}
+
+func logUnaryRequestStats(ctx context.Context, lg *zap.Logger, info *grpc.UnaryServerInfo, startTime time.Time, req interface{}, resp interface{}) {
+	duration := time.Since(startTime)
+	remote := "No remote client info."
+	peerInfo, ok := peer.FromContext(ctx)
+	if ok {
+		remote = peerInfo.Addr.String()
+	}
+	responseType := info.FullMethod
+	var reqCount, respCount int64
+	var reqSize, respSize int
+	var reqContent string
+	switch _resp := resp.(type) {
+	case *pb.RangeResponse:
+		_req, ok := req.(*pb.RangeRequest)
+		if ok {
+			reqCount = 0
+			reqSize = _req.Size()
+			reqContent = _req.String()
+		}
+		if _resp != nil {
+			respCount = _resp.GetCount()
+			respSize = _resp.Size()
+		}
+	case *pb.PutResponse:
+		_req, ok := req.(*pb.PutRequest)
+		if ok {
+			reqCount = 1
+			reqSize = _req.Size()
+			reqContent = pb.NewLoggablePutRequest(_req).String()
+			// redact value field from request content, see PR #9821
+		}
+		if _resp != nil {
+			respCount = 0
+			respSize = _resp.Size()
+		}
+	case *pb.DeleteRangeResponse:
+		_req, ok := req.(*pb.DeleteRangeRequest)
+		if ok {
+			reqCount = 0
+			reqSize = _req.Size()
+			reqContent = _req.String()
+		}
+		if _resp != nil {
+			respCount = _resp.GetDeleted()
+			respSize = _resp.Size()
+		}
+	case *pb.TxnResponse:
+		_req, ok := req.(*pb.TxnRequest)
+		if ok && _resp != nil {
+			if _resp.GetSucceeded() { // determine the 'actual' count and size of request based on success or failure
+				reqCount = int64(len(_req.GetSuccess()))
+				reqSize = 0
+				for _, r := range _req.GetSuccess() {
+					reqSize += r.Size()
+				}
+			} else {
+				reqCount = int64(len(_req.GetFailure()))
+				reqSize = 0
+				for _, r := range _req.GetFailure() {
+					reqSize += r.Size()
+				}
+			}
+			reqContent = pb.NewLoggableTxnRequest(_req).String()
+			// redact value field from request content, see PR #9821
+		}
+		if _resp != nil {
+			respCount = 0
+			respSize = _resp.Size()
+		}
+	default:
+		reqCount = -1
+		reqSize = -1
+		respCount = -1
+		respSize = -1
+	}
+
+	logGenericRequestStats(lg, startTime, duration, remote, responseType, reqCount, reqSize, respCount, respSize, reqContent)
+}
+
+func logGenericRequestStats(lg *zap.Logger, startTime time.Time, duration time.Duration, remote string, responseType string,
+	reqCount int64, reqSize int, respCount int64, respSize int, reqContent string) {
+	if lg == nil {
+		plog.Debugf("start time = %v, "+
+			"time spent = %v, "+
+			"remote = %s, "+
+			"response type = %s, "+
+			"request count = %d, "+
+			"request size = %d, "+
+			"response count = %d, "+
+			"response size = %d, "+
+			"request content = %s",
+			startTime, duration, remote, responseType, reqCount, reqSize, respCount, respSize, reqContent,
+		)
+	} else {
+		lg.Debug("request stats",
+			zap.Time("start time", startTime),
+			zap.Duration("time spent", duration),
+			zap.String("remote", remote),
+			zap.String("response type", responseType),
+			zap.Int64("request count", reqCount),
+			zap.Int("request size", reqSize),
+			zap.Int64("response count", respCount),
+			zap.Int("response size", respSize),
+			zap.String("request content", reqContent),
+		)
+	}
+}
+
+func newStreamInterceptor(s *etcdserver.EtcdServer) grpc.StreamServerInterceptor {
+	smap := monitorLeader(s)
+
+	return func(srv interface{}, ss grpc.ServerStream, info *grpc.StreamServerInfo, handler grpc.StreamHandler) error {
+		if !api.IsCapabilityEnabled(api.V3rpcCapability) {
+			return rpctypes.ErrGRPCNotCapable
+		}
+
+		if s.IsMemberExist(s.ID()) && s.IsLearner() { // learner does not support stream RPC
+			return rpctypes.ErrGPRCNotSupportedForLearner
+		}
+
+		md, ok := metadata.FromIncomingContext(ss.Context())
+		if ok {
+			if ks := md[rpctypes.MetadataRequireLeaderKey]; len(ks) > 0 && ks[0] == rpctypes.MetadataHasLeader {
+				if s.Leader() == types.ID(raft.None) {
+					return rpctypes.ErrGRPCNoLeader
+				}
+
+				cctx, cancel := context.WithCancel(ss.Context())
+				ss = serverStreamWithCtx{ctx: cctx, cancel: &cancel, ServerStream: ss}
+
+				smap.mu.Lock()
+				smap.streams[ss] = struct{}{}
+				smap.mu.Unlock()
+
+				defer func() {
+					smap.mu.Lock()
+					delete(smap.streams, ss)
+					smap.mu.Unlock()
+					cancel()
+				}()
+
+			}
+		}
+
+		return handler(srv, ss)
+	}
+}
+
+type serverStreamWithCtx struct {
+	grpc.ServerStream
+	ctx    context.Context
+	cancel *context.CancelFunc
+}
+
+func (ssc serverStreamWithCtx) Context() context.Context { return ssc.ctx }
+
+func monitorLeader(s *etcdserver.EtcdServer) *streamsMap {
+	smap := &streamsMap{
+		streams: make(map[grpc.ServerStream]struct{}),
+	}
+
+	go func() {
+		election := time.Duration(s.Cfg.TickMs) * time.Duration(s.Cfg.ElectionTicks) * time.Millisecond
+		noLeaderCnt := 0
+
+		for {
+			select {
+			case <-s.StopNotify():
+				return
+			case <-time.After(election):
+				if s.Leader() == types.ID(raft.None) {
+					noLeaderCnt++
+				} else {
+					noLeaderCnt = 0
+				}
+
+				// We are more conservative on canceling existing streams. Reconnecting streams
+				// cost much more than just rejecting new requests. So we wait until the member
+				// cannot find a leader for maxNoLeaderCnt election timeouts to cancel existing streams.
+				if noLeaderCnt >= maxNoLeaderCnt {
+					smap.mu.Lock()
+					for ss := range smap.streams {
+						if ssWithCtx, ok := ss.(serverStreamWithCtx); ok {
+							(*ssWithCtx.cancel)()
+							<-ss.Context().Done()
+						}
+					}
+					smap.streams = make(map[grpc.ServerStream]struct{})
+					smap.mu.Unlock()
+				}
+			}
+		}
+	}()
+
+	return smap
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/key.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/key.go
new file mode 100644
index 0000000..ff59bac
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/key.go
@@ -0,0 +1,277 @@
+// Copyright 2015 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 v3rpc implements etcd v3 RPC system based on gRPC.
+package v3rpc
+
+import (
+	"context"
+
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/pkg/adt"
+
+	"github.com/coreos/pkg/capnslog"
+)
+
+var (
+	plog = capnslog.NewPackageLogger("go.etcd.io/etcd", "etcdserver/api/v3rpc")
+)
+
+type kvServer struct {
+	hdr header
+	kv  etcdserver.RaftKV
+	// maxTxnOps is the max operations per txn.
+	// e.g suppose maxTxnOps = 128.
+	// Txn.Success can have at most 128 operations,
+	// and Txn.Failure can have at most 128 operations.
+	maxTxnOps uint
+}
+
+func NewKVServer(s *etcdserver.EtcdServer) pb.KVServer {
+	return &kvServer{hdr: newHeader(s), kv: s, maxTxnOps: s.Cfg.MaxTxnOps}
+}
+
+func (s *kvServer) Range(ctx context.Context, r *pb.RangeRequest) (*pb.RangeResponse, error) {
+	if err := checkRangeRequest(r); err != nil {
+		return nil, err
+	}
+
+	resp, err := s.kv.Range(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+
+	s.hdr.fill(resp.Header)
+	return resp, nil
+}
+
+func (s *kvServer) Put(ctx context.Context, r *pb.PutRequest) (*pb.PutResponse, error) {
+	if err := checkPutRequest(r); err != nil {
+		return nil, err
+	}
+
+	resp, err := s.kv.Put(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+
+	s.hdr.fill(resp.Header)
+	return resp, nil
+}
+
+func (s *kvServer) DeleteRange(ctx context.Context, r *pb.DeleteRangeRequest) (*pb.DeleteRangeResponse, error) {
+	if err := checkDeleteRequest(r); err != nil {
+		return nil, err
+	}
+
+	resp, err := s.kv.DeleteRange(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+
+	s.hdr.fill(resp.Header)
+	return resp, nil
+}
+
+func (s *kvServer) Txn(ctx context.Context, r *pb.TxnRequest) (*pb.TxnResponse, error) {
+	if err := checkTxnRequest(r, int(s.maxTxnOps)); err != nil {
+		return nil, err
+	}
+	// check for forbidden put/del overlaps after checking request to avoid quadratic blowup
+	if _, _, err := checkIntervals(r.Success); err != nil {
+		return nil, err
+	}
+	if _, _, err := checkIntervals(r.Failure); err != nil {
+		return nil, err
+	}
+
+	resp, err := s.kv.Txn(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+
+	s.hdr.fill(resp.Header)
+	return resp, nil
+}
+
+func (s *kvServer) Compact(ctx context.Context, r *pb.CompactionRequest) (*pb.CompactionResponse, error) {
+	resp, err := s.kv.Compact(ctx, r)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+
+	s.hdr.fill(resp.Header)
+	return resp, nil
+}
+
+func checkRangeRequest(r *pb.RangeRequest) error {
+	if len(r.Key) == 0 {
+		return rpctypes.ErrGRPCEmptyKey
+	}
+	return nil
+}
+
+func checkPutRequest(r *pb.PutRequest) error {
+	if len(r.Key) == 0 {
+		return rpctypes.ErrGRPCEmptyKey
+	}
+	if r.IgnoreValue && len(r.Value) != 0 {
+		return rpctypes.ErrGRPCValueProvided
+	}
+	if r.IgnoreLease && r.Lease != 0 {
+		return rpctypes.ErrGRPCLeaseProvided
+	}
+	return nil
+}
+
+func checkDeleteRequest(r *pb.DeleteRangeRequest) error {
+	if len(r.Key) == 0 {
+		return rpctypes.ErrGRPCEmptyKey
+	}
+	return nil
+}
+
+func checkTxnRequest(r *pb.TxnRequest, maxTxnOps int) error {
+	opc := len(r.Compare)
+	if opc < len(r.Success) {
+		opc = len(r.Success)
+	}
+	if opc < len(r.Failure) {
+		opc = len(r.Failure)
+	}
+	if opc > maxTxnOps {
+		return rpctypes.ErrGRPCTooManyOps
+	}
+
+	for _, c := range r.Compare {
+		if len(c.Key) == 0 {
+			return rpctypes.ErrGRPCEmptyKey
+		}
+	}
+	for _, u := range r.Success {
+		if err := checkRequestOp(u, maxTxnOps-opc); err != nil {
+			return err
+		}
+	}
+	for _, u := range r.Failure {
+		if err := checkRequestOp(u, maxTxnOps-opc); err != nil {
+			return err
+		}
+	}
+
+	return nil
+}
+
+// checkIntervals tests whether puts and deletes overlap for a list of ops. If
+// there is an overlap, returns an error. If no overlap, return put and delete
+// sets for recursive evaluation.
+func checkIntervals(reqs []*pb.RequestOp) (map[string]struct{}, adt.IntervalTree, error) {
+	dels := adt.NewIntervalTree()
+
+	// collect deletes from this level; build first to check lower level overlapped puts
+	for _, req := range reqs {
+		tv, ok := req.Request.(*pb.RequestOp_RequestDeleteRange)
+		if !ok {
+			continue
+		}
+		dreq := tv.RequestDeleteRange
+		if dreq == nil {
+			continue
+		}
+		var iv adt.Interval
+		if len(dreq.RangeEnd) != 0 {
+			iv = adt.NewStringAffineInterval(string(dreq.Key), string(dreq.RangeEnd))
+		} else {
+			iv = adt.NewStringAffinePoint(string(dreq.Key))
+		}
+		dels.Insert(iv, struct{}{})
+	}
+
+	// collect children puts/deletes
+	puts := make(map[string]struct{})
+	for _, req := range reqs {
+		tv, ok := req.Request.(*pb.RequestOp_RequestTxn)
+		if !ok {
+			continue
+		}
+		putsThen, delsThen, err := checkIntervals(tv.RequestTxn.Success)
+		if err != nil {
+			return nil, dels, err
+		}
+		putsElse, delsElse, err := checkIntervals(tv.RequestTxn.Failure)
+		if err != nil {
+			return nil, dels, err
+		}
+		for k := range putsThen {
+			if _, ok := puts[k]; ok {
+				return nil, dels, rpctypes.ErrGRPCDuplicateKey
+			}
+			if dels.Intersects(adt.NewStringAffinePoint(k)) {
+				return nil, dels, rpctypes.ErrGRPCDuplicateKey
+			}
+			puts[k] = struct{}{}
+		}
+		for k := range putsElse {
+			if _, ok := puts[k]; ok {
+				// if key is from putsThen, overlap is OK since
+				// either then/else are mutually exclusive
+				if _, isSafe := putsThen[k]; !isSafe {
+					return nil, dels, rpctypes.ErrGRPCDuplicateKey
+				}
+			}
+			if dels.Intersects(adt.NewStringAffinePoint(k)) {
+				return nil, dels, rpctypes.ErrGRPCDuplicateKey
+			}
+			puts[k] = struct{}{}
+		}
+		dels.Union(delsThen, adt.NewStringAffineInterval("\x00", ""))
+		dels.Union(delsElse, adt.NewStringAffineInterval("\x00", ""))
+	}
+
+	// collect and check this level's puts
+	for _, req := range reqs {
+		tv, ok := req.Request.(*pb.RequestOp_RequestPut)
+		if !ok || tv.RequestPut == nil {
+			continue
+		}
+		k := string(tv.RequestPut.Key)
+		if _, ok := puts[k]; ok {
+			return nil, dels, rpctypes.ErrGRPCDuplicateKey
+		}
+		if dels.Intersects(adt.NewStringAffinePoint(k)) {
+			return nil, dels, rpctypes.ErrGRPCDuplicateKey
+		}
+		puts[k] = struct{}{}
+	}
+	return puts, dels, nil
+}
+
+func checkRequestOp(u *pb.RequestOp, maxTxnOps int) error {
+	// TODO: ensure only one of the field is set.
+	switch uv := u.Request.(type) {
+	case *pb.RequestOp_RequestRange:
+		return checkRangeRequest(uv.RequestRange)
+	case *pb.RequestOp_RequestPut:
+		return checkPutRequest(uv.RequestPut)
+	case *pb.RequestOp_RequestDeleteRange:
+		return checkDeleteRequest(uv.RequestDeleteRange)
+	case *pb.RequestOp_RequestTxn:
+		return checkTxnRequest(uv.RequestTxn, maxTxnOps)
+	default:
+		// empty op / nil entry
+		return rpctypes.ErrGRPCKeyNotFound
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/lease.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/lease.go
new file mode 100644
index 0000000..7441bee
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/lease.go
@@ -0,0 +1,169 @@
+// Copyright 2016 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 v3rpc
+
+import (
+	"context"
+	"io"
+
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/lease"
+
+	"go.uber.org/zap"
+)
+
+type LeaseServer struct {
+	lg  *zap.Logger
+	hdr header
+	le  etcdserver.Lessor
+}
+
+func NewLeaseServer(s *etcdserver.EtcdServer) pb.LeaseServer {
+	return &LeaseServer{lg: s.Cfg.Logger, le: s, hdr: newHeader(s)}
+}
+
+func (ls *LeaseServer) LeaseGrant(ctx context.Context, cr *pb.LeaseGrantRequest) (*pb.LeaseGrantResponse, error) {
+	resp, err := ls.le.LeaseGrant(ctx, cr)
+
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	ls.hdr.fill(resp.Header)
+	return resp, nil
+}
+
+func (ls *LeaseServer) LeaseRevoke(ctx context.Context, rr *pb.LeaseRevokeRequest) (*pb.LeaseRevokeResponse, error) {
+	resp, err := ls.le.LeaseRevoke(ctx, rr)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	ls.hdr.fill(resp.Header)
+	return resp, nil
+}
+
+func (ls *LeaseServer) LeaseTimeToLive(ctx context.Context, rr *pb.LeaseTimeToLiveRequest) (*pb.LeaseTimeToLiveResponse, error) {
+	resp, err := ls.le.LeaseTimeToLive(ctx, rr)
+	if err != nil && err != lease.ErrLeaseNotFound {
+		return nil, togRPCError(err)
+	}
+	if err == lease.ErrLeaseNotFound {
+		resp = &pb.LeaseTimeToLiveResponse{
+			Header: &pb.ResponseHeader{},
+			ID:     rr.ID,
+			TTL:    -1,
+		}
+	}
+	ls.hdr.fill(resp.Header)
+	return resp, nil
+}
+
+func (ls *LeaseServer) LeaseLeases(ctx context.Context, rr *pb.LeaseLeasesRequest) (*pb.LeaseLeasesResponse, error) {
+	resp, err := ls.le.LeaseLeases(ctx, rr)
+	if err != nil && err != lease.ErrLeaseNotFound {
+		return nil, togRPCError(err)
+	}
+	if err == lease.ErrLeaseNotFound {
+		resp = &pb.LeaseLeasesResponse{
+			Header: &pb.ResponseHeader{},
+			Leases: []*pb.LeaseStatus{},
+		}
+	}
+	ls.hdr.fill(resp.Header)
+	return resp, nil
+}
+
+func (ls *LeaseServer) LeaseKeepAlive(stream pb.Lease_LeaseKeepAliveServer) (err error) {
+	errc := make(chan error, 1)
+	go func() {
+		errc <- ls.leaseKeepAlive(stream)
+	}()
+	select {
+	case err = <-errc:
+	case <-stream.Context().Done():
+		// the only server-side cancellation is noleader for now.
+		err = stream.Context().Err()
+		if err == context.Canceled {
+			err = rpctypes.ErrGRPCNoLeader
+		}
+	}
+	return err
+}
+
+func (ls *LeaseServer) leaseKeepAlive(stream pb.Lease_LeaseKeepAliveServer) error {
+	for {
+		req, err := stream.Recv()
+		if err == io.EOF {
+			return nil
+		}
+		if err != nil {
+			if isClientCtxErr(stream.Context().Err(), err) {
+				if ls.lg != nil {
+					ls.lg.Debug("failed to receive lease keepalive request from gRPC stream", zap.Error(err))
+				} else {
+					plog.Debugf("failed to receive lease keepalive request from gRPC stream (%q)", err.Error())
+				}
+			} else {
+				if ls.lg != nil {
+					ls.lg.Warn("failed to receive lease keepalive request from gRPC stream", zap.Error(err))
+				} else {
+					plog.Warningf("failed to receive lease keepalive request from gRPC stream (%q)", err.Error())
+				}
+				streamFailures.WithLabelValues("receive", "lease-keepalive").Inc()
+			}
+			return err
+		}
+
+		// Create header before we sent out the renew request.
+		// This can make sure that the revision is strictly smaller or equal to
+		// when the keepalive happened at the local server (when the local server is the leader)
+		// or remote leader.
+		// Without this, a lease might be revoked at rev 3 but client can see the keepalive succeeded
+		// at rev 4.
+		resp := &pb.LeaseKeepAliveResponse{ID: req.ID, Header: &pb.ResponseHeader{}}
+		ls.hdr.fill(resp.Header)
+
+		ttl, err := ls.le.LeaseRenew(stream.Context(), lease.LeaseID(req.ID))
+		if err == lease.ErrLeaseNotFound {
+			err = nil
+			ttl = 0
+		}
+
+		if err != nil {
+			return togRPCError(err)
+		}
+
+		resp.TTL = ttl
+		err = stream.Send(resp)
+		if err != nil {
+			if isClientCtxErr(stream.Context().Err(), err) {
+				if ls.lg != nil {
+					ls.lg.Debug("failed to send lease keepalive response to gRPC stream", zap.Error(err))
+				} else {
+					plog.Debugf("failed to send lease keepalive response to gRPC stream (%q)", err.Error())
+				}
+			} else {
+				if ls.lg != nil {
+					ls.lg.Warn("failed to send lease keepalive response to gRPC stream", zap.Error(err))
+				} else {
+					plog.Warningf("failed to send lease keepalive response to gRPC stream (%q)", err.Error())
+				}
+				streamFailures.WithLabelValues("send", "lease-keepalive").Inc()
+			}
+			return err
+		}
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/maintenance.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/maintenance.go
new file mode 100644
index 0000000..c51271a
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/maintenance.go
@@ -0,0 +1,260 @@
+// Copyright 2016 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 v3rpc
+
+import (
+	"context"
+	"crypto/sha256"
+	"io"
+
+	"go.etcd.io/etcd/auth"
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/mvcc"
+	"go.etcd.io/etcd/mvcc/backend"
+	"go.etcd.io/etcd/raft"
+	"go.etcd.io/etcd/version"
+
+	"go.uber.org/zap"
+)
+
+type KVGetter interface {
+	KV() mvcc.ConsistentWatchableKV
+}
+
+type BackendGetter interface {
+	Backend() backend.Backend
+}
+
+type Alarmer interface {
+	// Alarms is implemented in Server interface located in etcdserver/server.go
+	// It returns a list of alarms present in the AlarmStore
+	Alarms() []*pb.AlarmMember
+	Alarm(ctx context.Context, ar *pb.AlarmRequest) (*pb.AlarmResponse, error)
+}
+
+type LeaderTransferrer interface {
+	MoveLeader(ctx context.Context, lead, target uint64) error
+}
+
+type AuthGetter interface {
+	AuthInfoFromCtx(ctx context.Context) (*auth.AuthInfo, error)
+	AuthStore() auth.AuthStore
+}
+
+type ClusterStatusGetter interface {
+	IsLearner() bool
+}
+
+type maintenanceServer struct {
+	lg  *zap.Logger
+	rg  etcdserver.RaftStatusGetter
+	kg  KVGetter
+	bg  BackendGetter
+	a   Alarmer
+	lt  LeaderTransferrer
+	hdr header
+	cs  ClusterStatusGetter
+}
+
+func NewMaintenanceServer(s *etcdserver.EtcdServer) pb.MaintenanceServer {
+	srv := &maintenanceServer{lg: s.Cfg.Logger, rg: s, kg: s, bg: s, a: s, lt: s, hdr: newHeader(s), cs: s}
+	return &authMaintenanceServer{srv, s}
+}
+
+func (ms *maintenanceServer) Defragment(ctx context.Context, sr *pb.DefragmentRequest) (*pb.DefragmentResponse, error) {
+	if ms.lg != nil {
+		ms.lg.Info("starting defragment")
+	} else {
+		plog.Noticef("starting to defragment the storage backend...")
+	}
+	err := ms.bg.Backend().Defrag()
+	if err != nil {
+		if ms.lg != nil {
+			ms.lg.Warn("failed to defragment", zap.Error(err))
+		} else {
+			plog.Errorf("failed to defragment the storage backend (%v)", err)
+		}
+		return nil, err
+	}
+	if ms.lg != nil {
+		ms.lg.Info("finished defragment")
+	} else {
+		plog.Noticef("finished defragmenting the storage backend")
+	}
+	return &pb.DefragmentResponse{}, nil
+}
+
+func (ms *maintenanceServer) Snapshot(sr *pb.SnapshotRequest, srv pb.Maintenance_SnapshotServer) error {
+	snap := ms.bg.Backend().Snapshot()
+	pr, pw := io.Pipe()
+
+	defer pr.Close()
+
+	go func() {
+		snap.WriteTo(pw)
+		if err := snap.Close(); err != nil {
+			if ms.lg != nil {
+				ms.lg.Warn("failed to close snapshot", zap.Error(err))
+			} else {
+				plog.Errorf("error closing snapshot (%v)", err)
+			}
+		}
+		pw.Close()
+	}()
+
+	// send file data
+	h := sha256.New()
+	br := int64(0)
+	buf := make([]byte, 32*1024)
+	sz := snap.Size()
+	for br < sz {
+		n, err := io.ReadFull(pr, buf)
+		if err != nil && err != io.EOF && err != io.ErrUnexpectedEOF {
+			return togRPCError(err)
+		}
+		br += int64(n)
+		resp := &pb.SnapshotResponse{
+			RemainingBytes: uint64(sz - br),
+			Blob:           buf[:n],
+		}
+		if err = srv.Send(resp); err != nil {
+			return togRPCError(err)
+		}
+		h.Write(buf[:n])
+	}
+
+	// send sha
+	sha := h.Sum(nil)
+	hresp := &pb.SnapshotResponse{RemainingBytes: 0, Blob: sha}
+	if err := srv.Send(hresp); err != nil {
+		return togRPCError(err)
+	}
+
+	return nil
+}
+
+func (ms *maintenanceServer) Hash(ctx context.Context, r *pb.HashRequest) (*pb.HashResponse, error) {
+	h, rev, err := ms.kg.KV().Hash()
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	resp := &pb.HashResponse{Header: &pb.ResponseHeader{Revision: rev}, Hash: h}
+	ms.hdr.fill(resp.Header)
+	return resp, nil
+}
+
+func (ms *maintenanceServer) HashKV(ctx context.Context, r *pb.HashKVRequest) (*pb.HashKVResponse, error) {
+	h, rev, compactRev, err := ms.kg.KV().HashByRev(r.Revision)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+
+	resp := &pb.HashKVResponse{Header: &pb.ResponseHeader{Revision: rev}, Hash: h, CompactRevision: compactRev}
+	ms.hdr.fill(resp.Header)
+	return resp, nil
+}
+
+func (ms *maintenanceServer) Alarm(ctx context.Context, ar *pb.AlarmRequest) (*pb.AlarmResponse, error) {
+	return ms.a.Alarm(ctx, ar)
+}
+
+func (ms *maintenanceServer) Status(ctx context.Context, ar *pb.StatusRequest) (*pb.StatusResponse, error) {
+	hdr := &pb.ResponseHeader{}
+	ms.hdr.fill(hdr)
+	resp := &pb.StatusResponse{
+		Header:           hdr,
+		Version:          version.Version,
+		Leader:           uint64(ms.rg.Leader()),
+		RaftIndex:        ms.rg.CommittedIndex(),
+		RaftAppliedIndex: ms.rg.AppliedIndex(),
+		RaftTerm:         ms.rg.Term(),
+		DbSize:           ms.bg.Backend().Size(),
+		DbSizeInUse:      ms.bg.Backend().SizeInUse(),
+		IsLearner:        ms.cs.IsLearner(),
+	}
+	if resp.Leader == raft.None {
+		resp.Errors = append(resp.Errors, etcdserver.ErrNoLeader.Error())
+	}
+	for _, a := range ms.a.Alarms() {
+		resp.Errors = append(resp.Errors, a.String())
+	}
+	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
+}
+
+func (ams *authMaintenanceServer) isAuthenticated(ctx context.Context) error {
+	authInfo, err := ams.ag.AuthInfoFromCtx(ctx)
+	if err != nil {
+		return err
+	}
+
+	return ams.ag.AuthStore().IsAdminPermitted(authInfo)
+}
+
+func (ams *authMaintenanceServer) Defragment(ctx context.Context, sr *pb.DefragmentRequest) (*pb.DefragmentResponse, error) {
+	if err := ams.isAuthenticated(ctx); err != nil {
+		return nil, err
+	}
+
+	return ams.maintenanceServer.Defragment(ctx, sr)
+}
+
+func (ams *authMaintenanceServer) Snapshot(sr *pb.SnapshotRequest, srv pb.Maintenance_SnapshotServer) error {
+	if err := ams.isAuthenticated(srv.Context()); err != nil {
+		return err
+	}
+
+	return ams.maintenanceServer.Snapshot(sr, srv)
+}
+
+func (ams *authMaintenanceServer) Hash(ctx context.Context, r *pb.HashRequest) (*pb.HashResponse, error) {
+	if err := ams.isAuthenticated(ctx); err != nil {
+		return nil, err
+	}
+
+	return ams.maintenanceServer.Hash(ctx, r)
+}
+
+func (ams *authMaintenanceServer) HashKV(ctx context.Context, r *pb.HashKVRequest) (*pb.HashKVResponse, error) {
+	if err := ams.isAuthenticated(ctx); err != nil {
+		return nil, err
+	}
+	return ams.maintenanceServer.HashKV(ctx, r)
+}
+
+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)
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/member.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/member.go
new file mode 100644
index 0000000..b2ebc98
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/member.go
@@ -0,0 +1,119 @@
+// Copyright 2016 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 v3rpc
+
+import (
+	"context"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/api"
+	"go.etcd.io/etcd/etcdserver/api/membership"
+	"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/pkg/types"
+)
+
+type ClusterServer struct {
+	cluster api.Cluster
+	server  etcdserver.ServerV3
+}
+
+func NewClusterServer(s etcdserver.ServerV3) *ClusterServer {
+	return &ClusterServer{
+		cluster: s.Cluster(),
+		server:  s,
+	}
+}
+
+func (cs *ClusterServer) MemberAdd(ctx context.Context, r *pb.MemberAddRequest) (*pb.MemberAddResponse, error) {
+	urls, err := types.NewURLs(r.PeerURLs)
+	if err != nil {
+		return nil, rpctypes.ErrGRPCMemberBadURLs
+	}
+
+	now := time.Now()
+	var m *membership.Member
+	if r.IsLearner {
+		m = membership.NewMemberAsLearner("", urls, "", &now)
+	} else {
+		m = membership.NewMember("", urls, "", &now)
+	}
+	membs, merr := cs.server.AddMember(ctx, *m)
+	if merr != nil {
+		return nil, togRPCError(merr)
+	}
+
+	return &pb.MemberAddResponse{
+		Header: cs.header(),
+		Member: &pb.Member{
+			ID:        uint64(m.ID),
+			PeerURLs:  m.PeerURLs,
+			IsLearner: m.IsLearner,
+		},
+		Members: membersToProtoMembers(membs),
+	}, nil
+}
+
+func (cs *ClusterServer) MemberRemove(ctx context.Context, r *pb.MemberRemoveRequest) (*pb.MemberRemoveResponse, error) {
+	membs, err := cs.server.RemoveMember(ctx, r.ID)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return &pb.MemberRemoveResponse{Header: cs.header(), Members: membersToProtoMembers(membs)}, nil
+}
+
+func (cs *ClusterServer) MemberUpdate(ctx context.Context, r *pb.MemberUpdateRequest) (*pb.MemberUpdateResponse, error) {
+	m := membership.Member{
+		ID:             types.ID(r.ID),
+		RaftAttributes: membership.RaftAttributes{PeerURLs: r.PeerURLs},
+	}
+	membs, err := cs.server.UpdateMember(ctx, m)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return &pb.MemberUpdateResponse{Header: cs.header(), Members: membersToProtoMembers(membs)}, nil
+}
+
+func (cs *ClusterServer) MemberList(ctx context.Context, r *pb.MemberListRequest) (*pb.MemberListResponse, error) {
+	membs := membersToProtoMembers(cs.cluster.Members())
+	return &pb.MemberListResponse{Header: cs.header(), Members: membs}, nil
+}
+
+func (cs *ClusterServer) MemberPromote(ctx context.Context, r *pb.MemberPromoteRequest) (*pb.MemberPromoteResponse, error) {
+	membs, err := cs.server.PromoteMember(ctx, r.ID)
+	if err != nil {
+		return nil, togRPCError(err)
+	}
+	return &pb.MemberPromoteResponse{Header: cs.header(), Members: membersToProtoMembers(membs)}, nil
+}
+
+func (cs *ClusterServer) header() *pb.ResponseHeader {
+	return &pb.ResponseHeader{ClusterId: uint64(cs.cluster.ID()), MemberId: uint64(cs.server.ID()), RaftTerm: cs.server.Term()}
+}
+
+func membersToProtoMembers(membs []*membership.Member) []*pb.Member {
+	protoMembs := make([]*pb.Member, len(membs))
+	for i := range membs {
+		protoMembs[i] = &pb.Member{
+			Name:       membs[i].Name,
+			ID:         uint64(membs[i].ID),
+			PeerURLs:   membs[i].PeerURLs,
+			ClientURLs: membs[i].ClientURLs,
+			IsLearner:  membs[i].IsLearner,
+		}
+	}
+	return protoMembs
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/metrics.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/metrics.go
new file mode 100644
index 0000000..d633d27
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/metrics.go
@@ -0,0 +1,48 @@
+// Copyright 2016 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 v3rpc
+
+import "github.com/prometheus/client_golang/prometheus"
+
+var (
+	sentBytes = prometheus.NewCounter(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "client_grpc_sent_bytes_total",
+		Help:      "The total number of bytes sent to grpc clients.",
+	})
+
+	receivedBytes = prometheus.NewCounter(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "client_grpc_received_bytes_total",
+		Help:      "The total number of bytes received from grpc clients.",
+	})
+
+	streamFailures = prometheus.NewCounterVec(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "network",
+		Name:      "server_stream_failures_total",
+		Help:      "The total number of stream failures from the local server.",
+	},
+		[]string{"Type", "API"},
+	)
+)
+
+func init() {
+	prometheus.MustRegister(sentBytes)
+	prometheus.MustRegister(receivedBytes)
+	prometheus.MustRegister(streamFailures)
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/quota.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/quota.go
new file mode 100644
index 0000000..a145b8b
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/quota.go
@@ -0,0 +1,90 @@
+// Copyright 2016 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 v3rpc
+
+import (
+	"context"
+
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/pkg/types"
+)
+
+type quotaKVServer struct {
+	pb.KVServer
+	qa quotaAlarmer
+}
+
+type quotaAlarmer struct {
+	q  etcdserver.Quota
+	a  Alarmer
+	id types.ID
+}
+
+// check whether request satisfies the quota. If there is not enough space,
+// ignore request and raise the free space alarm.
+func (qa *quotaAlarmer) check(ctx context.Context, r interface{}) error {
+	if qa.q.Available(r) {
+		return nil
+	}
+	req := &pb.AlarmRequest{
+		MemberID: uint64(qa.id),
+		Action:   pb.AlarmRequest_ACTIVATE,
+		Alarm:    pb.AlarmType_NOSPACE,
+	}
+	qa.a.Alarm(ctx, req)
+	return rpctypes.ErrGRPCNoSpace
+}
+
+func NewQuotaKVServer(s *etcdserver.EtcdServer) pb.KVServer {
+	return &quotaKVServer{
+		NewKVServer(s),
+		quotaAlarmer{etcdserver.NewBackendQuota(s, "kv"), s, s.ID()},
+	}
+}
+
+func (s *quotaKVServer) Put(ctx context.Context, r *pb.PutRequest) (*pb.PutResponse, error) {
+	if err := s.qa.check(ctx, r); err != nil {
+		return nil, err
+	}
+	return s.KVServer.Put(ctx, r)
+}
+
+func (s *quotaKVServer) Txn(ctx context.Context, r *pb.TxnRequest) (*pb.TxnResponse, error) {
+	if err := s.qa.check(ctx, r); err != nil {
+		return nil, err
+	}
+	return s.KVServer.Txn(ctx, r)
+}
+
+type quotaLeaseServer struct {
+	pb.LeaseServer
+	qa quotaAlarmer
+}
+
+func (s *quotaLeaseServer) LeaseGrant(ctx context.Context, cr *pb.LeaseGrantRequest) (*pb.LeaseGrantResponse, error) {
+	if err := s.qa.check(ctx, cr); err != nil {
+		return nil, err
+	}
+	return s.LeaseServer.LeaseGrant(ctx, cr)
+}
+
+func NewQuotaLeaseServer(s *etcdserver.EtcdServer) pb.LeaseServer {
+	return &quotaLeaseServer{
+		NewLeaseServer(s),
+		quotaAlarmer{etcdserver.NewBackendQuota(s, "lease"), s, s.ID()},
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/util.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/util.go
new file mode 100644
index 0000000..281ddc7
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/util.go
@@ -0,0 +1,136 @@
+// Copyright 2016 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 v3rpc
+
+import (
+	"context"
+	"strings"
+
+	"go.etcd.io/etcd/auth"
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/api/membership"
+	"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/lease"
+	"go.etcd.io/etcd/mvcc"
+
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+)
+
+var toGRPCErrorMap = map[error]error{
+	membership.ErrIDRemoved:               rpctypes.ErrGRPCMemberNotFound,
+	membership.ErrIDNotFound:              rpctypes.ErrGRPCMemberNotFound,
+	membership.ErrIDExists:                rpctypes.ErrGRPCMemberExist,
+	membership.ErrPeerURLexists:           rpctypes.ErrGRPCPeerURLExist,
+	membership.ErrMemberNotLearner:        rpctypes.ErrGRPCMemberNotLearner,
+	membership.ErrTooManyLearners:         rpctypes.ErrGRPCTooManyLearners,
+	etcdserver.ErrNotEnoughStartedMembers: rpctypes.ErrMemberNotEnoughStarted,
+	etcdserver.ErrLearnerNotReady:         rpctypes.ErrGRPCLearnerNotReady,
+
+	mvcc.ErrCompacted:             rpctypes.ErrGRPCCompacted,
+	mvcc.ErrFutureRev:             rpctypes.ErrGRPCFutureRev,
+	etcdserver.ErrRequestTooLarge: rpctypes.ErrGRPCRequestTooLarge,
+	etcdserver.ErrNoSpace:         rpctypes.ErrGRPCNoSpace,
+	etcdserver.ErrTooManyRequests: rpctypes.ErrTooManyRequests,
+
+	etcdserver.ErrNoLeader:                   rpctypes.ErrGRPCNoLeader,
+	etcdserver.ErrNotLeader:                  rpctypes.ErrGRPCNotLeader,
+	etcdserver.ErrLeaderChanged:              rpctypes.ErrGRPCLeaderChanged,
+	etcdserver.ErrStopped:                    rpctypes.ErrGRPCStopped,
+	etcdserver.ErrTimeout:                    rpctypes.ErrGRPCTimeout,
+	etcdserver.ErrTimeoutDueToLeaderFail:     rpctypes.ErrGRPCTimeoutDueToLeaderFail,
+	etcdserver.ErrTimeoutDueToConnectionLost: rpctypes.ErrGRPCTimeoutDueToConnectionLost,
+	etcdserver.ErrUnhealthy:                  rpctypes.ErrGRPCUnhealthy,
+	etcdserver.ErrKeyNotFound:                rpctypes.ErrGRPCKeyNotFound,
+	etcdserver.ErrCorrupt:                    rpctypes.ErrGRPCCorrupt,
+	etcdserver.ErrBadLeaderTransferee:        rpctypes.ErrGRPCBadLeaderTransferee,
+
+	lease.ErrLeaseNotFound:    rpctypes.ErrGRPCLeaseNotFound,
+	lease.ErrLeaseExists:      rpctypes.ErrGRPCLeaseExist,
+	lease.ErrLeaseTTLTooLarge: rpctypes.ErrGRPCLeaseTTLTooLarge,
+
+	auth.ErrRootUserNotExist:     rpctypes.ErrGRPCRootUserNotExist,
+	auth.ErrRootRoleNotExist:     rpctypes.ErrGRPCRootRoleNotExist,
+	auth.ErrUserAlreadyExist:     rpctypes.ErrGRPCUserAlreadyExist,
+	auth.ErrUserEmpty:            rpctypes.ErrGRPCUserEmpty,
+	auth.ErrUserNotFound:         rpctypes.ErrGRPCUserNotFound,
+	auth.ErrRoleAlreadyExist:     rpctypes.ErrGRPCRoleAlreadyExist,
+	auth.ErrRoleNotFound:         rpctypes.ErrGRPCRoleNotFound,
+	auth.ErrRoleEmpty:            rpctypes.ErrGRPCRoleEmpty,
+	auth.ErrAuthFailed:           rpctypes.ErrGRPCAuthFailed,
+	auth.ErrPermissionDenied:     rpctypes.ErrGRPCPermissionDenied,
+	auth.ErrRoleNotGranted:       rpctypes.ErrGRPCRoleNotGranted,
+	auth.ErrPermissionNotGranted: rpctypes.ErrGRPCPermissionNotGranted,
+	auth.ErrAuthNotEnabled:       rpctypes.ErrGRPCAuthNotEnabled,
+	auth.ErrInvalidAuthToken:     rpctypes.ErrGRPCInvalidAuthToken,
+	auth.ErrInvalidAuthMgmt:      rpctypes.ErrGRPCInvalidAuthMgmt,
+}
+
+func togRPCError(err error) error {
+	// let gRPC server convert to codes.Canceled, codes.DeadlineExceeded
+	if err == context.Canceled || err == context.DeadlineExceeded {
+		return err
+	}
+	grpcErr, ok := toGRPCErrorMap[err]
+	if !ok {
+		return status.Error(codes.Unknown, err.Error())
+	}
+	return grpcErr
+}
+
+func isClientCtxErr(ctxErr error, err error) bool {
+	if ctxErr != nil {
+		return true
+	}
+
+	ev, ok := status.FromError(err)
+	if !ok {
+		return false
+	}
+
+	switch ev.Code() {
+	case codes.Canceled, codes.DeadlineExceeded:
+		// client-side context cancel or deadline exceeded
+		// "rpc error: code = Canceled desc = context canceled"
+		// "rpc error: code = DeadlineExceeded desc = context deadline exceeded"
+		return true
+	case codes.Unavailable:
+		msg := ev.Message()
+		// client-side context cancel or deadline exceeded with TLS ("http2.errClientDisconnected")
+		// "rpc error: code = Unavailable desc = client disconnected"
+		if msg == "client disconnected" {
+			return true
+		}
+		// "grpc/transport.ClientTransport.CloseStream" on canceled streams
+		// "rpc error: code = Unavailable desc = stream error: stream ID 21; CANCEL")
+		if strings.HasPrefix(msg, "stream error: ") && strings.HasSuffix(msg, "; CANCEL") {
+			return true
+		}
+	}
+	return false
+}
+
+// in v3.4, learner is allowed to serve serializable read and endpoint status
+func isRPCSupportedForLearner(req interface{}) bool {
+	switch r := req.(type) {
+	case *pb.StatusRequest:
+		return true
+	case *pb.RangeRequest:
+		return r.Serializable
+	default:
+		return false
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/watch.go b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/watch.go
new file mode 100644
index 0000000..f41cb6c
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/api/v3rpc/watch.go
@@ -0,0 +1,584 @@
+// Copyright 2015 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 v3rpc
+
+import (
+	"context"
+	"io"
+	"math/rand"
+	"sync"
+	"time"
+
+	"go.etcd.io/etcd/auth"
+	"go.etcd.io/etcd/etcdserver"
+	"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/mvcc"
+	"go.etcd.io/etcd/mvcc/mvccpb"
+
+	"go.uber.org/zap"
+)
+
+type watchServer struct {
+	lg *zap.Logger
+
+	clusterID int64
+	memberID  int64
+
+	maxRequestBytes int
+
+	sg        etcdserver.RaftStatusGetter
+	watchable mvcc.WatchableKV
+	ag        AuthGetter
+}
+
+// NewWatchServer returns a new watch server.
+func NewWatchServer(s *etcdserver.EtcdServer) pb.WatchServer {
+	return &watchServer{
+		lg: s.Cfg.Logger,
+
+		clusterID: int64(s.Cluster().ID()),
+		memberID:  int64(s.ID()),
+
+		maxRequestBytes: int(s.Cfg.MaxRequestBytes + grpcOverheadBytes),
+
+		sg:        s,
+		watchable: s.Watchable(),
+		ag:        s,
+	}
+}
+
+var (
+	// External test can read this with GetProgressReportInterval()
+	// and change this to a small value to finish fast with
+	// SetProgressReportInterval().
+	progressReportInterval   = 10 * time.Minute
+	progressReportIntervalMu sync.RWMutex
+)
+
+// GetProgressReportInterval returns the current progress report interval (for testing).
+func GetProgressReportInterval() time.Duration {
+	progressReportIntervalMu.RLock()
+	interval := progressReportInterval
+	progressReportIntervalMu.RUnlock()
+
+	// add rand(1/10*progressReportInterval) as jitter so that etcdserver will not
+	// send progress notifications to watchers around the same time even when watchers
+	// are created around the same time (which is common when a client restarts itself).
+	jitter := time.Duration(rand.Int63n(int64(interval) / 10))
+
+	return interval + jitter
+}
+
+// SetProgressReportInterval updates the current progress report interval (for testing).
+func SetProgressReportInterval(newTimeout time.Duration) {
+	progressReportIntervalMu.Lock()
+	progressReportInterval = newTimeout
+	progressReportIntervalMu.Unlock()
+}
+
+// We send ctrl response inside the read loop. We do not want
+// send to block read, but we still want ctrl response we sent to
+// be serialized. Thus we use a buffered chan to solve the problem.
+// A small buffer should be OK for most cases, since we expect the
+// ctrl requests are infrequent.
+const ctrlStreamBufLen = 16
+
+// serverWatchStream is an etcd server side stream. It receives requests
+// from client side gRPC stream. It receives watch events from mvcc.WatchStream,
+// and creates responses that forwarded to gRPC stream.
+// It also forwards control message like watch created and canceled.
+type serverWatchStream struct {
+	lg *zap.Logger
+
+	clusterID int64
+	memberID  int64
+
+	maxRequestBytes int
+
+	sg        etcdserver.RaftStatusGetter
+	watchable mvcc.WatchableKV
+	ag        AuthGetter
+
+	gRPCStream  pb.Watch_WatchServer
+	watchStream mvcc.WatchStream
+	ctrlStream  chan *pb.WatchResponse
+
+	// mu protects progress, prevKV, fragment
+	mu sync.RWMutex
+	// tracks the watchID that stream might need to send progress to
+	// TODO: combine progress and prevKV into a single struct?
+	progress map[mvcc.WatchID]bool
+	// record watch IDs that need return previous key-value pair
+	prevKV map[mvcc.WatchID]bool
+	// records fragmented watch IDs
+	fragment map[mvcc.WatchID]bool
+
+	// closec indicates the stream is closed.
+	closec chan struct{}
+
+	// wg waits for the send loop to complete
+	wg sync.WaitGroup
+}
+
+func (ws *watchServer) Watch(stream pb.Watch_WatchServer) (err error) {
+	sws := serverWatchStream{
+		lg: ws.lg,
+
+		clusterID: ws.clusterID,
+		memberID:  ws.memberID,
+
+		maxRequestBytes: ws.maxRequestBytes,
+
+		sg:        ws.sg,
+		watchable: ws.watchable,
+		ag:        ws.ag,
+
+		gRPCStream:  stream,
+		watchStream: ws.watchable.NewWatchStream(),
+		// chan for sending control response like watcher created and canceled.
+		ctrlStream: make(chan *pb.WatchResponse, ctrlStreamBufLen),
+
+		progress: make(map[mvcc.WatchID]bool),
+		prevKV:   make(map[mvcc.WatchID]bool),
+		fragment: make(map[mvcc.WatchID]bool),
+
+		closec: make(chan struct{}),
+	}
+
+	sws.wg.Add(1)
+	go func() {
+		sws.sendLoop()
+		sws.wg.Done()
+	}()
+
+	errc := make(chan error, 1)
+	// Ideally recvLoop would also use sws.wg to signal its completion
+	// but when stream.Context().Done() is closed, the stream's recv
+	// may continue to block since it uses a different context, leading to
+	// deadlock when calling sws.close().
+	go func() {
+		if rerr := sws.recvLoop(); rerr != nil {
+			if isClientCtxErr(stream.Context().Err(), rerr) {
+				if sws.lg != nil {
+					sws.lg.Debug("failed to receive watch request from gRPC stream", zap.Error(rerr))
+				} else {
+					plog.Debugf("failed to receive watch request from gRPC stream (%q)", rerr.Error())
+				}
+			} else {
+				if sws.lg != nil {
+					sws.lg.Warn("failed to receive watch request from gRPC stream", zap.Error(rerr))
+				} else {
+					plog.Warningf("failed to receive watch request from gRPC stream (%q)", rerr.Error())
+				}
+				streamFailures.WithLabelValues("receive", "watch").Inc()
+			}
+			errc <- rerr
+		}
+	}()
+
+	select {
+	case err = <-errc:
+		close(sws.ctrlStream)
+
+	case <-stream.Context().Done():
+		err = stream.Context().Err()
+		// the only server-side cancellation is noleader for now.
+		if err == context.Canceled {
+			err = rpctypes.ErrGRPCNoLeader
+		}
+	}
+
+	sws.close()
+	return err
+}
+
+func (sws *serverWatchStream) isWatchPermitted(wcr *pb.WatchCreateRequest) bool {
+	authInfo, err := sws.ag.AuthInfoFromCtx(sws.gRPCStream.Context())
+	if err != nil {
+		return false
+	}
+	if authInfo == nil {
+		// if auth is enabled, IsRangePermitted() can cause an error
+		authInfo = &auth.AuthInfo{}
+	}
+	return sws.ag.AuthStore().IsRangePermitted(authInfo, wcr.Key, wcr.RangeEnd) == nil
+}
+
+func (sws *serverWatchStream) recvLoop() error {
+	for {
+		req, err := sws.gRPCStream.Recv()
+		if err == io.EOF {
+			return nil
+		}
+		if err != nil {
+			return err
+		}
+
+		switch uv := req.RequestUnion.(type) {
+		case *pb.WatchRequest_CreateRequest:
+			if uv.CreateRequest == nil {
+				break
+			}
+
+			creq := uv.CreateRequest
+			if len(creq.Key) == 0 {
+				// \x00 is the smallest key
+				creq.Key = []byte{0}
+			}
+			if len(creq.RangeEnd) == 0 {
+				// force nil since watchstream.Watch distinguishes
+				// between nil and []byte{} for single key / >=
+				creq.RangeEnd = nil
+			}
+			if len(creq.RangeEnd) == 1 && creq.RangeEnd[0] == 0 {
+				// support  >= key queries
+				creq.RangeEnd = []byte{}
+			}
+
+			if !sws.isWatchPermitted(creq) {
+				wr := &pb.WatchResponse{
+					Header:       sws.newResponseHeader(sws.watchStream.Rev()),
+					WatchId:      creq.WatchId,
+					Canceled:     true,
+					Created:      true,
+					CancelReason: rpctypes.ErrGRPCPermissionDenied.Error(),
+				}
+
+				select {
+				case sws.ctrlStream <- wr:
+				case <-sws.closec:
+				}
+				return nil
+			}
+
+			filters := FiltersFromRequest(creq)
+
+			wsrev := sws.watchStream.Rev()
+			rev := creq.StartRevision
+			if rev == 0 {
+				rev = wsrev + 1
+			}
+			id, err := sws.watchStream.Watch(mvcc.WatchID(creq.WatchId), creq.Key, creq.RangeEnd, rev, filters...)
+			if err == nil {
+				sws.mu.Lock()
+				if creq.ProgressNotify {
+					sws.progress[id] = true
+				}
+				if creq.PrevKv {
+					sws.prevKV[id] = true
+				}
+				if creq.Fragment {
+					sws.fragment[id] = true
+				}
+				sws.mu.Unlock()
+			}
+			wr := &pb.WatchResponse{
+				Header:   sws.newResponseHeader(wsrev),
+				WatchId:  int64(id),
+				Created:  true,
+				Canceled: err != nil,
+			}
+			if err != nil {
+				wr.CancelReason = err.Error()
+			}
+			select {
+			case sws.ctrlStream <- wr:
+			case <-sws.closec:
+				return nil
+			}
+
+		case *pb.WatchRequest_CancelRequest:
+			if uv.CancelRequest != nil {
+				id := uv.CancelRequest.WatchId
+				err := sws.watchStream.Cancel(mvcc.WatchID(id))
+				if err == nil {
+					sws.ctrlStream <- &pb.WatchResponse{
+						Header:   sws.newResponseHeader(sws.watchStream.Rev()),
+						WatchId:  id,
+						Canceled: true,
+					}
+					sws.mu.Lock()
+					delete(sws.progress, mvcc.WatchID(id))
+					delete(sws.prevKV, mvcc.WatchID(id))
+					delete(sws.fragment, mvcc.WatchID(id))
+					sws.mu.Unlock()
+				}
+			}
+		case *pb.WatchRequest_ProgressRequest:
+			if uv.ProgressRequest != nil {
+				sws.ctrlStream <- &pb.WatchResponse{
+					Header:  sws.newResponseHeader(sws.watchStream.Rev()),
+					WatchId: -1, // response is not associated with any WatchId and will be broadcast to all watch channels
+				}
+			}
+		default:
+			// we probably should not shutdown the entire stream when
+			// receive an valid command.
+			// so just do nothing instead.
+			continue
+		}
+	}
+}
+
+func (sws *serverWatchStream) sendLoop() {
+	// watch ids that are currently active
+	ids := make(map[mvcc.WatchID]struct{})
+	// watch responses pending on a watch id creation message
+	pending := make(map[mvcc.WatchID][]*pb.WatchResponse)
+
+	interval := GetProgressReportInterval()
+	progressTicker := time.NewTicker(interval)
+
+	defer func() {
+		progressTicker.Stop()
+		// drain the chan to clean up pending events
+		for ws := range sws.watchStream.Chan() {
+			mvcc.ReportEventReceived(len(ws.Events))
+		}
+		for _, wrs := range pending {
+			for _, ws := range wrs {
+				mvcc.ReportEventReceived(len(ws.Events))
+			}
+		}
+	}()
+
+	for {
+		select {
+		case wresp, ok := <-sws.watchStream.Chan():
+			if !ok {
+				return
+			}
+
+			// TODO: evs is []mvccpb.Event type
+			// either return []*mvccpb.Event from the mvcc package
+			// or define protocol buffer with []mvccpb.Event.
+			evs := wresp.Events
+			events := make([]*mvccpb.Event, len(evs))
+			sws.mu.RLock()
+			needPrevKV := sws.prevKV[wresp.WatchID]
+			sws.mu.RUnlock()
+			for i := range evs {
+				events[i] = &evs[i]
+				if needPrevKV {
+					opt := mvcc.RangeOptions{Rev: evs[i].Kv.ModRevision - 1}
+					r, err := sws.watchable.Range(evs[i].Kv.Key, nil, opt)
+					if err == nil && len(r.KVs) != 0 {
+						events[i].PrevKv = &(r.KVs[0])
+					}
+				}
+			}
+
+			canceled := wresp.CompactRevision != 0
+			wr := &pb.WatchResponse{
+				Header:          sws.newResponseHeader(wresp.Revision),
+				WatchId:         int64(wresp.WatchID),
+				Events:          events,
+				CompactRevision: wresp.CompactRevision,
+				Canceled:        canceled,
+			}
+
+			if _, okID := ids[wresp.WatchID]; !okID {
+				// buffer if id not yet announced
+				wrs := append(pending[wresp.WatchID], wr)
+				pending[wresp.WatchID] = wrs
+				continue
+			}
+
+			mvcc.ReportEventReceived(len(evs))
+
+			sws.mu.RLock()
+			fragmented, ok := sws.fragment[wresp.WatchID]
+			sws.mu.RUnlock()
+
+			var serr error
+			if !fragmented && !ok {
+				serr = sws.gRPCStream.Send(wr)
+			} else {
+				serr = sendFragments(wr, sws.maxRequestBytes, sws.gRPCStream.Send)
+			}
+
+			if serr != nil {
+				if isClientCtxErr(sws.gRPCStream.Context().Err(), serr) {
+					if sws.lg != nil {
+						sws.lg.Debug("failed to send watch response to gRPC stream", zap.Error(serr))
+					} else {
+						plog.Debugf("failed to send watch response to gRPC stream (%q)", serr.Error())
+					}
+				} else {
+					if sws.lg != nil {
+						sws.lg.Warn("failed to send watch response to gRPC stream", zap.Error(serr))
+					} else {
+						plog.Warningf("failed to send watch response to gRPC stream (%q)", serr.Error())
+					}
+					streamFailures.WithLabelValues("send", "watch").Inc()
+				}
+				return
+			}
+
+			sws.mu.Lock()
+			if len(evs) > 0 && sws.progress[wresp.WatchID] {
+				// elide next progress update if sent a key update
+				sws.progress[wresp.WatchID] = false
+			}
+			sws.mu.Unlock()
+
+		case c, ok := <-sws.ctrlStream:
+			if !ok {
+				return
+			}
+
+			if err := sws.gRPCStream.Send(c); err != nil {
+				if isClientCtxErr(sws.gRPCStream.Context().Err(), err) {
+					if sws.lg != nil {
+						sws.lg.Debug("failed to send watch control response to gRPC stream", zap.Error(err))
+					} else {
+						plog.Debugf("failed to send watch control response to gRPC stream (%q)", err.Error())
+					}
+				} else {
+					if sws.lg != nil {
+						sws.lg.Warn("failed to send watch control response to gRPC stream", zap.Error(err))
+					} else {
+						plog.Warningf("failed to send watch control response to gRPC stream (%q)", err.Error())
+					}
+					streamFailures.WithLabelValues("send", "watch").Inc()
+				}
+				return
+			}
+
+			// track id creation
+			wid := mvcc.WatchID(c.WatchId)
+			if c.Canceled {
+				delete(ids, wid)
+				continue
+			}
+			if c.Created {
+				// flush buffered events
+				ids[wid] = struct{}{}
+				for _, v := range pending[wid] {
+					mvcc.ReportEventReceived(len(v.Events))
+					if err := sws.gRPCStream.Send(v); err != nil {
+						if isClientCtxErr(sws.gRPCStream.Context().Err(), err) {
+							if sws.lg != nil {
+								sws.lg.Debug("failed to send pending watch response to gRPC stream", zap.Error(err))
+							} else {
+								plog.Debugf("failed to send pending watch response to gRPC stream (%q)", err.Error())
+							}
+						} else {
+							if sws.lg != nil {
+								sws.lg.Warn("failed to send pending watch response to gRPC stream", zap.Error(err))
+							} else {
+								plog.Warningf("failed to send pending watch response to gRPC stream (%q)", err.Error())
+							}
+							streamFailures.WithLabelValues("send", "watch").Inc()
+						}
+						return
+					}
+				}
+				delete(pending, wid)
+			}
+
+		case <-progressTicker.C:
+			sws.mu.Lock()
+			for id, ok := range sws.progress {
+				if ok {
+					sws.watchStream.RequestProgress(id)
+				}
+				sws.progress[id] = true
+			}
+			sws.mu.Unlock()
+
+		case <-sws.closec:
+			return
+		}
+	}
+}
+
+func sendFragments(
+	wr *pb.WatchResponse,
+	maxRequestBytes int,
+	sendFunc func(*pb.WatchResponse) error) error {
+	// no need to fragment if total request size is smaller
+	// than max request limit or response contains only one event
+	if wr.Size() < maxRequestBytes || len(wr.Events) < 2 {
+		return sendFunc(wr)
+	}
+
+	ow := *wr
+	ow.Events = make([]*mvccpb.Event, 0)
+	ow.Fragment = true
+
+	var idx int
+	for {
+		cur := ow
+		for _, ev := range wr.Events[idx:] {
+			cur.Events = append(cur.Events, ev)
+			if len(cur.Events) > 1 && cur.Size() >= maxRequestBytes {
+				cur.Events = cur.Events[:len(cur.Events)-1]
+				break
+			}
+			idx++
+		}
+		if idx == len(wr.Events) {
+			// last response has no more fragment
+			cur.Fragment = false
+		}
+		if err := sendFunc(&cur); err != nil {
+			return err
+		}
+		if !cur.Fragment {
+			break
+		}
+	}
+	return nil
+}
+
+func (sws *serverWatchStream) close() {
+	sws.watchStream.Close()
+	close(sws.closec)
+	sws.wg.Wait()
+}
+
+func (sws *serverWatchStream) newResponseHeader(rev int64) *pb.ResponseHeader {
+	return &pb.ResponseHeader{
+		ClusterId: uint64(sws.clusterID),
+		MemberId:  uint64(sws.memberID),
+		Revision:  rev,
+		RaftTerm:  sws.sg.Term(),
+	}
+}
+
+func filterNoDelete(e mvccpb.Event) bool {
+	return e.Type == mvccpb.DELETE
+}
+
+func filterNoPut(e mvccpb.Event) bool {
+	return e.Type == mvccpb.PUT
+}
+
+// FiltersFromRequest returns "mvcc.FilterFunc" from a given watch create request.
+func FiltersFromRequest(creq *pb.WatchCreateRequest) []mvcc.FilterFunc {
+	filters := make([]mvcc.FilterFunc, 0, len(creq.Filters))
+	for _, ft := range creq.Filters {
+		switch ft {
+		case pb.WatchCreateRequest_NOPUT:
+			filters = append(filters, filterNoPut)
+		case pb.WatchCreateRequest_NODELETE:
+			filters = append(filters, filterNoDelete)
+		default:
+		}
+	}
+	return filters
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/apply.go b/vendor/go.etcd.io/etcd/etcdserver/apply.go
new file mode 100644
index 0000000..1f06ad0
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/apply.go
@@ -0,0 +1,1019 @@
+// Copyright 2016 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import (
+	"bytes"
+	"context"
+	"fmt"
+	"sort"
+	"time"
+
+	"go.etcd.io/etcd/auth"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/lease"
+	"go.etcd.io/etcd/mvcc"
+	"go.etcd.io/etcd/mvcc/mvccpb"
+	"go.etcd.io/etcd/pkg/types"
+
+	"github.com/gogo/protobuf/proto"
+	"go.uber.org/zap"
+)
+
+const (
+	warnApplyDuration = 100 * time.Millisecond
+)
+
+type applyResult struct {
+	resp proto.Message
+	err  error
+	// physc signals the physical effect of the request has completed in addition
+	// to being logically reflected by the node. Currently only used for
+	// Compaction requests.
+	physc <-chan struct{}
+}
+
+// applierV3 is the interface for processing V3 raft messages
+type applierV3 interface {
+	Apply(r *pb.InternalRaftRequest) *applyResult
+
+	Put(txn mvcc.TxnWrite, p *pb.PutRequest) (*pb.PutResponse, error)
+	Range(txn mvcc.TxnRead, r *pb.RangeRequest) (*pb.RangeResponse, error)
+	DeleteRange(txn mvcc.TxnWrite, dr *pb.DeleteRangeRequest) (*pb.DeleteRangeResponse, error)
+	Txn(rt *pb.TxnRequest) (*pb.TxnResponse, error)
+	Compaction(compaction *pb.CompactionRequest) (*pb.CompactionResponse, <-chan struct{}, error)
+
+	LeaseGrant(lc *pb.LeaseGrantRequest) (*pb.LeaseGrantResponse, error)
+	LeaseRevoke(lc *pb.LeaseRevokeRequest) (*pb.LeaseRevokeResponse, error)
+
+	LeaseCheckpoint(lc *pb.LeaseCheckpointRequest) (*pb.LeaseCheckpointResponse, error)
+
+	Alarm(*pb.AlarmRequest) (*pb.AlarmResponse, error)
+
+	Authenticate(r *pb.InternalAuthenticateRequest) (*pb.AuthenticateResponse, error)
+
+	AuthEnable() (*pb.AuthEnableResponse, error)
+	AuthDisable() (*pb.AuthDisableResponse, error)
+
+	UserAdd(ua *pb.AuthUserAddRequest) (*pb.AuthUserAddResponse, error)
+	UserDelete(ua *pb.AuthUserDeleteRequest) (*pb.AuthUserDeleteResponse, error)
+	UserChangePassword(ua *pb.AuthUserChangePasswordRequest) (*pb.AuthUserChangePasswordResponse, error)
+	UserGrantRole(ua *pb.AuthUserGrantRoleRequest) (*pb.AuthUserGrantRoleResponse, error)
+	UserGet(ua *pb.AuthUserGetRequest) (*pb.AuthUserGetResponse, error)
+	UserRevokeRole(ua *pb.AuthUserRevokeRoleRequest) (*pb.AuthUserRevokeRoleResponse, error)
+	RoleAdd(ua *pb.AuthRoleAddRequest) (*pb.AuthRoleAddResponse, error)
+	RoleGrantPermission(ua *pb.AuthRoleGrantPermissionRequest) (*pb.AuthRoleGrantPermissionResponse, error)
+	RoleGet(ua *pb.AuthRoleGetRequest) (*pb.AuthRoleGetResponse, error)
+	RoleRevokePermission(ua *pb.AuthRoleRevokePermissionRequest) (*pb.AuthRoleRevokePermissionResponse, error)
+	RoleDelete(ua *pb.AuthRoleDeleteRequest) (*pb.AuthRoleDeleteResponse, error)
+	UserList(ua *pb.AuthUserListRequest) (*pb.AuthUserListResponse, error)
+	RoleList(ua *pb.AuthRoleListRequest) (*pb.AuthRoleListResponse, error)
+}
+
+type checkReqFunc func(mvcc.ReadView, *pb.RequestOp) error
+
+type applierV3backend struct {
+	s *EtcdServer
+
+	checkPut   checkReqFunc
+	checkRange checkReqFunc
+}
+
+func (s *EtcdServer) newApplierV3Backend() applierV3 {
+	base := &applierV3backend{s: s}
+	base.checkPut = func(rv mvcc.ReadView, req *pb.RequestOp) error {
+		return base.checkRequestPut(rv, req)
+	}
+	base.checkRange = func(rv mvcc.ReadView, req *pb.RequestOp) error {
+		return base.checkRequestRange(rv, req)
+	}
+	return base
+}
+
+func (s *EtcdServer) newApplierV3() applierV3 {
+	return newAuthApplierV3(
+		s.AuthStore(),
+		newQuotaApplierV3(s, s.newApplierV3Backend()),
+		s.lessor,
+	)
+}
+
+func (a *applierV3backend) Apply(r *pb.InternalRaftRequest) *applyResult {
+	ar := &applyResult{}
+	defer func(start time.Time) {
+		warnOfExpensiveRequest(a.s.getLogger(), start, &pb.InternalRaftStringer{Request: r}, ar.resp, ar.err)
+	}(time.Now())
+
+	// call into a.s.applyV3.F instead of a.F so upper appliers can check individual calls
+	switch {
+	case r.Range != nil:
+		ar.resp, ar.err = a.s.applyV3.Range(nil, r.Range)
+	case r.Put != nil:
+		ar.resp, ar.err = a.s.applyV3.Put(nil, r.Put)
+	case r.DeleteRange != nil:
+		ar.resp, ar.err = a.s.applyV3.DeleteRange(nil, r.DeleteRange)
+	case r.Txn != nil:
+		ar.resp, ar.err = a.s.applyV3.Txn(r.Txn)
+	case r.Compaction != nil:
+		ar.resp, ar.physc, ar.err = a.s.applyV3.Compaction(r.Compaction)
+	case r.LeaseGrant != nil:
+		ar.resp, ar.err = a.s.applyV3.LeaseGrant(r.LeaseGrant)
+	case r.LeaseRevoke != nil:
+		ar.resp, ar.err = a.s.applyV3.LeaseRevoke(r.LeaseRevoke)
+	case r.LeaseCheckpoint != nil:
+		ar.resp, ar.err = a.s.applyV3.LeaseCheckpoint(r.LeaseCheckpoint)
+	case r.Alarm != nil:
+		ar.resp, ar.err = a.s.applyV3.Alarm(r.Alarm)
+	case r.Authenticate != nil:
+		ar.resp, ar.err = a.s.applyV3.Authenticate(r.Authenticate)
+	case r.AuthEnable != nil:
+		ar.resp, ar.err = a.s.applyV3.AuthEnable()
+	case r.AuthDisable != nil:
+		ar.resp, ar.err = a.s.applyV3.AuthDisable()
+	case r.AuthUserAdd != nil:
+		ar.resp, ar.err = a.s.applyV3.UserAdd(r.AuthUserAdd)
+	case r.AuthUserDelete != nil:
+		ar.resp, ar.err = a.s.applyV3.UserDelete(r.AuthUserDelete)
+	case r.AuthUserChangePassword != nil:
+		ar.resp, ar.err = a.s.applyV3.UserChangePassword(r.AuthUserChangePassword)
+	case r.AuthUserGrantRole != nil:
+		ar.resp, ar.err = a.s.applyV3.UserGrantRole(r.AuthUserGrantRole)
+	case r.AuthUserGet != nil:
+		ar.resp, ar.err = a.s.applyV3.UserGet(r.AuthUserGet)
+	case r.AuthUserRevokeRole != nil:
+		ar.resp, ar.err = a.s.applyV3.UserRevokeRole(r.AuthUserRevokeRole)
+	case r.AuthRoleAdd != nil:
+		ar.resp, ar.err = a.s.applyV3.RoleAdd(r.AuthRoleAdd)
+	case r.AuthRoleGrantPermission != nil:
+		ar.resp, ar.err = a.s.applyV3.RoleGrantPermission(r.AuthRoleGrantPermission)
+	case r.AuthRoleGet != nil:
+		ar.resp, ar.err = a.s.applyV3.RoleGet(r.AuthRoleGet)
+	case r.AuthRoleRevokePermission != nil:
+		ar.resp, ar.err = a.s.applyV3.RoleRevokePermission(r.AuthRoleRevokePermission)
+	case r.AuthRoleDelete != nil:
+		ar.resp, ar.err = a.s.applyV3.RoleDelete(r.AuthRoleDelete)
+	case r.AuthUserList != nil:
+		ar.resp, ar.err = a.s.applyV3.UserList(r.AuthUserList)
+	case r.AuthRoleList != nil:
+		ar.resp, ar.err = a.s.applyV3.RoleList(r.AuthRoleList)
+	default:
+		panic("not implemented")
+	}
+	return ar
+}
+
+func (a *applierV3backend) Put(txn mvcc.TxnWrite, p *pb.PutRequest) (resp *pb.PutResponse, err error) {
+	resp = &pb.PutResponse{}
+	resp.Header = &pb.ResponseHeader{}
+
+	val, leaseID := p.Value, lease.LeaseID(p.Lease)
+	if txn == nil {
+		if leaseID != lease.NoLease {
+			if l := a.s.lessor.Lookup(leaseID); l == nil {
+				return nil, lease.ErrLeaseNotFound
+			}
+		}
+		txn = a.s.KV().Write()
+		defer txn.End()
+	}
+
+	var rr *mvcc.RangeResult
+	if p.IgnoreValue || p.IgnoreLease || p.PrevKv {
+		rr, err = txn.Range(p.Key, nil, mvcc.RangeOptions{})
+		if err != nil {
+			return nil, err
+		}
+	}
+	if p.IgnoreValue || p.IgnoreLease {
+		if rr == nil || len(rr.KVs) == 0 {
+			// ignore_{lease,value} flag expects previous key-value pair
+			return nil, ErrKeyNotFound
+		}
+	}
+	if p.IgnoreValue {
+		val = rr.KVs[0].Value
+	}
+	if p.IgnoreLease {
+		leaseID = lease.LeaseID(rr.KVs[0].Lease)
+	}
+	if p.PrevKv {
+		if rr != nil && len(rr.KVs) != 0 {
+			resp.PrevKv = &rr.KVs[0]
+		}
+	}
+
+	resp.Header.Revision = txn.Put(p.Key, val, leaseID)
+	return resp, nil
+}
+
+func (a *applierV3backend) DeleteRange(txn mvcc.TxnWrite, dr *pb.DeleteRangeRequest) (*pb.DeleteRangeResponse, error) {
+	resp := &pb.DeleteRangeResponse{}
+	resp.Header = &pb.ResponseHeader{}
+	end := mkGteRange(dr.RangeEnd)
+
+	if txn == nil {
+		txn = a.s.kv.Write()
+		defer txn.End()
+	}
+
+	if dr.PrevKv {
+		rr, err := txn.Range(dr.Key, end, mvcc.RangeOptions{})
+		if err != nil {
+			return nil, err
+		}
+		if rr != nil {
+			resp.PrevKvs = make([]*mvccpb.KeyValue, len(rr.KVs))
+			for i := range rr.KVs {
+				resp.PrevKvs[i] = &rr.KVs[i]
+			}
+		}
+	}
+
+	resp.Deleted, resp.Header.Revision = txn.DeleteRange(dr.Key, end)
+	return resp, nil
+}
+
+func (a *applierV3backend) Range(txn mvcc.TxnRead, r *pb.RangeRequest) (*pb.RangeResponse, error) {
+	resp := &pb.RangeResponse{}
+	resp.Header = &pb.ResponseHeader{}
+
+	if txn == nil {
+		txn = a.s.kv.Read()
+		defer txn.End()
+	}
+
+	limit := r.Limit
+	if r.SortOrder != pb.RangeRequest_NONE ||
+		r.MinModRevision != 0 || r.MaxModRevision != 0 ||
+		r.MinCreateRevision != 0 || r.MaxCreateRevision != 0 {
+		// fetch everything; sort and truncate afterwards
+		limit = 0
+	}
+	if limit > 0 {
+		// fetch one extra for 'more' flag
+		limit = limit + 1
+	}
+
+	ro := mvcc.RangeOptions{
+		Limit: limit,
+		Rev:   r.Revision,
+		Count: r.CountOnly,
+	}
+
+	rr, err := txn.Range(r.Key, mkGteRange(r.RangeEnd), ro)
+	if err != nil {
+		return nil, err
+	}
+
+	if r.MaxModRevision != 0 {
+		f := func(kv *mvccpb.KeyValue) bool { return kv.ModRevision > r.MaxModRevision }
+		pruneKVs(rr, f)
+	}
+	if r.MinModRevision != 0 {
+		f := func(kv *mvccpb.KeyValue) bool { return kv.ModRevision < r.MinModRevision }
+		pruneKVs(rr, f)
+	}
+	if r.MaxCreateRevision != 0 {
+		f := func(kv *mvccpb.KeyValue) bool { return kv.CreateRevision > r.MaxCreateRevision }
+		pruneKVs(rr, f)
+	}
+	if r.MinCreateRevision != 0 {
+		f := func(kv *mvccpb.KeyValue) bool { return kv.CreateRevision < r.MinCreateRevision }
+		pruneKVs(rr, f)
+	}
+
+	sortOrder := r.SortOrder
+	if r.SortTarget != pb.RangeRequest_KEY && sortOrder == pb.RangeRequest_NONE {
+		// Since current mvcc.Range implementation returns results
+		// sorted by keys in lexiographically ascending order,
+		// sort ASCEND by default only when target is not 'KEY'
+		sortOrder = pb.RangeRequest_ASCEND
+	}
+	if sortOrder != pb.RangeRequest_NONE {
+		var sorter sort.Interface
+		switch {
+		case r.SortTarget == pb.RangeRequest_KEY:
+			sorter = &kvSortByKey{&kvSort{rr.KVs}}
+		case r.SortTarget == pb.RangeRequest_VERSION:
+			sorter = &kvSortByVersion{&kvSort{rr.KVs}}
+		case r.SortTarget == pb.RangeRequest_CREATE:
+			sorter = &kvSortByCreate{&kvSort{rr.KVs}}
+		case r.SortTarget == pb.RangeRequest_MOD:
+			sorter = &kvSortByMod{&kvSort{rr.KVs}}
+		case r.SortTarget == pb.RangeRequest_VALUE:
+			sorter = &kvSortByValue{&kvSort{rr.KVs}}
+		}
+		switch {
+		case sortOrder == pb.RangeRequest_ASCEND:
+			sort.Sort(sorter)
+		case sortOrder == pb.RangeRequest_DESCEND:
+			sort.Sort(sort.Reverse(sorter))
+		}
+	}
+
+	if r.Limit > 0 && len(rr.KVs) > int(r.Limit) {
+		rr.KVs = rr.KVs[:r.Limit]
+		resp.More = true
+	}
+
+	resp.Header.Revision = rr.Rev
+	resp.Count = int64(rr.Count)
+	resp.Kvs = make([]*mvccpb.KeyValue, len(rr.KVs))
+	for i := range rr.KVs {
+		if r.KeysOnly {
+			rr.KVs[i].Value = nil
+		}
+		resp.Kvs[i] = &rr.KVs[i]
+	}
+	return resp, nil
+}
+
+func (a *applierV3backend) Txn(rt *pb.TxnRequest) (*pb.TxnResponse, error) {
+	isWrite := !isTxnReadonly(rt)
+	txn := mvcc.NewReadOnlyTxnWrite(a.s.KV().Read())
+
+	txnPath := compareToPath(txn, rt)
+	if isWrite {
+		if _, err := checkRequests(txn, rt, txnPath, a.checkPut); err != nil {
+			txn.End()
+			return nil, err
+		}
+	}
+	if _, err := checkRequests(txn, rt, txnPath, a.checkRange); err != nil {
+		txn.End()
+		return nil, err
+	}
+
+	txnResp, _ := newTxnResp(rt, txnPath)
+
+	// When executing mutable txn ops, etcd must hold the txn lock so
+	// readers do not see any intermediate results. Since writes are
+	// serialized on the raft loop, the revision in the read view will
+	// be the revision of the write txn.
+	if isWrite {
+		txn.End()
+		txn = a.s.KV().Write()
+	}
+	a.applyTxn(txn, rt, txnPath, txnResp)
+	rev := txn.Rev()
+	if len(txn.Changes()) != 0 {
+		rev++
+	}
+	txn.End()
+
+	txnResp.Header.Revision = rev
+	return txnResp, nil
+}
+
+// newTxnResp allocates a txn response for a txn request given a path.
+func newTxnResp(rt *pb.TxnRequest, txnPath []bool) (txnResp *pb.TxnResponse, txnCount int) {
+	reqs := rt.Success
+	if !txnPath[0] {
+		reqs = rt.Failure
+	}
+	resps := make([]*pb.ResponseOp, len(reqs))
+	txnResp = &pb.TxnResponse{
+		Responses: resps,
+		Succeeded: txnPath[0],
+		Header:    &pb.ResponseHeader{},
+	}
+	for i, req := range reqs {
+		switch tv := req.Request.(type) {
+		case *pb.RequestOp_RequestRange:
+			resps[i] = &pb.ResponseOp{Response: &pb.ResponseOp_ResponseRange{}}
+		case *pb.RequestOp_RequestPut:
+			resps[i] = &pb.ResponseOp{Response: &pb.ResponseOp_ResponsePut{}}
+		case *pb.RequestOp_RequestDeleteRange:
+			resps[i] = &pb.ResponseOp{Response: &pb.ResponseOp_ResponseDeleteRange{}}
+		case *pb.RequestOp_RequestTxn:
+			resp, txns := newTxnResp(tv.RequestTxn, txnPath[1:])
+			resps[i] = &pb.ResponseOp{Response: &pb.ResponseOp_ResponseTxn{ResponseTxn: resp}}
+			txnPath = txnPath[1+txns:]
+			txnCount += txns + 1
+		default:
+		}
+	}
+	return txnResp, txnCount
+}
+
+func compareToPath(rv mvcc.ReadView, rt *pb.TxnRequest) []bool {
+	txnPath := make([]bool, 1)
+	ops := rt.Success
+	if txnPath[0] = applyCompares(rv, rt.Compare); !txnPath[0] {
+		ops = rt.Failure
+	}
+	for _, op := range ops {
+		tv, ok := op.Request.(*pb.RequestOp_RequestTxn)
+		if !ok || tv.RequestTxn == nil {
+			continue
+		}
+		txnPath = append(txnPath, compareToPath(rv, tv.RequestTxn)...)
+	}
+	return txnPath
+}
+
+func applyCompares(rv mvcc.ReadView, cmps []*pb.Compare) bool {
+	for _, c := range cmps {
+		if !applyCompare(rv, c) {
+			return false
+		}
+	}
+	return true
+}
+
+// applyCompare applies the compare request.
+// If the comparison succeeds, it returns true. Otherwise, returns false.
+func applyCompare(rv mvcc.ReadView, c *pb.Compare) bool {
+	// TODO: possible optimizations
+	// * chunk reads for large ranges to conserve memory
+	// * rewrite rules for common patterns:
+	//	ex. "[a, b) createrev > 0" => "limit 1 /\ kvs > 0"
+	// * caching
+	rr, err := rv.Range(c.Key, mkGteRange(c.RangeEnd), mvcc.RangeOptions{})
+	if err != nil {
+		return false
+	}
+	if len(rr.KVs) == 0 {
+		if c.Target == pb.Compare_VALUE {
+			// Always fail if comparing a value on a key/keys that doesn't exist;
+			// nil == empty string in grpc; no way to represent missing value
+			return false
+		}
+		return compareKV(c, mvccpb.KeyValue{})
+	}
+	for _, kv := range rr.KVs {
+		if !compareKV(c, kv) {
+			return false
+		}
+	}
+	return true
+}
+
+func compareKV(c *pb.Compare, ckv mvccpb.KeyValue) bool {
+	var result int
+	rev := int64(0)
+	switch c.Target {
+	case pb.Compare_VALUE:
+		v := []byte{}
+		if tv, _ := c.TargetUnion.(*pb.Compare_Value); tv != nil {
+			v = tv.Value
+		}
+		result = bytes.Compare(ckv.Value, v)
+	case pb.Compare_CREATE:
+		if tv, _ := c.TargetUnion.(*pb.Compare_CreateRevision); tv != nil {
+			rev = tv.CreateRevision
+		}
+		result = compareInt64(ckv.CreateRevision, rev)
+	case pb.Compare_MOD:
+		if tv, _ := c.TargetUnion.(*pb.Compare_ModRevision); tv != nil {
+			rev = tv.ModRevision
+		}
+		result = compareInt64(ckv.ModRevision, rev)
+	case pb.Compare_VERSION:
+		if tv, _ := c.TargetUnion.(*pb.Compare_Version); tv != nil {
+			rev = tv.Version
+		}
+		result = compareInt64(ckv.Version, rev)
+	case pb.Compare_LEASE:
+		if tv, _ := c.TargetUnion.(*pb.Compare_Lease); tv != nil {
+			rev = tv.Lease
+		}
+		result = compareInt64(ckv.Lease, rev)
+	}
+	switch c.Result {
+	case pb.Compare_EQUAL:
+		return result == 0
+	case pb.Compare_NOT_EQUAL:
+		return result != 0
+	case pb.Compare_GREATER:
+		return result > 0
+	case pb.Compare_LESS:
+		return result < 0
+	}
+	return true
+}
+
+func (a *applierV3backend) applyTxn(txn mvcc.TxnWrite, rt *pb.TxnRequest, txnPath []bool, tresp *pb.TxnResponse) (txns int) {
+	reqs := rt.Success
+	if !txnPath[0] {
+		reqs = rt.Failure
+	}
+
+	lg := a.s.getLogger()
+	for i, req := range reqs {
+		respi := tresp.Responses[i].Response
+		switch tv := req.Request.(type) {
+		case *pb.RequestOp_RequestRange:
+			resp, err := a.Range(txn, tv.RequestRange)
+			if err != nil {
+				if lg != nil {
+					lg.Panic("unexpected error during txn", zap.Error(err))
+				} else {
+					plog.Panicf("unexpected error during txn: %v", err)
+				}
+			}
+			respi.(*pb.ResponseOp_ResponseRange).ResponseRange = resp
+		case *pb.RequestOp_RequestPut:
+			resp, err := a.Put(txn, tv.RequestPut)
+			if err != nil {
+				if lg != nil {
+					lg.Panic("unexpected error during txn", zap.Error(err))
+				} else {
+					plog.Panicf("unexpected error during txn: %v", err)
+				}
+			}
+			respi.(*pb.ResponseOp_ResponsePut).ResponsePut = resp
+		case *pb.RequestOp_RequestDeleteRange:
+			resp, err := a.DeleteRange(txn, tv.RequestDeleteRange)
+			if err != nil {
+				if lg != nil {
+					lg.Panic("unexpected error during txn", zap.Error(err))
+				} else {
+					plog.Panicf("unexpected error during txn: %v", err)
+				}
+			}
+			respi.(*pb.ResponseOp_ResponseDeleteRange).ResponseDeleteRange = resp
+		case *pb.RequestOp_RequestTxn:
+			resp := respi.(*pb.ResponseOp_ResponseTxn).ResponseTxn
+			applyTxns := a.applyTxn(txn, tv.RequestTxn, txnPath[1:], resp)
+			txns += applyTxns + 1
+			txnPath = txnPath[applyTxns+1:]
+		default:
+			// empty union
+		}
+	}
+	return txns
+}
+
+func (a *applierV3backend) Compaction(compaction *pb.CompactionRequest) (*pb.CompactionResponse, <-chan struct{}, error) {
+	resp := &pb.CompactionResponse{}
+	resp.Header = &pb.ResponseHeader{}
+	ch, err := a.s.KV().Compact(compaction.Revision)
+	if err != nil {
+		return nil, ch, err
+	}
+	// get the current revision. which key to get is not important.
+	rr, _ := a.s.KV().Range([]byte("compaction"), nil, mvcc.RangeOptions{})
+	resp.Header.Revision = rr.Rev
+	return resp, ch, err
+}
+
+func (a *applierV3backend) LeaseGrant(lc *pb.LeaseGrantRequest) (*pb.LeaseGrantResponse, error) {
+	l, err := a.s.lessor.Grant(lease.LeaseID(lc.ID), lc.TTL)
+	resp := &pb.LeaseGrantResponse{}
+	if err == nil {
+		resp.ID = int64(l.ID)
+		resp.TTL = l.TTL()
+		resp.Header = newHeader(a.s)
+	}
+	return resp, err
+}
+
+func (a *applierV3backend) LeaseRevoke(lc *pb.LeaseRevokeRequest) (*pb.LeaseRevokeResponse, error) {
+	err := a.s.lessor.Revoke(lease.LeaseID(lc.ID))
+	return &pb.LeaseRevokeResponse{Header: newHeader(a.s)}, err
+}
+
+func (a *applierV3backend) LeaseCheckpoint(lc *pb.LeaseCheckpointRequest) (*pb.LeaseCheckpointResponse, error) {
+	for _, c := range lc.Checkpoints {
+		err := a.s.lessor.Checkpoint(lease.LeaseID(c.ID), c.Remaining_TTL)
+		if err != nil {
+			return &pb.LeaseCheckpointResponse{Header: newHeader(a.s)}, err
+		}
+	}
+	return &pb.LeaseCheckpointResponse{Header: newHeader(a.s)}, nil
+}
+
+func (a *applierV3backend) Alarm(ar *pb.AlarmRequest) (*pb.AlarmResponse, error) {
+	resp := &pb.AlarmResponse{}
+	oldCount := len(a.s.alarmStore.Get(ar.Alarm))
+
+	lg := a.s.getLogger()
+	switch ar.Action {
+	case pb.AlarmRequest_GET:
+		resp.Alarms = a.s.alarmStore.Get(ar.Alarm)
+	case pb.AlarmRequest_ACTIVATE:
+		m := a.s.alarmStore.Activate(types.ID(ar.MemberID), ar.Alarm)
+		if m == nil {
+			break
+		}
+		resp.Alarms = append(resp.Alarms, m)
+		activated := oldCount == 0 && len(a.s.alarmStore.Get(m.Alarm)) == 1
+		if !activated {
+			break
+		}
+
+		if lg != nil {
+			lg.Warn("alarm raised", zap.String("alarm", m.Alarm.String()), zap.String("from", types.ID(m.MemberID).String()))
+		} else {
+			plog.Warningf("alarm %v raised by peer %s", m.Alarm, types.ID(m.MemberID))
+		}
+		switch m.Alarm {
+		case pb.AlarmType_CORRUPT:
+			a.s.applyV3 = newApplierV3Corrupt(a)
+		case pb.AlarmType_NOSPACE:
+			a.s.applyV3 = newApplierV3Capped(a)
+		default:
+			if lg != nil {
+				lg.Warn("unimplemented alarm activation", zap.String("alarm", fmt.Sprintf("%+v", m)))
+			} else {
+				plog.Errorf("unimplemented alarm activation (%+v)", m)
+			}
+		}
+	case pb.AlarmRequest_DEACTIVATE:
+		m := a.s.alarmStore.Deactivate(types.ID(ar.MemberID), ar.Alarm)
+		if m == nil {
+			break
+		}
+		resp.Alarms = append(resp.Alarms, m)
+		deactivated := oldCount > 0 && len(a.s.alarmStore.Get(ar.Alarm)) == 0
+		if !deactivated {
+			break
+		}
+
+		switch m.Alarm {
+		case pb.AlarmType_NOSPACE, pb.AlarmType_CORRUPT:
+			// TODO: check kv hash before deactivating CORRUPT?
+			if lg != nil {
+				lg.Warn("alarm disarmed", zap.String("alarm", m.Alarm.String()), zap.String("from", types.ID(m.MemberID).String()))
+			} else {
+				plog.Infof("alarm disarmed %+v", ar)
+			}
+			a.s.applyV3 = a.s.newApplierV3()
+		default:
+			if lg != nil {
+				lg.Warn("unimplemented alarm deactivation", zap.String("alarm", fmt.Sprintf("%+v", m)))
+			} else {
+				plog.Errorf("unimplemented alarm deactivation (%+v)", m)
+			}
+		}
+	default:
+		return nil, nil
+	}
+	return resp, nil
+}
+
+type applierV3Capped struct {
+	applierV3
+	q backendQuota
+}
+
+// newApplierV3Capped creates an applyV3 that will reject Puts and transactions
+// with Puts so that the number of keys in the store is capped.
+func newApplierV3Capped(base applierV3) applierV3 { return &applierV3Capped{applierV3: base} }
+
+func (a *applierV3Capped) Put(txn mvcc.TxnWrite, p *pb.PutRequest) (*pb.PutResponse, error) {
+	return nil, ErrNoSpace
+}
+
+func (a *applierV3Capped) Txn(r *pb.TxnRequest) (*pb.TxnResponse, error) {
+	if a.q.Cost(r) > 0 {
+		return nil, ErrNoSpace
+	}
+	return a.applierV3.Txn(r)
+}
+
+func (a *applierV3Capped) LeaseGrant(lc *pb.LeaseGrantRequest) (*pb.LeaseGrantResponse, error) {
+	return nil, ErrNoSpace
+}
+
+func (a *applierV3backend) AuthEnable() (*pb.AuthEnableResponse, error) {
+	err := a.s.AuthStore().AuthEnable()
+	if err != nil {
+		return nil, err
+	}
+	return &pb.AuthEnableResponse{Header: newHeader(a.s)}, nil
+}
+
+func (a *applierV3backend) AuthDisable() (*pb.AuthDisableResponse, error) {
+	a.s.AuthStore().AuthDisable()
+	return &pb.AuthDisableResponse{Header: newHeader(a.s)}, nil
+}
+
+func (a *applierV3backend) Authenticate(r *pb.InternalAuthenticateRequest) (*pb.AuthenticateResponse, error) {
+	ctx := context.WithValue(context.WithValue(a.s.ctx, auth.AuthenticateParamIndex{}, a.s.consistIndex.ConsistentIndex()), auth.AuthenticateParamSimpleTokenPrefix{}, r.SimpleToken)
+	resp, err := a.s.AuthStore().Authenticate(ctx, r.Name, r.Password)
+	if resp != nil {
+		resp.Header = newHeader(a.s)
+	}
+	return resp, err
+}
+
+func (a *applierV3backend) UserAdd(r *pb.AuthUserAddRequest) (*pb.AuthUserAddResponse, error) {
+	resp, err := a.s.AuthStore().UserAdd(r)
+	if resp != nil {
+		resp.Header = newHeader(a.s)
+	}
+	return resp, err
+}
+
+func (a *applierV3backend) UserDelete(r *pb.AuthUserDeleteRequest) (*pb.AuthUserDeleteResponse, error) {
+	resp, err := a.s.AuthStore().UserDelete(r)
+	if resp != nil {
+		resp.Header = newHeader(a.s)
+	}
+	return resp, err
+}
+
+func (a *applierV3backend) UserChangePassword(r *pb.AuthUserChangePasswordRequest) (*pb.AuthUserChangePasswordResponse, error) {
+	resp, err := a.s.AuthStore().UserChangePassword(r)
+	if resp != nil {
+		resp.Header = newHeader(a.s)
+	}
+	return resp, err
+}
+
+func (a *applierV3backend) UserGrantRole(r *pb.AuthUserGrantRoleRequest) (*pb.AuthUserGrantRoleResponse, error) {
+	resp, err := a.s.AuthStore().UserGrantRole(r)
+	if resp != nil {
+		resp.Header = newHeader(a.s)
+	}
+	return resp, err
+}
+
+func (a *applierV3backend) UserGet(r *pb.AuthUserGetRequest) (*pb.AuthUserGetResponse, error) {
+	resp, err := a.s.AuthStore().UserGet(r)
+	if resp != nil {
+		resp.Header = newHeader(a.s)
+	}
+	return resp, err
+}
+
+func (a *applierV3backend) UserRevokeRole(r *pb.AuthUserRevokeRoleRequest) (*pb.AuthUserRevokeRoleResponse, error) {
+	resp, err := a.s.AuthStore().UserRevokeRole(r)
+	if resp != nil {
+		resp.Header = newHeader(a.s)
+	}
+	return resp, err
+}
+
+func (a *applierV3backend) RoleAdd(r *pb.AuthRoleAddRequest) (*pb.AuthRoleAddResponse, error) {
+	resp, err := a.s.AuthStore().RoleAdd(r)
+	if resp != nil {
+		resp.Header = newHeader(a.s)
+	}
+	return resp, err
+}
+
+func (a *applierV3backend) RoleGrantPermission(r *pb.AuthRoleGrantPermissionRequest) (*pb.AuthRoleGrantPermissionResponse, error) {
+	resp, err := a.s.AuthStore().RoleGrantPermission(r)
+	if resp != nil {
+		resp.Header = newHeader(a.s)
+	}
+	return resp, err
+}
+
+func (a *applierV3backend) RoleGet(r *pb.AuthRoleGetRequest) (*pb.AuthRoleGetResponse, error) {
+	resp, err := a.s.AuthStore().RoleGet(r)
+	if resp != nil {
+		resp.Header = newHeader(a.s)
+	}
+	return resp, err
+}
+
+func (a *applierV3backend) RoleRevokePermission(r *pb.AuthRoleRevokePermissionRequest) (*pb.AuthRoleRevokePermissionResponse, error) {
+	resp, err := a.s.AuthStore().RoleRevokePermission(r)
+	if resp != nil {
+		resp.Header = newHeader(a.s)
+	}
+	return resp, err
+}
+
+func (a *applierV3backend) RoleDelete(r *pb.AuthRoleDeleteRequest) (*pb.AuthRoleDeleteResponse, error) {
+	resp, err := a.s.AuthStore().RoleDelete(r)
+	if resp != nil {
+		resp.Header = newHeader(a.s)
+	}
+	return resp, err
+}
+
+func (a *applierV3backend) UserList(r *pb.AuthUserListRequest) (*pb.AuthUserListResponse, error) {
+	resp, err := a.s.AuthStore().UserList(r)
+	if resp != nil {
+		resp.Header = newHeader(a.s)
+	}
+	return resp, err
+}
+
+func (a *applierV3backend) RoleList(r *pb.AuthRoleListRequest) (*pb.AuthRoleListResponse, error) {
+	resp, err := a.s.AuthStore().RoleList(r)
+	if resp != nil {
+		resp.Header = newHeader(a.s)
+	}
+	return resp, err
+}
+
+type quotaApplierV3 struct {
+	applierV3
+	q Quota
+}
+
+func newQuotaApplierV3(s *EtcdServer, app applierV3) applierV3 {
+	return &quotaApplierV3{app, NewBackendQuota(s, "v3-applier")}
+}
+
+func (a *quotaApplierV3) Put(txn mvcc.TxnWrite, p *pb.PutRequest) (*pb.PutResponse, error) {
+	ok := a.q.Available(p)
+	resp, err := a.applierV3.Put(txn, p)
+	if err == nil && !ok {
+		err = ErrNoSpace
+	}
+	return resp, err
+}
+
+func (a *quotaApplierV3) Txn(rt *pb.TxnRequest) (*pb.TxnResponse, error) {
+	ok := a.q.Available(rt)
+	resp, err := a.applierV3.Txn(rt)
+	if err == nil && !ok {
+		err = ErrNoSpace
+	}
+	return resp, err
+}
+
+func (a *quotaApplierV3) LeaseGrant(lc *pb.LeaseGrantRequest) (*pb.LeaseGrantResponse, error) {
+	ok := a.q.Available(lc)
+	resp, err := a.applierV3.LeaseGrant(lc)
+	if err == nil && !ok {
+		err = ErrNoSpace
+	}
+	return resp, err
+}
+
+type kvSort struct{ kvs []mvccpb.KeyValue }
+
+func (s *kvSort) Swap(i, j int) {
+	t := s.kvs[i]
+	s.kvs[i] = s.kvs[j]
+	s.kvs[j] = t
+}
+func (s *kvSort) Len() int { return len(s.kvs) }
+
+type kvSortByKey struct{ *kvSort }
+
+func (s *kvSortByKey) Less(i, j int) bool {
+	return bytes.Compare(s.kvs[i].Key, s.kvs[j].Key) < 0
+}
+
+type kvSortByVersion struct{ *kvSort }
+
+func (s *kvSortByVersion) Less(i, j int) bool {
+	return (s.kvs[i].Version - s.kvs[j].Version) < 0
+}
+
+type kvSortByCreate struct{ *kvSort }
+
+func (s *kvSortByCreate) Less(i, j int) bool {
+	return (s.kvs[i].CreateRevision - s.kvs[j].CreateRevision) < 0
+}
+
+type kvSortByMod struct{ *kvSort }
+
+func (s *kvSortByMod) Less(i, j int) bool {
+	return (s.kvs[i].ModRevision - s.kvs[j].ModRevision) < 0
+}
+
+type kvSortByValue struct{ *kvSort }
+
+func (s *kvSortByValue) Less(i, j int) bool {
+	return bytes.Compare(s.kvs[i].Value, s.kvs[j].Value) < 0
+}
+
+func checkRequests(rv mvcc.ReadView, rt *pb.TxnRequest, txnPath []bool, f checkReqFunc) (int, error) {
+	txnCount := 0
+	reqs := rt.Success
+	if !txnPath[0] {
+		reqs = rt.Failure
+	}
+	for _, req := range reqs {
+		if tv, ok := req.Request.(*pb.RequestOp_RequestTxn); ok && tv.RequestTxn != nil {
+			txns, err := checkRequests(rv, tv.RequestTxn, txnPath[1:], f)
+			if err != nil {
+				return 0, err
+			}
+			txnCount += txns + 1
+			txnPath = txnPath[txns+1:]
+			continue
+		}
+		if err := f(rv, req); err != nil {
+			return 0, err
+		}
+	}
+	return txnCount, nil
+}
+
+func (a *applierV3backend) checkRequestPut(rv mvcc.ReadView, reqOp *pb.RequestOp) error {
+	tv, ok := reqOp.Request.(*pb.RequestOp_RequestPut)
+	if !ok || tv.RequestPut == nil {
+		return nil
+	}
+	req := tv.RequestPut
+	if req.IgnoreValue || req.IgnoreLease {
+		// expects previous key-value, error if not exist
+		rr, err := rv.Range(req.Key, nil, mvcc.RangeOptions{})
+		if err != nil {
+			return err
+		}
+		if rr == nil || len(rr.KVs) == 0 {
+			return ErrKeyNotFound
+		}
+	}
+	if lease.LeaseID(req.Lease) != lease.NoLease {
+		if l := a.s.lessor.Lookup(lease.LeaseID(req.Lease)); l == nil {
+			return lease.ErrLeaseNotFound
+		}
+	}
+	return nil
+}
+
+func (a *applierV3backend) checkRequestRange(rv mvcc.ReadView, reqOp *pb.RequestOp) error {
+	tv, ok := reqOp.Request.(*pb.RequestOp_RequestRange)
+	if !ok || tv.RequestRange == nil {
+		return nil
+	}
+	req := tv.RequestRange
+	switch {
+	case req.Revision == 0:
+		return nil
+	case req.Revision > rv.Rev():
+		return mvcc.ErrFutureRev
+	case req.Revision < rv.FirstRev():
+		return mvcc.ErrCompacted
+	}
+	return nil
+}
+
+func compareInt64(a, b int64) int {
+	switch {
+	case a < b:
+		return -1
+	case a > b:
+		return 1
+	default:
+		return 0
+	}
+}
+
+// mkGteRange determines if the range end is a >= range. This works around grpc
+// sending empty byte strings as nil; >= is encoded in the range end as '\0'.
+// If it is a GTE range, then []byte{} is returned to indicate the empty byte
+// string (vs nil being no byte string).
+func mkGteRange(rangeEnd []byte) []byte {
+	if len(rangeEnd) == 1 && rangeEnd[0] == 0 {
+		return []byte{}
+	}
+	return rangeEnd
+}
+
+func noSideEffect(r *pb.InternalRaftRequest) bool {
+	return r.Range != nil || r.AuthUserGet != nil || r.AuthRoleGet != nil
+}
+
+func removeNeedlessRangeReqs(txn *pb.TxnRequest) {
+	f := func(ops []*pb.RequestOp) []*pb.RequestOp {
+		j := 0
+		for i := 0; i < len(ops); i++ {
+			if _, ok := ops[i].Request.(*pb.RequestOp_RequestRange); ok {
+				continue
+			}
+			ops[j] = ops[i]
+			j++
+		}
+
+		return ops[:j]
+	}
+
+	txn.Success = f(txn.Success)
+	txn.Failure = f(txn.Failure)
+}
+
+func pruneKVs(rr *mvcc.RangeResult, isPrunable func(*mvccpb.KeyValue) bool) {
+	j := 0
+	for i := range rr.KVs {
+		rr.KVs[j] = rr.KVs[i]
+		if !isPrunable(&rr.KVs[i]) {
+			j++
+		}
+	}
+	rr.KVs = rr.KVs[:j]
+}
+
+func newHeader(s *EtcdServer) *pb.ResponseHeader {
+	return &pb.ResponseHeader{
+		ClusterId: uint64(s.Cluster().ID()),
+		MemberId:  uint64(s.ID()),
+		Revision:  s.KV().Rev(),
+		RaftTerm:  s.Term(),
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/apply_auth.go b/vendor/go.etcd.io/etcd/etcdserver/apply_auth.go
new file mode 100644
index 0000000..4b094ad
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/apply_auth.go
@@ -0,0 +1,242 @@
+// Copyright 2016 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import (
+	"sync"
+
+	"go.etcd.io/etcd/auth"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/lease"
+	"go.etcd.io/etcd/mvcc"
+)
+
+type authApplierV3 struct {
+	applierV3
+	as     auth.AuthStore
+	lessor lease.Lessor
+
+	// mu serializes Apply so that user isn't corrupted and so that
+	// serialized requests don't leak data from TOCTOU errors
+	mu sync.Mutex
+
+	authInfo auth.AuthInfo
+}
+
+func newAuthApplierV3(as auth.AuthStore, base applierV3, lessor lease.Lessor) *authApplierV3 {
+	return &authApplierV3{applierV3: base, as: as, lessor: lessor}
+}
+
+func (aa *authApplierV3) Apply(r *pb.InternalRaftRequest) *applyResult {
+	aa.mu.Lock()
+	defer aa.mu.Unlock()
+	if r.Header != nil {
+		// backward-compatible with pre-3.0 releases when internalRaftRequest
+		// does not have header field
+		aa.authInfo.Username = r.Header.Username
+		aa.authInfo.Revision = r.Header.AuthRevision
+	}
+	if needAdminPermission(r) {
+		if err := aa.as.IsAdminPermitted(&aa.authInfo); err != nil {
+			aa.authInfo.Username = ""
+			aa.authInfo.Revision = 0
+			return &applyResult{err: err}
+		}
+	}
+	ret := aa.applierV3.Apply(r)
+	aa.authInfo.Username = ""
+	aa.authInfo.Revision = 0
+	return ret
+}
+
+func (aa *authApplierV3) Put(txn mvcc.TxnWrite, r *pb.PutRequest) (*pb.PutResponse, error) {
+	if err := aa.as.IsPutPermitted(&aa.authInfo, r.Key); err != nil {
+		return nil, err
+	}
+
+	if err := aa.checkLeasePuts(lease.LeaseID(r.Lease)); err != nil {
+		// The specified lease is already attached with a key that cannot
+		// be written by this user. It means the user cannot revoke the
+		// lease so attaching the lease to the newly written key should
+		// be forbidden.
+		return nil, err
+	}
+
+	if r.PrevKv {
+		err := aa.as.IsRangePermitted(&aa.authInfo, r.Key, nil)
+		if err != nil {
+			return nil, err
+		}
+	}
+	return aa.applierV3.Put(txn, r)
+}
+
+func (aa *authApplierV3) Range(txn mvcc.TxnRead, r *pb.RangeRequest) (*pb.RangeResponse, error) {
+	if err := aa.as.IsRangePermitted(&aa.authInfo, r.Key, r.RangeEnd); err != nil {
+		return nil, err
+	}
+	return aa.applierV3.Range(txn, r)
+}
+
+func (aa *authApplierV3) DeleteRange(txn mvcc.TxnWrite, r *pb.DeleteRangeRequest) (*pb.DeleteRangeResponse, error) {
+	if err := aa.as.IsDeleteRangePermitted(&aa.authInfo, r.Key, r.RangeEnd); err != nil {
+		return nil, err
+	}
+	if r.PrevKv {
+		err := aa.as.IsRangePermitted(&aa.authInfo, r.Key, r.RangeEnd)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	return aa.applierV3.DeleteRange(txn, r)
+}
+
+func checkTxnReqsPermission(as auth.AuthStore, ai *auth.AuthInfo, reqs []*pb.RequestOp) error {
+	for _, requ := range reqs {
+		switch tv := requ.Request.(type) {
+		case *pb.RequestOp_RequestRange:
+			if tv.RequestRange == nil {
+				continue
+			}
+
+			if err := as.IsRangePermitted(ai, tv.RequestRange.Key, tv.RequestRange.RangeEnd); err != nil {
+				return err
+			}
+
+		case *pb.RequestOp_RequestPut:
+			if tv.RequestPut == nil {
+				continue
+			}
+
+			if err := as.IsPutPermitted(ai, tv.RequestPut.Key); err != nil {
+				return err
+			}
+
+		case *pb.RequestOp_RequestDeleteRange:
+			if tv.RequestDeleteRange == nil {
+				continue
+			}
+
+			if tv.RequestDeleteRange.PrevKv {
+				err := as.IsRangePermitted(ai, tv.RequestDeleteRange.Key, tv.RequestDeleteRange.RangeEnd)
+				if err != nil {
+					return err
+				}
+			}
+
+			err := as.IsDeleteRangePermitted(ai, tv.RequestDeleteRange.Key, tv.RequestDeleteRange.RangeEnd)
+			if err != nil {
+				return err
+			}
+		}
+	}
+
+	return nil
+}
+
+func checkTxnAuth(as auth.AuthStore, ai *auth.AuthInfo, rt *pb.TxnRequest) error {
+	for _, c := range rt.Compare {
+		if err := as.IsRangePermitted(ai, c.Key, c.RangeEnd); err != nil {
+			return err
+		}
+	}
+	if err := checkTxnReqsPermission(as, ai, rt.Success); err != nil {
+		return err
+	}
+	return checkTxnReqsPermission(as, ai, rt.Failure)
+}
+
+func (aa *authApplierV3) Txn(rt *pb.TxnRequest) (*pb.TxnResponse, error) {
+	if err := checkTxnAuth(aa.as, &aa.authInfo, rt); err != nil {
+		return nil, err
+	}
+	return aa.applierV3.Txn(rt)
+}
+
+func (aa *authApplierV3) LeaseRevoke(lc *pb.LeaseRevokeRequest) (*pb.LeaseRevokeResponse, error) {
+	if err := aa.checkLeasePuts(lease.LeaseID(lc.ID)); err != nil {
+		return nil, err
+	}
+	return aa.applierV3.LeaseRevoke(lc)
+}
+
+func (aa *authApplierV3) checkLeasePuts(leaseID lease.LeaseID) error {
+	lease := aa.lessor.Lookup(leaseID)
+	if lease != nil {
+		for _, key := range lease.Keys() {
+			if err := aa.as.IsPutPermitted(&aa.authInfo, []byte(key)); err != nil {
+				return err
+			}
+		}
+	}
+
+	return nil
+}
+
+func (aa *authApplierV3) UserGet(r *pb.AuthUserGetRequest) (*pb.AuthUserGetResponse, error) {
+	err := aa.as.IsAdminPermitted(&aa.authInfo)
+	if err != nil && r.Name != aa.authInfo.Username {
+		aa.authInfo.Username = ""
+		aa.authInfo.Revision = 0
+		return &pb.AuthUserGetResponse{}, err
+	}
+
+	return aa.applierV3.UserGet(r)
+}
+
+func (aa *authApplierV3) RoleGet(r *pb.AuthRoleGetRequest) (*pb.AuthRoleGetResponse, error) {
+	err := aa.as.IsAdminPermitted(&aa.authInfo)
+	if err != nil && !aa.as.HasRole(aa.authInfo.Username, r.Role) {
+		aa.authInfo.Username = ""
+		aa.authInfo.Revision = 0
+		return &pb.AuthRoleGetResponse{}, err
+	}
+
+	return aa.applierV3.RoleGet(r)
+}
+
+func needAdminPermission(r *pb.InternalRaftRequest) bool {
+	switch {
+	case r.AuthEnable != nil:
+		return true
+	case r.AuthDisable != nil:
+		return true
+	case r.AuthUserAdd != nil:
+		return true
+	case r.AuthUserDelete != nil:
+		return true
+	case r.AuthUserChangePassword != nil:
+		return true
+	case r.AuthUserGrantRole != nil:
+		return true
+	case r.AuthUserRevokeRole != nil:
+		return true
+	case r.AuthRoleAdd != nil:
+		return true
+	case r.AuthRoleGrantPermission != nil:
+		return true
+	case r.AuthRoleRevokePermission != nil:
+		return true
+	case r.AuthRoleDelete != nil:
+		return true
+	case r.AuthUserList != nil:
+		return true
+	case r.AuthRoleList != nil:
+		return true
+	default:
+		return false
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/apply_v2.go b/vendor/go.etcd.io/etcd/etcdserver/apply_v2.go
new file mode 100644
index 0000000..c77df19
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/apply_v2.go
@@ -0,0 +1,147 @@
+// Copyright 2016 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import (
+	"encoding/json"
+	"path"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver/api"
+	"go.etcd.io/etcd/etcdserver/api/membership"
+	"go.etcd.io/etcd/etcdserver/api/v2store"
+	"go.etcd.io/etcd/pkg/pbutil"
+
+	"github.com/coreos/go-semver/semver"
+	"go.uber.org/zap"
+)
+
+// ApplierV2 is the interface for processing V2 raft messages
+type ApplierV2 interface {
+	Delete(r *RequestV2) Response
+	Post(r *RequestV2) Response
+	Put(r *RequestV2) Response
+	QGet(r *RequestV2) Response
+	Sync(r *RequestV2) Response
+}
+
+func NewApplierV2(lg *zap.Logger, s v2store.Store, c *membership.RaftCluster) ApplierV2 {
+	return &applierV2store{lg: lg, store: s, cluster: c}
+}
+
+type applierV2store struct {
+	lg      *zap.Logger
+	store   v2store.Store
+	cluster *membership.RaftCluster
+}
+
+func (a *applierV2store) Delete(r *RequestV2) Response {
+	switch {
+	case r.PrevIndex > 0 || r.PrevValue != "":
+		return toResponse(a.store.CompareAndDelete(r.Path, r.PrevValue, r.PrevIndex))
+	default:
+		return toResponse(a.store.Delete(r.Path, r.Dir, r.Recursive))
+	}
+}
+
+func (a *applierV2store) Post(r *RequestV2) Response {
+	return toResponse(a.store.Create(r.Path, r.Dir, r.Val, true, r.TTLOptions()))
+}
+
+func (a *applierV2store) Put(r *RequestV2) Response {
+	ttlOptions := r.TTLOptions()
+	exists, existsSet := pbutil.GetBool(r.PrevExist)
+	switch {
+	case existsSet:
+		if exists {
+			if r.PrevIndex == 0 && r.PrevValue == "" {
+				return toResponse(a.store.Update(r.Path, r.Val, ttlOptions))
+			}
+			return toResponse(a.store.CompareAndSwap(r.Path, r.PrevValue, r.PrevIndex, r.Val, ttlOptions))
+		}
+		return toResponse(a.store.Create(r.Path, r.Dir, r.Val, false, ttlOptions))
+	case r.PrevIndex > 0 || r.PrevValue != "":
+		return toResponse(a.store.CompareAndSwap(r.Path, r.PrevValue, r.PrevIndex, r.Val, ttlOptions))
+	default:
+		if storeMemberAttributeRegexp.MatchString(r.Path) {
+			id := membership.MustParseMemberIDFromKey(path.Dir(r.Path))
+			var attr membership.Attributes
+			if err := json.Unmarshal([]byte(r.Val), &attr); err != nil {
+				if a.lg != nil {
+					a.lg.Panic("failed to unmarshal", zap.String("value", r.Val), zap.Error(err))
+				} else {
+					plog.Panicf("unmarshal %s should never fail: %v", r.Val, err)
+				}
+			}
+			if a.cluster != nil {
+				a.cluster.UpdateAttributes(id, attr)
+			}
+			// return an empty response since there is no consumer.
+			return Response{}
+		}
+		if r.Path == membership.StoreClusterVersionKey() {
+			if a.cluster != nil {
+				a.cluster.SetVersion(semver.Must(semver.NewVersion(r.Val)), api.UpdateCapability)
+			}
+			// return an empty response since there is no consumer.
+			return Response{}
+		}
+		return toResponse(a.store.Set(r.Path, r.Dir, r.Val, ttlOptions))
+	}
+}
+
+func (a *applierV2store) QGet(r *RequestV2) Response {
+	return toResponse(a.store.Get(r.Path, r.Recursive, r.Sorted))
+}
+
+func (a *applierV2store) Sync(r *RequestV2) Response {
+	a.store.DeleteExpiredKeys(time.Unix(0, r.Time))
+	return Response{}
+}
+
+// applyV2Request interprets r as a call to v2store.X
+// and returns a Response interpreted from v2store.Event
+func (s *EtcdServer) applyV2Request(r *RequestV2) Response {
+	defer warnOfExpensiveRequest(s.getLogger(), time.Now(), r, nil, nil)
+
+	switch r.Method {
+	case "POST":
+		return s.applyV2.Post(r)
+	case "PUT":
+		return s.applyV2.Put(r)
+	case "DELETE":
+		return s.applyV2.Delete(r)
+	case "QGET":
+		return s.applyV2.QGet(r)
+	case "SYNC":
+		return s.applyV2.Sync(r)
+	default:
+		// This should never be reached, but just in case:
+		return Response{Err: ErrUnknownMethod}
+	}
+}
+
+func (r *RequestV2) TTLOptions() v2store.TTLOptionSet {
+	refresh, _ := pbutil.GetBool(r.Refresh)
+	ttlOptions := v2store.TTLOptionSet{Refresh: refresh}
+	if r.Expiration != 0 {
+		ttlOptions.ExpireTime = time.Unix(0, r.Expiration)
+	}
+	return ttlOptions
+}
+
+func toResponse(ev *v2store.Event, err error) Response {
+	return Response{Event: ev, Err: err}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/backend.go b/vendor/go.etcd.io/etcd/etcdserver/backend.go
new file mode 100644
index 0000000..01ba192
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/backend.go
@@ -0,0 +1,112 @@
+// Copyright 2017 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import (
+	"fmt"
+	"os"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver/api/snap"
+	"go.etcd.io/etcd/lease"
+	"go.etcd.io/etcd/mvcc"
+	"go.etcd.io/etcd/mvcc/backend"
+	"go.etcd.io/etcd/raft/raftpb"
+
+	"go.uber.org/zap"
+)
+
+func newBackend(cfg ServerConfig) backend.Backend {
+	bcfg := backend.DefaultBackendConfig()
+	bcfg.Path = cfg.backendPath()
+	if cfg.BackendBatchLimit != 0 {
+		bcfg.BatchLimit = cfg.BackendBatchLimit
+		if cfg.Logger != nil {
+			cfg.Logger.Info("setting backend batch limit", zap.Int("batch limit", cfg.BackendBatchLimit))
+		}
+	}
+	if cfg.BackendBatchInterval != 0 {
+		bcfg.BatchInterval = cfg.BackendBatchInterval
+		if cfg.Logger != nil {
+			cfg.Logger.Info("setting backend batch interval", zap.Duration("batch interval", cfg.BackendBatchInterval))
+		}
+	}
+	bcfg.BackendFreelistType = cfg.BackendFreelistType
+	bcfg.Logger = cfg.Logger
+	if cfg.QuotaBackendBytes > 0 && cfg.QuotaBackendBytes != DefaultQuotaBytes {
+		// permit 10% excess over quota for disarm
+		bcfg.MmapSize = uint64(cfg.QuotaBackendBytes + cfg.QuotaBackendBytes/10)
+	}
+	return backend.New(bcfg)
+}
+
+// openSnapshotBackend renames a snapshot db to the current etcd db and opens it.
+func openSnapshotBackend(cfg ServerConfig, ss *snap.Snapshotter, snapshot raftpb.Snapshot) (backend.Backend, error) {
+	snapPath, err := ss.DBFilePath(snapshot.Metadata.Index)
+	if err != nil {
+		return nil, fmt.Errorf("failed to find database snapshot file (%v)", err)
+	}
+	if err := os.Rename(snapPath, cfg.backendPath()); err != nil {
+		return nil, fmt.Errorf("failed to rename database snapshot file (%v)", err)
+	}
+	return openBackend(cfg), nil
+}
+
+// openBackend returns a backend using the current etcd db.
+func openBackend(cfg ServerConfig) backend.Backend {
+	fn := cfg.backendPath()
+
+	now, beOpened := time.Now(), make(chan backend.Backend)
+	go func() {
+		beOpened <- newBackend(cfg)
+	}()
+
+	select {
+	case be := <-beOpened:
+		if cfg.Logger != nil {
+			cfg.Logger.Info("opened backend db", zap.String("path", fn), zap.Duration("took", time.Since(now)))
+		}
+		return be
+
+	case <-time.After(10 * time.Second):
+		if cfg.Logger != nil {
+			cfg.Logger.Info(
+				"db file is flocked by another process, or taking too long",
+				zap.String("path", fn),
+				zap.Duration("took", time.Since(now)),
+			)
+		} else {
+			plog.Warningf("another etcd process is using %q and holds the file lock, or loading backend file is taking >10 seconds", fn)
+			plog.Warningf("waiting for it to exit before starting...")
+		}
+	}
+
+	return <-beOpened
+}
+
+// recoverBackendSnapshot recovers the DB from a snapshot in case etcd crashes
+// before updating the backend db after persisting raft snapshot to disk,
+// violating the invariant snapshot.Metadata.Index < db.consistentIndex. In this
+// case, replace the db with the snapshot db sent by the leader.
+func recoverSnapshotBackend(cfg ServerConfig, oldbe backend.Backend, snapshot raftpb.Snapshot) (backend.Backend, error) {
+	var cIndex consistentIndex
+	kv := mvcc.New(cfg.Logger, oldbe, &lease.FakeLessor{}, &cIndex, mvcc.StoreConfig{CompactionBatchLimit: cfg.CompactionBatchLimit})
+	defer kv.Close()
+	if snapshot.Metadata.Index <= kv.ConsistentIndex() {
+		return oldbe, nil
+	}
+	oldbe.Close()
+	return openSnapshotBackend(cfg, snap.New(cfg.Logger, cfg.SnapDir()), snapshot)
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/cluster_util.go b/vendor/go.etcd.io/etcd/etcdserver/cluster_util.go
new file mode 100644
index 0000000..f92706c
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/cluster_util.go
@@ -0,0 +1,407 @@
+// Copyright 2015 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import (
+	"context"
+	"encoding/json"
+	"fmt"
+	"io/ioutil"
+	"net/http"
+	"sort"
+	"strings"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver/api/membership"
+	"go.etcd.io/etcd/pkg/types"
+	"go.etcd.io/etcd/version"
+
+	"github.com/coreos/go-semver/semver"
+	"go.uber.org/zap"
+)
+
+// isMemberBootstrapped tries to check if the given member has been bootstrapped
+// in the given cluster.
+func isMemberBootstrapped(lg *zap.Logger, cl *membership.RaftCluster, member string, rt http.RoundTripper, timeout time.Duration) bool {
+	rcl, err := getClusterFromRemotePeers(lg, getRemotePeerURLs(cl, member), timeout, false, rt)
+	if err != nil {
+		return false
+	}
+	id := cl.MemberByName(member).ID
+	m := rcl.Member(id)
+	if m == nil {
+		return false
+	}
+	if len(m.ClientURLs) > 0 {
+		return true
+	}
+	return false
+}
+
+// GetClusterFromRemotePeers takes a set of URLs representing etcd peers, and
+// attempts to construct a Cluster by accessing the members endpoint on one of
+// these URLs. The first URL to provide a response is used. If no URLs provide
+// a response, or a Cluster cannot be successfully created from a received
+// response, an error is returned.
+// Each request has a 10-second timeout. Because the upper limit of TTL is 5s,
+// 10 second is enough for building connection and finishing request.
+func GetClusterFromRemotePeers(lg *zap.Logger, urls []string, rt http.RoundTripper) (*membership.RaftCluster, error) {
+	return getClusterFromRemotePeers(lg, urls, 10*time.Second, true, rt)
+}
+
+// If logerr is true, it prints out more error messages.
+func getClusterFromRemotePeers(lg *zap.Logger, urls []string, timeout time.Duration, logerr bool, rt http.RoundTripper) (*membership.RaftCluster, error) {
+	cc := &http.Client{
+		Transport: rt,
+		Timeout:   timeout,
+	}
+	for _, u := range urls {
+		addr := u + "/members"
+		resp, err := cc.Get(addr)
+		if err != nil {
+			if logerr {
+				if lg != nil {
+					lg.Warn("failed to get cluster response", zap.String("address", addr), zap.Error(err))
+				} else {
+					plog.Warningf("could not get cluster response from %s: %v", u, err)
+				}
+			}
+			continue
+		}
+		b, err := ioutil.ReadAll(resp.Body)
+		resp.Body.Close()
+		if err != nil {
+			if logerr {
+				if lg != nil {
+					lg.Warn("failed to read body of cluster response", zap.String("address", addr), zap.Error(err))
+				} else {
+					plog.Warningf("could not read the body of cluster response: %v", err)
+				}
+			}
+			continue
+		}
+		var membs []*membership.Member
+		if err = json.Unmarshal(b, &membs); err != nil {
+			if logerr {
+				if lg != nil {
+					lg.Warn("failed to unmarshal cluster response", zap.String("address", addr), zap.Error(err))
+				} else {
+					plog.Warningf("could not unmarshal cluster response: %v", err)
+				}
+			}
+			continue
+		}
+		id, err := types.IDFromString(resp.Header.Get("X-Etcd-Cluster-ID"))
+		if err != nil {
+			if logerr {
+				if lg != nil {
+					lg.Warn(
+						"failed to parse cluster ID",
+						zap.String("address", addr),
+						zap.String("header", resp.Header.Get("X-Etcd-Cluster-ID")),
+						zap.Error(err),
+					)
+				} else {
+					plog.Warningf("could not parse the cluster ID from cluster res: %v", err)
+				}
+			}
+			continue
+		}
+
+		// check the length of membership members
+		// if the membership members are present then prepare and return raft cluster
+		// if membership members are not present then the raft cluster formed will be
+		// an invalid empty cluster hence return failed to get raft cluster member(s) from the given urls error
+		if len(membs) > 0 {
+			return membership.NewClusterFromMembers(lg, "", id, membs), nil
+		}
+		return nil, fmt.Errorf("failed to get raft cluster member(s) from the given URLs")
+	}
+	return nil, fmt.Errorf("could not retrieve cluster information from the given URLs")
+}
+
+// getRemotePeerURLs returns peer urls of remote members in the cluster. The
+// returned list is sorted in ascending lexicographical order.
+func getRemotePeerURLs(cl *membership.RaftCluster, local string) []string {
+	us := make([]string, 0)
+	for _, m := range cl.Members() {
+		if m.Name == local {
+			continue
+		}
+		us = append(us, m.PeerURLs...)
+	}
+	sort.Strings(us)
+	return us
+}
+
+// getVersions returns the versions of the members in the given cluster.
+// The key of the returned map is the member's ID. The value of the returned map
+// is the semver versions string, including server and cluster.
+// If it fails to get the version of a member, the key will be nil.
+func getVersions(lg *zap.Logger, cl *membership.RaftCluster, local types.ID, rt http.RoundTripper) map[string]*version.Versions {
+	members := cl.Members()
+	vers := make(map[string]*version.Versions)
+	for _, m := range members {
+		if m.ID == local {
+			cv := "not_decided"
+			if cl.Version() != nil {
+				cv = cl.Version().String()
+			}
+			vers[m.ID.String()] = &version.Versions{Server: version.Version, Cluster: cv}
+			continue
+		}
+		ver, err := getVersion(lg, m, rt)
+		if err != nil {
+			if lg != nil {
+				lg.Warn("failed to get version", zap.String("remote-member-id", m.ID.String()), zap.Error(err))
+			} else {
+				plog.Warningf("cannot get the version of member %s (%v)", m.ID, err)
+			}
+			vers[m.ID.String()] = nil
+		} else {
+			vers[m.ID.String()] = ver
+		}
+	}
+	return vers
+}
+
+// decideClusterVersion decides the cluster version based on the versions map.
+// The returned version is the min server version in the map, or nil if the min
+// version in unknown.
+func decideClusterVersion(lg *zap.Logger, vers map[string]*version.Versions) *semver.Version {
+	var cv *semver.Version
+	lv := semver.Must(semver.NewVersion(version.Version))
+
+	for mid, ver := range vers {
+		if ver == nil {
+			return nil
+		}
+		v, err := semver.NewVersion(ver.Server)
+		if err != nil {
+			if lg != nil {
+				lg.Warn(
+					"failed to parse server version of remote member",
+					zap.String("remote-member-id", mid),
+					zap.String("remote-member-version", ver.Server),
+					zap.Error(err),
+				)
+			} else {
+				plog.Errorf("cannot understand the version of member %s (%v)", mid, err)
+			}
+			return nil
+		}
+		if lv.LessThan(*v) {
+			if lg != nil {
+				lg.Warn(
+					"leader found higher-versioned member",
+					zap.String("local-member-version", lv.String()),
+					zap.String("remote-member-id", mid),
+					zap.String("remote-member-version", ver.Server),
+				)
+			} else {
+				plog.Warningf("the local etcd version %s is not up-to-date", lv.String())
+				plog.Warningf("member %s has a higher version %s", mid, ver.Server)
+			}
+		}
+		if cv == nil {
+			cv = v
+		} else if v.LessThan(*cv) {
+			cv = v
+		}
+	}
+	return cv
+}
+
+// isCompatibleWithCluster return true if the local member has a compatible version with
+// the current running cluster.
+// The version is considered as compatible when at least one of the other members in the cluster has a
+// cluster version in the range of [MinClusterVersion, Version] and no known members has a cluster version
+// out of the range.
+// We set this rule since when the local member joins, another member might be offline.
+func isCompatibleWithCluster(lg *zap.Logger, cl *membership.RaftCluster, local types.ID, rt http.RoundTripper) bool {
+	vers := getVersions(lg, cl, local, rt)
+	minV := semver.Must(semver.NewVersion(version.MinClusterVersion))
+	maxV := semver.Must(semver.NewVersion(version.Version))
+	maxV = &semver.Version{
+		Major: maxV.Major,
+		Minor: maxV.Minor,
+	}
+	return isCompatibleWithVers(lg, vers, local, minV, maxV)
+}
+
+func isCompatibleWithVers(lg *zap.Logger, vers map[string]*version.Versions, local types.ID, minV, maxV *semver.Version) bool {
+	var ok bool
+	for id, v := range vers {
+		// ignore comparison with local version
+		if id == local.String() {
+			continue
+		}
+		if v == nil {
+			continue
+		}
+		clusterv, err := semver.NewVersion(v.Cluster)
+		if err != nil {
+			if lg != nil {
+				lg.Warn(
+					"failed to parse cluster version of remote member",
+					zap.String("remote-member-id", id),
+					zap.String("remote-member-cluster-version", v.Cluster),
+					zap.Error(err),
+				)
+			} else {
+				plog.Errorf("cannot understand the cluster version of member %s (%v)", id, err)
+			}
+			continue
+		}
+		if clusterv.LessThan(*minV) {
+			if lg != nil {
+				lg.Warn(
+					"cluster version of remote member is not compatible; too low",
+					zap.String("remote-member-id", id),
+					zap.String("remote-member-cluster-version", clusterv.String()),
+					zap.String("minimum-cluster-version-supported", minV.String()),
+				)
+			} else {
+				plog.Warningf("the running cluster version(%v) is lower than the minimal cluster version(%v) supported", clusterv.String(), minV.String())
+			}
+			return false
+		}
+		if maxV.LessThan(*clusterv) {
+			if lg != nil {
+				lg.Warn(
+					"cluster version of remote member is not compatible; too high",
+					zap.String("remote-member-id", id),
+					zap.String("remote-member-cluster-version", clusterv.String()),
+					zap.String("minimum-cluster-version-supported", minV.String()),
+				)
+			} else {
+				plog.Warningf("the running cluster version(%v) is higher than the maximum cluster version(%v) supported", clusterv.String(), maxV.String())
+			}
+			return false
+		}
+		ok = true
+	}
+	return ok
+}
+
+// getVersion returns the Versions of the given member via its
+// peerURLs. Returns the last error if it fails to get the version.
+func getVersion(lg *zap.Logger, m *membership.Member, rt http.RoundTripper) (*version.Versions, error) {
+	cc := &http.Client{
+		Transport: rt,
+	}
+	var (
+		err  error
+		resp *http.Response
+	)
+
+	for _, u := range m.PeerURLs {
+		addr := u + "/version"
+		resp, err = cc.Get(addr)
+		if err != nil {
+			if lg != nil {
+				lg.Warn(
+					"failed to reach the peer URL",
+					zap.String("address", addr),
+					zap.String("remote-member-id", m.ID.String()),
+					zap.Error(err),
+				)
+			} else {
+				plog.Warningf("failed to reach the peerURL(%s) of member %s (%v)", u, m.ID, err)
+			}
+			continue
+		}
+		var b []byte
+		b, err = ioutil.ReadAll(resp.Body)
+		resp.Body.Close()
+		if err != nil {
+			if lg != nil {
+				lg.Warn(
+					"failed to read body of response",
+					zap.String("address", addr),
+					zap.String("remote-member-id", m.ID.String()),
+					zap.Error(err),
+				)
+			} else {
+				plog.Warningf("failed to read out the response body from the peerURL(%s) of member %s (%v)", u, m.ID, err)
+			}
+			continue
+		}
+		var vers version.Versions
+		if err = json.Unmarshal(b, &vers); err != nil {
+			if lg != nil {
+				lg.Warn(
+					"failed to unmarshal response",
+					zap.String("address", addr),
+					zap.String("remote-member-id", m.ID.String()),
+					zap.Error(err),
+				)
+			} else {
+				plog.Warningf("failed to unmarshal the response body got from the peerURL(%s) of member %s (%v)", u, m.ID, err)
+			}
+			continue
+		}
+		return &vers, nil
+	}
+	return nil, err
+}
+
+func promoteMemberHTTP(ctx context.Context, url string, id uint64, peerRt http.RoundTripper) ([]*membership.Member, error) {
+	cc := &http.Client{Transport: peerRt}
+	// TODO: refactor member http handler code
+	// cannot import etcdhttp, so manually construct url
+	requestUrl := url + "/members/promote/" + fmt.Sprintf("%d", id)
+	req, err := http.NewRequest("POST", requestUrl, nil)
+	if err != nil {
+		return nil, err
+	}
+	req = req.WithContext(ctx)
+	resp, err := cc.Do(req)
+	if err != nil {
+		return nil, err
+	}
+	defer resp.Body.Close()
+	b, err := ioutil.ReadAll(resp.Body)
+	if err != nil {
+		return nil, err
+	}
+
+	if resp.StatusCode == http.StatusRequestTimeout {
+		return nil, ErrTimeout
+	}
+	if resp.StatusCode == http.StatusPreconditionFailed {
+		// both ErrMemberNotLearner and ErrLearnerNotReady have same http status code
+		if strings.Contains(string(b), ErrLearnerNotReady.Error()) {
+			return nil, ErrLearnerNotReady
+		}
+		if strings.Contains(string(b), membership.ErrMemberNotLearner.Error()) {
+			return nil, membership.ErrMemberNotLearner
+		}
+		return nil, fmt.Errorf("member promote: unknown error(%s)", string(b))
+	}
+	if resp.StatusCode == http.StatusNotFound {
+		return nil, membership.ErrIDNotFound
+	}
+
+	if resp.StatusCode != http.StatusOK { // all other types of errors
+		return nil, fmt.Errorf("member promote: unknown error(%s)", string(b))
+	}
+
+	var membs []*membership.Member
+	if err := json.Unmarshal(b, &membs); err != nil {
+		return nil, err
+	}
+	return membs, nil
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/config.go b/vendor/go.etcd.io/etcd/etcdserver/config.go
new file mode 100644
index 0000000..88cd721
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/config.go
@@ -0,0 +1,307 @@
+// Copyright 2015 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import (
+	"context"
+	"fmt"
+	"path/filepath"
+	"sort"
+	"strings"
+	"time"
+
+	"go.etcd.io/etcd/pkg/netutil"
+	"go.etcd.io/etcd/pkg/transport"
+	"go.etcd.io/etcd/pkg/types"
+
+	bolt "go.etcd.io/bbolt"
+	"go.uber.org/zap"
+	"go.uber.org/zap/zapcore"
+)
+
+// ServerConfig holds the configuration of etcd as taken from the command line or discovery.
+type ServerConfig struct {
+	Name           string
+	DiscoveryURL   string
+	DiscoveryProxy string
+	ClientURLs     types.URLs
+	PeerURLs       types.URLs
+	DataDir        string
+	// DedicatedWALDir config will make the etcd to write the WAL to the WALDir
+	// rather than the dataDir/member/wal.
+	DedicatedWALDir string
+
+	SnapshotCount uint64
+
+	// SnapshotCatchUpEntries is the number of entries for a slow follower
+	// to catch-up after compacting the raft storage entries.
+	// We expect the follower has a millisecond level latency with the leader.
+	// The max throughput is around 10K. Keep a 5K entries is enough for helping
+	// follower to catch up.
+	// WARNING: only change this for tests. Always use "DefaultSnapshotCatchUpEntries"
+	SnapshotCatchUpEntries uint64
+
+	MaxSnapFiles uint
+	MaxWALFiles  uint
+
+	// BackendBatchInterval is the maximum time before commit the backend transaction.
+	BackendBatchInterval time.Duration
+	// BackendBatchLimit is the maximum operations before commit the backend transaction.
+	BackendBatchLimit int
+
+	// BackendFreelistType is the type of the backend boltdb freelist.
+	BackendFreelistType bolt.FreelistType
+
+	InitialPeerURLsMap  types.URLsMap
+	InitialClusterToken string
+	NewCluster          bool
+	PeerTLSInfo         transport.TLSInfo
+
+	CORS map[string]struct{}
+
+	// HostWhitelist lists acceptable hostnames from client requests.
+	// If server is insecure (no TLS), server only accepts requests
+	// whose Host header value exists in this white list.
+	HostWhitelist map[string]struct{}
+
+	TickMs        uint
+	ElectionTicks int
+
+	// InitialElectionTickAdvance is true, then local member fast-forwards
+	// election ticks to speed up "initial" leader election trigger. This
+	// benefits the case of larger election ticks. For instance, cross
+	// datacenter deployment may require longer election timeout of 10-second.
+	// If true, local node does not need wait up to 10-second. Instead,
+	// forwards its election ticks to 8-second, and have only 2-second left
+	// before leader election.
+	//
+	// Major assumptions are that:
+	//  - cluster has no active leader thus advancing ticks enables faster
+	//    leader election, or
+	//  - cluster already has an established leader, and rejoining follower
+	//    is likely to receive heartbeats from the leader after tick advance
+	//    and before election timeout.
+	//
+	// However, when network from leader to rejoining follower is congested,
+	// and the follower does not receive leader heartbeat within left election
+	// ticks, disruptive election has to happen thus affecting cluster
+	// availabilities.
+	//
+	// Disabling this would slow down initial bootstrap process for cross
+	// datacenter deployments. Make your own tradeoffs by configuring
+	// --initial-election-tick-advance at the cost of slow initial bootstrap.
+	//
+	// If single-node, it advances ticks regardless.
+	//
+	// See https://github.com/etcd-io/etcd/issues/9333 for more detail.
+	InitialElectionTickAdvance bool
+
+	BootstrapTimeout time.Duration
+
+	AutoCompactionRetention time.Duration
+	AutoCompactionMode      string
+	CompactionBatchLimit    int
+	QuotaBackendBytes       int64
+	MaxTxnOps               uint
+
+	// MaxRequestBytes is the maximum request size to send over raft.
+	MaxRequestBytes uint
+
+	StrictReconfigCheck bool
+
+	// ClientCertAuthEnabled is true when cert has been signed by the client CA.
+	ClientCertAuthEnabled bool
+
+	AuthToken  string
+	BcryptCost uint
+
+	// InitialCorruptCheck is true to check data corruption on boot
+	// before serving any peer/client traffic.
+	InitialCorruptCheck bool
+	CorruptCheckTime    time.Duration
+
+	// PreVote is true to enable Raft Pre-Vote.
+	PreVote bool
+
+	// Logger logs server-side operations.
+	// If not nil, it disables "capnslog" and uses the given logger.
+	Logger *zap.Logger
+
+	// LoggerConfig is server logger configuration for Raft logger.
+	// Must be either: "LoggerConfig != nil" or "LoggerCore != nil && LoggerWriteSyncer != nil".
+	LoggerConfig *zap.Config
+	// LoggerCore is "zapcore.Core" for raft logger.
+	// Must be either: "LoggerConfig != nil" or "LoggerCore != nil && LoggerWriteSyncer != nil".
+	LoggerCore        zapcore.Core
+	LoggerWriteSyncer zapcore.WriteSyncer
+
+	Debug bool
+
+	ForceNewCluster bool
+
+	// EnableLeaseCheckpoint enables primary lessor to persist lease remainingTTL to prevent indefinite auto-renewal of long lived leases.
+	EnableLeaseCheckpoint bool
+	// LeaseCheckpointInterval time.Duration is the wait duration between lease checkpoints.
+	LeaseCheckpointInterval time.Duration
+
+	EnableGRPCGateway bool
+}
+
+// VerifyBootstrap sanity-checks the initial config for bootstrap case
+// and returns an error for things that should never happen.
+func (c *ServerConfig) VerifyBootstrap() error {
+	if err := c.hasLocalMember(); err != nil {
+		return err
+	}
+	if err := c.advertiseMatchesCluster(); err != nil {
+		return err
+	}
+	if checkDuplicateURL(c.InitialPeerURLsMap) {
+		return fmt.Errorf("initial cluster %s has duplicate url", c.InitialPeerURLsMap)
+	}
+	if c.InitialPeerURLsMap.String() == "" && c.DiscoveryURL == "" {
+		return fmt.Errorf("initial cluster unset and no discovery URL found")
+	}
+	return nil
+}
+
+// VerifyJoinExisting sanity-checks the initial config for join existing cluster
+// case and returns an error for things that should never happen.
+func (c *ServerConfig) VerifyJoinExisting() error {
+	// The member has announced its peer urls to the cluster before starting; no need to
+	// set the configuration again.
+	if err := c.hasLocalMember(); err != nil {
+		return err
+	}
+	if checkDuplicateURL(c.InitialPeerURLsMap) {
+		return fmt.Errorf("initial cluster %s has duplicate url", c.InitialPeerURLsMap)
+	}
+	if c.DiscoveryURL != "" {
+		return fmt.Errorf("discovery URL should not be set when joining existing initial cluster")
+	}
+	return nil
+}
+
+// hasLocalMember checks that the cluster at least contains the local server.
+func (c *ServerConfig) hasLocalMember() error {
+	if urls := c.InitialPeerURLsMap[c.Name]; urls == nil {
+		return fmt.Errorf("couldn't find local name %q in the initial cluster configuration", c.Name)
+	}
+	return nil
+}
+
+// advertiseMatchesCluster confirms peer URLs match those in the cluster peer list.
+func (c *ServerConfig) advertiseMatchesCluster() error {
+	urls, apurls := c.InitialPeerURLsMap[c.Name], c.PeerURLs.StringSlice()
+	urls.Sort()
+	sort.Strings(apurls)
+	ctx, cancel := context.WithTimeout(context.TODO(), 30*time.Second)
+	defer cancel()
+	ok, err := netutil.URLStringsEqual(ctx, c.Logger, apurls, urls.StringSlice())
+	if ok {
+		return nil
+	}
+
+	initMap, apMap := make(map[string]struct{}), make(map[string]struct{})
+	for _, url := range c.PeerURLs {
+		apMap[url.String()] = struct{}{}
+	}
+	for _, url := range c.InitialPeerURLsMap[c.Name] {
+		initMap[url.String()] = struct{}{}
+	}
+
+	missing := []string{}
+	for url := range initMap {
+		if _, ok := apMap[url]; !ok {
+			missing = append(missing, url)
+		}
+	}
+	if len(missing) > 0 {
+		for i := range missing {
+			missing[i] = c.Name + "=" + missing[i]
+		}
+		mstr := strings.Join(missing, ",")
+		apStr := strings.Join(apurls, ",")
+		return fmt.Errorf("--initial-cluster has %s but missing from --initial-advertise-peer-urls=%s (%v)", mstr, apStr, err)
+	}
+
+	for url := range apMap {
+		if _, ok := initMap[url]; !ok {
+			missing = append(missing, url)
+		}
+	}
+	if len(missing) > 0 {
+		mstr := strings.Join(missing, ",")
+		umap := types.URLsMap(map[string]types.URLs{c.Name: c.PeerURLs})
+		return fmt.Errorf("--initial-advertise-peer-urls has %s but missing from --initial-cluster=%s", mstr, umap.String())
+	}
+
+	// resolved URLs from "--initial-advertise-peer-urls" and "--initial-cluster" did not match or failed
+	apStr := strings.Join(apurls, ",")
+	umap := types.URLsMap(map[string]types.URLs{c.Name: c.PeerURLs})
+	return fmt.Errorf("failed to resolve %s to match --initial-cluster=%s (%v)", apStr, umap.String(), err)
+}
+
+func (c *ServerConfig) MemberDir() string { return filepath.Join(c.DataDir, "member") }
+
+func (c *ServerConfig) WALDir() string {
+	if c.DedicatedWALDir != "" {
+		return c.DedicatedWALDir
+	}
+	return filepath.Join(c.MemberDir(), "wal")
+}
+
+func (c *ServerConfig) SnapDir() string { return filepath.Join(c.MemberDir(), "snap") }
+
+func (c *ServerConfig) ShouldDiscover() bool { return c.DiscoveryURL != "" }
+
+// ReqTimeout returns timeout for request to finish.
+func (c *ServerConfig) ReqTimeout() time.Duration {
+	// 5s for queue waiting, computation and disk IO delay
+	// + 2 * election timeout for possible leader election
+	return 5*time.Second + 2*time.Duration(c.ElectionTicks*int(c.TickMs))*time.Millisecond
+}
+
+func (c *ServerConfig) electionTimeout() time.Duration {
+	return time.Duration(c.ElectionTicks*int(c.TickMs)) * time.Millisecond
+}
+
+func (c *ServerConfig) peerDialTimeout() time.Duration {
+	// 1s for queue wait and election timeout
+	return time.Second + time.Duration(c.ElectionTicks*int(c.TickMs))*time.Millisecond
+}
+
+func checkDuplicateURL(urlsmap types.URLsMap) bool {
+	um := make(map[string]bool)
+	for _, urls := range urlsmap {
+		for _, url := range urls {
+			u := url.String()
+			if um[u] {
+				return true
+			}
+			um[u] = true
+		}
+	}
+	return false
+}
+
+func (c *ServerConfig) bootstrapTimeout() time.Duration {
+	if c.BootstrapTimeout != 0 {
+		return c.BootstrapTimeout
+	}
+	return time.Second
+}
+
+func (c *ServerConfig) backendPath() string { return filepath.Join(c.SnapDir(), "db") }
diff --git a/vendor/go.etcd.io/etcd/etcdserver/consistent_index.go b/vendor/go.etcd.io/etcd/etcdserver/consistent_index.go
new file mode 100644
index 0000000..d513f67
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/consistent_index.go
@@ -0,0 +1,33 @@
+// Copyright 2015 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import (
+	"sync/atomic"
+)
+
+// consistentIndex represents the offset of an entry in a consistent replica log.
+// It implements the mvcc.ConsistentIndexGetter interface.
+// It is always set to the offset of current entry before executing the entry,
+// so ConsistentWatchableKV could get the consistent index from it.
+type consistentIndex uint64
+
+func (i *consistentIndex) setConsistentIndex(v uint64) {
+	atomic.StoreUint64((*uint64)(i), v)
+}
+
+func (i *consistentIndex) ConsistentIndex() uint64 {
+	return atomic.LoadUint64((*uint64)(i))
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/corrupt.go b/vendor/go.etcd.io/etcd/etcdserver/corrupt.go
new file mode 100644
index 0000000..32678a7
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/corrupt.go
@@ -0,0 +1,411 @@
+// Copyright 2017 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import (
+	"context"
+	"fmt"
+	"time"
+
+	"go.etcd.io/etcd/clientv3"
+	"go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/mvcc"
+	"go.etcd.io/etcd/pkg/types"
+
+	"go.uber.org/zap"
+)
+
+// CheckInitialHashKV compares initial hash values with its peers
+// before serving any peer/client traffic. Only mismatch when hashes
+// are different at requested revision, with same compact revision.
+func (s *EtcdServer) CheckInitialHashKV() error {
+	if !s.Cfg.InitialCorruptCheck {
+		return nil
+	}
+
+	lg := s.getLogger()
+
+	if lg != nil {
+		lg.Info(
+			"starting initial corruption check",
+			zap.String("local-member-id", s.ID().String()),
+			zap.Duration("timeout", s.Cfg.ReqTimeout()),
+		)
+	} else {
+		plog.Infof("%s starting initial corruption check with timeout %v...", s.ID(), s.Cfg.ReqTimeout())
+	}
+
+	h, rev, crev, err := s.kv.HashByRev(0)
+	if err != nil {
+		return fmt.Errorf("%s failed to fetch hash (%v)", s.ID(), err)
+	}
+	peers := s.getPeerHashKVs(rev)
+	mismatch := 0
+	for _, p := range peers {
+		if p.resp != nil {
+			peerID := types.ID(p.resp.Header.MemberId)
+			fields := []zap.Field{
+				zap.String("local-member-id", s.ID().String()),
+				zap.Int64("local-member-revision", rev),
+				zap.Int64("local-member-compact-revision", crev),
+				zap.Uint32("local-member-hash", h),
+				zap.String("remote-peer-id", peerID.String()),
+				zap.Strings("remote-peer-endpoints", p.eps),
+				zap.Int64("remote-peer-revision", p.resp.Header.Revision),
+				zap.Int64("remote-peer-compact-revision", p.resp.CompactRevision),
+				zap.Uint32("remote-peer-hash", p.resp.Hash),
+			}
+
+			if h != p.resp.Hash {
+				if crev == p.resp.CompactRevision {
+					if lg != nil {
+						lg.Warn("found different hash values from remote peer", fields...)
+					} else {
+						plog.Errorf("%s's hash %d != %s's hash %d (revision %d, peer revision %d, compact revision %d)", s.ID(), h, peerID, p.resp.Hash, rev, p.resp.Header.Revision, crev)
+					}
+					mismatch++
+				} else {
+					if lg != nil {
+						lg.Warn("found different compact revision values from remote peer", fields...)
+					} else {
+						plog.Warningf("%s cannot check hash of peer(%s): peer has a different compact revision %d (revision:%d)", s.ID(), peerID, p.resp.CompactRevision, rev)
+					}
+				}
+			}
+
+			continue
+		}
+
+		if p.err != nil {
+			switch p.err {
+			case rpctypes.ErrFutureRev:
+				if lg != nil {
+					lg.Warn(
+						"cannot fetch hash from slow remote peer",
+						zap.String("local-member-id", s.ID().String()),
+						zap.Int64("local-member-revision", rev),
+						zap.Int64("local-member-compact-revision", crev),
+						zap.Uint32("local-member-hash", h),
+						zap.String("remote-peer-id", p.id.String()),
+						zap.Strings("remote-peer-endpoints", p.eps),
+						zap.Error(err),
+					)
+				} else {
+					plog.Warningf("%s cannot check the hash of peer(%q) at revision %d: peer is lagging behind(%q)", s.ID(), p.eps, rev, p.err.Error())
+				}
+			case rpctypes.ErrCompacted:
+				if lg != nil {
+					lg.Warn(
+						"cannot fetch hash from remote peer; local member is behind",
+						zap.String("local-member-id", s.ID().String()),
+						zap.Int64("local-member-revision", rev),
+						zap.Int64("local-member-compact-revision", crev),
+						zap.Uint32("local-member-hash", h),
+						zap.String("remote-peer-id", p.id.String()),
+						zap.Strings("remote-peer-endpoints", p.eps),
+						zap.Error(err),
+					)
+				} else {
+					plog.Warningf("%s cannot check the hash of peer(%q) at revision %d: local node is lagging behind(%q)", s.ID(), p.eps, rev, p.err.Error())
+				}
+			}
+		}
+	}
+	if mismatch > 0 {
+		return fmt.Errorf("%s found data inconsistency with peers", s.ID())
+	}
+
+	if lg != nil {
+		lg.Info(
+			"initial corruption checking passed; no corruption",
+			zap.String("local-member-id", s.ID().String()),
+		)
+	} else {
+		plog.Infof("%s succeeded on initial corruption checking: no corruption", s.ID())
+	}
+	return nil
+}
+
+func (s *EtcdServer) monitorKVHash() {
+	t := s.Cfg.CorruptCheckTime
+	if t == 0 {
+		return
+	}
+
+	lg := s.getLogger()
+	if lg != nil {
+		lg.Info(
+			"enabled corruption checking",
+			zap.String("local-member-id", s.ID().String()),
+			zap.Duration("interval", t),
+		)
+	} else {
+		plog.Infof("enabled corruption checking with %s interval", t)
+	}
+
+	for {
+		select {
+		case <-s.stopping:
+			return
+		case <-time.After(t):
+		}
+		if !s.isLeader() {
+			continue
+		}
+		if err := s.checkHashKV(); err != nil {
+			if lg != nil {
+				lg.Warn("failed to check hash KV", zap.Error(err))
+			} else {
+				plog.Debugf("check hash kv failed %v", err)
+			}
+		}
+	}
+}
+
+func (s *EtcdServer) checkHashKV() error {
+	lg := s.getLogger()
+
+	h, rev, crev, err := s.kv.HashByRev(0)
+	if err != nil {
+		return err
+	}
+	peers := s.getPeerHashKVs(rev)
+
+	ctx, cancel := context.WithTimeout(context.Background(), s.Cfg.ReqTimeout())
+	err = s.linearizableReadNotify(ctx)
+	cancel()
+	if err != nil {
+		return err
+	}
+
+	h2, rev2, crev2, err := s.kv.HashByRev(0)
+	if err != nil {
+		return err
+	}
+
+	alarmed := false
+	mismatch := func(id uint64) {
+		if alarmed {
+			return
+		}
+		alarmed = true
+		a := &pb.AlarmRequest{
+			MemberID: id,
+			Action:   pb.AlarmRequest_ACTIVATE,
+			Alarm:    pb.AlarmType_CORRUPT,
+		}
+		s.goAttach(func() {
+			s.raftRequest(s.ctx, pb.InternalRaftRequest{Alarm: a})
+		})
+	}
+
+	if h2 != h && rev2 == rev && crev == crev2 {
+		if lg != nil {
+			lg.Warn(
+				"found hash mismatch",
+				zap.Int64("revision-1", rev),
+				zap.Int64("compact-revision-1", crev),
+				zap.Uint32("hash-1", h),
+				zap.Int64("revision-2", rev2),
+				zap.Int64("compact-revision-2", crev2),
+				zap.Uint32("hash-2", h2),
+			)
+		} else {
+			plog.Warningf("mismatched hashes %d and %d for revision %d", h, h2, rev)
+		}
+		mismatch(uint64(s.ID()))
+	}
+
+	for _, p := range peers {
+		if p.resp == nil {
+			continue
+		}
+		id := p.resp.Header.MemberId
+
+		// leader expects follower's latest revision less than or equal to leader's
+		if p.resp.Header.Revision > rev2 {
+			if lg != nil {
+				lg.Warn(
+					"revision from follower must be less than or equal to leader's",
+					zap.Int64("leader-revision", rev2),
+					zap.Int64("follower-revision", p.resp.Header.Revision),
+					zap.String("follower-peer-id", types.ID(id).String()),
+				)
+			} else {
+				plog.Warningf(
+					"revision %d from member %v, expected at most %d",
+					p.resp.Header.Revision,
+					types.ID(id),
+					rev2)
+			}
+			mismatch(id)
+		}
+
+		// leader expects follower's latest compact revision less than or equal to leader's
+		if p.resp.CompactRevision > crev2 {
+			if lg != nil {
+				lg.Warn(
+					"compact revision from follower must be less than or equal to leader's",
+					zap.Int64("leader-compact-revision", crev2),
+					zap.Int64("follower-compact-revision", p.resp.CompactRevision),
+					zap.String("follower-peer-id", types.ID(id).String()),
+				)
+			} else {
+				plog.Warningf(
+					"compact revision %d from member %v, expected at most %d",
+					p.resp.CompactRevision,
+					types.ID(id),
+					crev2,
+				)
+			}
+			mismatch(id)
+		}
+
+		// follower's compact revision is leader's old one, then hashes must match
+		if p.resp.CompactRevision == crev && p.resp.Hash != h {
+			if lg != nil {
+				lg.Warn(
+					"same compact revision then hashes must match",
+					zap.Int64("leader-compact-revision", crev2),
+					zap.Uint32("leader-hash", h),
+					zap.Int64("follower-compact-revision", p.resp.CompactRevision),
+					zap.Uint32("follower-hash", p.resp.Hash),
+					zap.String("follower-peer-id", types.ID(id).String()),
+				)
+			} else {
+				plog.Warningf(
+					"hash %d at revision %d from member %v, expected hash %d",
+					p.resp.Hash,
+					rev,
+					types.ID(id),
+					h,
+				)
+			}
+			mismatch(id)
+		}
+	}
+	return nil
+}
+
+type peerHashKVResp struct {
+	id  types.ID
+	eps []string
+
+	resp *clientv3.HashKVResponse
+	err  error
+}
+
+func (s *EtcdServer) getPeerHashKVs(rev int64) (resps []*peerHashKVResp) {
+	// TODO: handle the case when "s.cluster.Members" have not
+	// been populated (e.g. no snapshot to load from disk)
+	mbs := s.cluster.Members()
+	pss := make([]peerHashKVResp, len(mbs))
+	for _, m := range mbs {
+		if m.ID == s.ID() {
+			continue
+		}
+		pss = append(pss, peerHashKVResp{id: m.ID, eps: m.PeerURLs})
+	}
+
+	lg := s.getLogger()
+
+	for _, p := range pss {
+		if len(p.eps) == 0 {
+			continue
+		}
+		cli, cerr := clientv3.New(clientv3.Config{
+			DialTimeout: s.Cfg.ReqTimeout(),
+			Endpoints:   p.eps,
+		})
+		if cerr != nil {
+			if lg != nil {
+				lg.Warn(
+					"failed to create client to peer URL",
+					zap.String("local-member-id", s.ID().String()),
+					zap.String("remote-peer-id", p.id.String()),
+					zap.Strings("remote-peer-endpoints", p.eps),
+					zap.Error(cerr),
+				)
+			} else {
+				plog.Warningf("%s failed to create client to peer %q for hash checking (%q)", s.ID(), p.eps, cerr.Error())
+			}
+			continue
+		}
+
+		respsLen := len(resps)
+		for _, c := range cli.Endpoints() {
+			ctx, cancel := context.WithTimeout(context.Background(), s.Cfg.ReqTimeout())
+			var resp *clientv3.HashKVResponse
+			resp, cerr = cli.HashKV(ctx, c, rev)
+			cancel()
+			if cerr == nil {
+				resps = append(resps, &peerHashKVResp{id: p.id, eps: p.eps, resp: resp, err: nil})
+				break
+			}
+			if lg != nil {
+				lg.Warn(
+					"failed hash kv request",
+					zap.String("local-member-id", s.ID().String()),
+					zap.Int64("requested-revision", rev),
+					zap.String("remote-peer-endpoint", c),
+					zap.Error(cerr),
+				)
+			} else {
+				plog.Warningf("%s hash-kv error %q on peer %q with revision %d", s.ID(), cerr.Error(), c, rev)
+			}
+		}
+		cli.Close()
+
+		if respsLen == len(resps) {
+			resps = append(resps, &peerHashKVResp{id: p.id, eps: p.eps, resp: nil, err: cerr})
+		}
+	}
+	return resps
+}
+
+type applierV3Corrupt struct {
+	applierV3
+}
+
+func newApplierV3Corrupt(a applierV3) *applierV3Corrupt { return &applierV3Corrupt{a} }
+
+func (a *applierV3Corrupt) Put(txn mvcc.TxnWrite, p *pb.PutRequest) (*pb.PutResponse, error) {
+	return nil, ErrCorrupt
+}
+
+func (a *applierV3Corrupt) Range(txn mvcc.TxnRead, p *pb.RangeRequest) (*pb.RangeResponse, error) {
+	return nil, ErrCorrupt
+}
+
+func (a *applierV3Corrupt) DeleteRange(txn mvcc.TxnWrite, p *pb.DeleteRangeRequest) (*pb.DeleteRangeResponse, error) {
+	return nil, ErrCorrupt
+}
+
+func (a *applierV3Corrupt) Txn(rt *pb.TxnRequest) (*pb.TxnResponse, error) {
+	return nil, ErrCorrupt
+}
+
+func (a *applierV3Corrupt) Compaction(compaction *pb.CompactionRequest) (*pb.CompactionResponse, <-chan struct{}, error) {
+	return nil, nil, ErrCorrupt
+}
+
+func (a *applierV3Corrupt) LeaseGrant(lc *pb.LeaseGrantRequest) (*pb.LeaseGrantResponse, error) {
+	return nil, ErrCorrupt
+}
+
+func (a *applierV3Corrupt) LeaseRevoke(lc *pb.LeaseRevokeRequest) (*pb.LeaseRevokeResponse, error) {
+	return nil, ErrCorrupt
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/doc.go b/vendor/go.etcd.io/etcd/etcdserver/doc.go
new file mode 100644
index 0000000..b195d2d
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/doc.go
@@ -0,0 +1,16 @@
+// Copyright 2015 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// Package etcdserver defines how etcd servers interact and store their states.
+package etcdserver
diff --git a/vendor/go.etcd.io/etcd/etcdserver/errors.go b/vendor/go.etcd.io/etcd/etcdserver/errors.go
new file mode 100644
index 0000000..d0fe289
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/errors.go
@@ -0,0 +1,51 @@
+// Copyright 2015 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import (
+	"errors"
+	"fmt"
+)
+
+var (
+	ErrUnknownMethod              = errors.New("etcdserver: unknown method")
+	ErrStopped                    = errors.New("etcdserver: server stopped")
+	ErrCanceled                   = errors.New("etcdserver: request cancelled")
+	ErrTimeout                    = errors.New("etcdserver: request timed out")
+	ErrTimeoutDueToLeaderFail     = errors.New("etcdserver: request timed out, possibly due to previous leader failure")
+	ErrTimeoutDueToConnectionLost = errors.New("etcdserver: request timed out, possibly due to connection lost")
+	ErrTimeoutLeaderTransfer      = errors.New("etcdserver: request timed out, leader transfer took too long")
+	ErrLeaderChanged              = errors.New("etcdserver: leader changed")
+	ErrNotEnoughStartedMembers    = errors.New("etcdserver: re-configuration failed due to not enough started members")
+	ErrLearnerNotReady            = errors.New("etcdserver: can only promote a learner member which is in sync with leader")
+	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")
+	ErrUnhealthy                  = errors.New("etcdserver: unhealthy cluster")
+	ErrKeyNotFound                = errors.New("etcdserver: key not found")
+	ErrCorrupt                    = errors.New("etcdserver: corrupt cluster")
+	ErrBadLeaderTransferee        = errors.New("etcdserver: bad leader transferee")
+)
+
+type DiscoveryError struct {
+	Op  string
+	Err error
+}
+
+func (e DiscoveryError) Error() string {
+	return fmt.Sprintf("failed to %s discovery cluster (%v)", e.Op, e.Err)
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/gw/rpc.pb.gw.go b/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/gw/rpc.pb.gw.go
new file mode 100644
index 0000000..904c321
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/etcdserverpb/gw/rpc.pb.gw.go
@@ -0,0 +1,2318 @@
+// Code generated by protoc-gen-grpc-gateway. DO NOT EDIT.
+// source: etcdserver/etcdserverpb/rpc.proto
+
+/*
+Package etcdserverpb is a reverse proxy.
+
+It translates gRPC into RESTful JSON APIs.
+*/
+package gw
+
+import (
+	"go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"io"
+	"net/http"
+
+	"github.com/golang/protobuf/proto"
+	"github.com/grpc-ecosystem/grpc-gateway/runtime"
+	"github.com/grpc-ecosystem/grpc-gateway/utilities"
+	"golang.org/x/net/context"
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/grpclog"
+	"google.golang.org/grpc/status"
+)
+
+var _ codes.Code
+var _ io.Reader
+var _ status.Status
+var _ = runtime.String
+var _ = utilities.NewDoubleArray
+
+func request_KV_Range_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.KVClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.RangeRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.Range(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_KV_Put_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.KVClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.PutRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.Put(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_KV_DeleteRange_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.KVClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.DeleteRangeRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.DeleteRange(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_KV_Txn_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.KVClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.TxnRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.Txn(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_KV_Compact_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.KVClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.CompactionRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.Compact(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Watch_Watch_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.WatchClient, req *http.Request, pathParams map[string]string) (etcdserverpb.Watch_WatchClient, runtime.ServerMetadata, error) {
+	var metadata runtime.ServerMetadata
+	stream, err := client.Watch(ctx)
+	if err != nil {
+		grpclog.Printf("Failed to start streaming: %v", err)
+		return nil, metadata, err
+	}
+	dec := marshaler.NewDecoder(req.Body)
+	handleSend := func() error {
+		var protoReq etcdserverpb.WatchRequest
+		err := dec.Decode(&protoReq)
+		if err == io.EOF {
+			return err
+		}
+		if err != nil {
+			grpclog.Printf("Failed to decode request: %v", err)
+			return err
+		}
+		if err := stream.Send(&protoReq); err != nil {
+			grpclog.Printf("Failed to send request: %v", err)
+			return err
+		}
+		return nil
+	}
+	if err := handleSend(); err != nil {
+		if cerr := stream.CloseSend(); cerr != nil {
+			grpclog.Printf("Failed to terminate client stream: %v", cerr)
+		}
+		if err == io.EOF {
+			return stream, metadata, nil
+		}
+		return nil, metadata, err
+	}
+	go func() {
+		for {
+			if err := handleSend(); err != nil {
+				break
+			}
+		}
+		if err := stream.CloseSend(); err != nil {
+			grpclog.Printf("Failed to terminate client stream: %v", err)
+		}
+	}()
+	header, err := stream.Header()
+	if err != nil {
+		grpclog.Printf("Failed to get header from client: %v", err)
+		return nil, metadata, err
+	}
+	metadata.HeaderMD = header
+	return stream, metadata, nil
+}
+
+func request_Lease_LeaseGrant_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.LeaseClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.LeaseGrantRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.LeaseGrant(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Lease_LeaseRevoke_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.LeaseClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.LeaseRevokeRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.LeaseRevoke(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Lease_LeaseRevoke_1(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.LeaseClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.LeaseRevokeRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.LeaseRevoke(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Lease_LeaseKeepAlive_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.LeaseClient, req *http.Request, pathParams map[string]string) (etcdserverpb.Lease_LeaseKeepAliveClient, runtime.ServerMetadata, error) {
+	var metadata runtime.ServerMetadata
+	stream, err := client.LeaseKeepAlive(ctx)
+	if err != nil {
+		grpclog.Printf("Failed to start streaming: %v", err)
+		return nil, metadata, err
+	}
+	dec := marshaler.NewDecoder(req.Body)
+	handleSend := func() error {
+		var protoReq etcdserverpb.LeaseKeepAliveRequest
+		err := dec.Decode(&protoReq)
+		if err == io.EOF {
+			return err
+		}
+		if err != nil {
+			grpclog.Printf("Failed to decode request: %v", err)
+			return err
+		}
+		if err := stream.Send(&protoReq); err != nil {
+			grpclog.Printf("Failed to send request: %v", err)
+			return err
+		}
+		return nil
+	}
+	if err := handleSend(); err != nil {
+		if cerr := stream.CloseSend(); cerr != nil {
+			grpclog.Printf("Failed to terminate client stream: %v", cerr)
+		}
+		if err == io.EOF {
+			return stream, metadata, nil
+		}
+		return nil, metadata, err
+	}
+	go func() {
+		for {
+			if err := handleSend(); err != nil {
+				break
+			}
+		}
+		if err := stream.CloseSend(); err != nil {
+			grpclog.Printf("Failed to terminate client stream: %v", err)
+		}
+	}()
+	header, err := stream.Header()
+	if err != nil {
+		grpclog.Printf("Failed to get header from client: %v", err)
+		return nil, metadata, err
+	}
+	metadata.HeaderMD = header
+	return stream, metadata, nil
+}
+
+func request_Lease_LeaseTimeToLive_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.LeaseClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.LeaseTimeToLiveRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.LeaseTimeToLive(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Lease_LeaseTimeToLive_1(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.LeaseClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.LeaseTimeToLiveRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.LeaseTimeToLive(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Lease_LeaseLeases_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.LeaseClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.LeaseLeasesRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.LeaseLeases(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Lease_LeaseLeases_1(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.LeaseClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.LeaseLeasesRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.LeaseLeases(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Cluster_MemberAdd_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.ClusterClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.MemberAddRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.MemberAdd(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Cluster_MemberRemove_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.ClusterClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.MemberRemoveRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.MemberRemove(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Cluster_MemberUpdate_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.ClusterClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.MemberUpdateRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.MemberUpdate(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Cluster_MemberList_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.ClusterClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.MemberListRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.MemberList(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Cluster_MemberPromote_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.ClusterClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.MemberPromoteRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.MemberPromote(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Maintenance_Alarm_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.AlarmRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.Alarm(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Maintenance_Status_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.StatusRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.Status(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Maintenance_Defragment_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.DefragmentRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.Defragment(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Maintenance_Hash_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.HashRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.Hash(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Maintenance_HashKV_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.HashKVRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.HashKV(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Maintenance_Snapshot_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.MaintenanceClient, req *http.Request, pathParams map[string]string) (etcdserverpb.Maintenance_SnapshotClient, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.SnapshotRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	stream, err := client.Snapshot(ctx, &protoReq)
+	if err != nil {
+		return nil, metadata, err
+	}
+	header, err := stream.Header()
+	if err != nil {
+		return nil, metadata, err
+	}
+	metadata.HeaderMD = header
+	return stream, metadata, nil
+
+}
+
+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 && err != io.EOF {
+		return nil, metadata, status.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
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.AuthEnable(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Auth_AuthDisable_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.AuthDisableRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.AuthDisable(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Auth_Authenticate_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.AuthenticateRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.Authenticate(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Auth_UserAdd_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.AuthUserAddRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.UserAdd(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Auth_UserGet_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.AuthUserGetRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.UserGet(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Auth_UserList_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.AuthUserListRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.UserList(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Auth_UserDelete_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.AuthUserDeleteRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.UserDelete(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Auth_UserChangePassword_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.AuthUserChangePasswordRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.UserChangePassword(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Auth_UserGrantRole_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.AuthUserGrantRoleRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.UserGrantRole(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Auth_UserRevokeRole_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.AuthUserRevokeRoleRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.UserRevokeRole(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Auth_RoleAdd_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.AuthRoleAddRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.RoleAdd(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Auth_RoleGet_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.AuthRoleGetRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.RoleGet(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Auth_RoleList_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.AuthRoleListRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.RoleList(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Auth_RoleDelete_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.AuthRoleDeleteRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.RoleDelete(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Auth_RoleGrantPermission_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.AuthRoleGrantPermissionRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.RoleGrantPermission(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+func request_Auth_RoleRevokePermission_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.AuthRoleRevokePermissionRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil && err != io.EOF {
+		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.RoleRevokePermission(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
+// RegisterKVHandlerFromEndpoint is same as RegisterKVHandler but
+// automatically dials to "endpoint" and closes the connection when "ctx" gets done.
+func RegisterKVHandlerFromEndpoint(ctx context.Context, mux *runtime.ServeMux, endpoint string, opts []grpc.DialOption) (err error) {
+	conn, err := grpc.Dial(endpoint, opts...)
+	if err != nil {
+		return err
+	}
+	defer func() {
+		if err != nil {
+			if cerr := conn.Close(); cerr != nil {
+				grpclog.Printf("Failed to close conn to %s: %v", endpoint, cerr)
+			}
+			return
+		}
+		go func() {
+			<-ctx.Done()
+			if cerr := conn.Close(); cerr != nil {
+				grpclog.Printf("Failed to close conn to %s: %v", endpoint, cerr)
+			}
+		}()
+	}()
+
+	return RegisterKVHandler(ctx, mux, conn)
+}
+
+// RegisterKVHandler registers the http handlers for service KV to "mux".
+// The handlers forward requests to the grpc endpoint over "conn".
+func RegisterKVHandler(ctx context.Context, mux *runtime.ServeMux, conn *grpc.ClientConn) error {
+	return RegisterKVHandlerClient(ctx, mux, etcdserverpb.NewKVClient(conn))
+}
+
+// RegisterKVHandler registers the http handlers for service KV to "mux".
+// The handlers forward requests to the grpc endpoint over the given implementation of "KVClient".
+// Note: the gRPC framework executes interceptors within the gRPC handler. If the passed in "KVClient"
+// doesn't go through the normal gRPC flow (creating a gRPC client etc.) then it will be up to the passed in
+// "KVClient" to call the correct interceptors.
+func RegisterKVHandlerClient(ctx context.Context, mux *runtime.ServeMux, client etcdserverpb.KVClient) error {
+
+	mux.Handle("POST", pattern_KV_Range_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_KV_Range_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_KV_Range_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_KV_Put_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_KV_Put_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_KV_Put_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_KV_DeleteRange_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_KV_DeleteRange_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_KV_DeleteRange_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_KV_Txn_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_KV_Txn_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_KV_Txn_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_KV_Compact_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_KV_Compact_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_KV_Compact_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	return nil
+}
+
+var (
+	pattern_KV_Range_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "kv", "range"}, ""))
+
+	pattern_KV_Put_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "kv", "put"}, ""))
+
+	pattern_KV_DeleteRange_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "kv", "deleterange"}, ""))
+
+	pattern_KV_Txn_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "kv", "txn"}, ""))
+
+	pattern_KV_Compact_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "kv", "compaction"}, ""))
+)
+
+var (
+	forward_KV_Range_0 = runtime.ForwardResponseMessage
+
+	forward_KV_Put_0 = runtime.ForwardResponseMessage
+
+	forward_KV_DeleteRange_0 = runtime.ForwardResponseMessage
+
+	forward_KV_Txn_0 = runtime.ForwardResponseMessage
+
+	forward_KV_Compact_0 = runtime.ForwardResponseMessage
+)
+
+// RegisterWatchHandlerFromEndpoint is same as RegisterWatchHandler but
+// automatically dials to "endpoint" and closes the connection when "ctx" gets done.
+func RegisterWatchHandlerFromEndpoint(ctx context.Context, mux *runtime.ServeMux, endpoint string, opts []grpc.DialOption) (err error) {
+	conn, err := grpc.Dial(endpoint, opts...)
+	if err != nil {
+		return err
+	}
+	defer func() {
+		if err != nil {
+			if cerr := conn.Close(); cerr != nil {
+				grpclog.Printf("Failed to close conn to %s: %v", endpoint, cerr)
+			}
+			return
+		}
+		go func() {
+			<-ctx.Done()
+			if cerr := conn.Close(); cerr != nil {
+				grpclog.Printf("Failed to close conn to %s: %v", endpoint, cerr)
+			}
+		}()
+	}()
+
+	return RegisterWatchHandler(ctx, mux, conn)
+}
+
+// RegisterWatchHandler registers the http handlers for service Watch to "mux".
+// The handlers forward requests to the grpc endpoint over "conn".
+func RegisterWatchHandler(ctx context.Context, mux *runtime.ServeMux, conn *grpc.ClientConn) error {
+	return RegisterWatchHandlerClient(ctx, mux, etcdserverpb.NewWatchClient(conn))
+}
+
+// RegisterWatchHandler registers the http handlers for service Watch to "mux".
+// The handlers forward requests to the grpc endpoint over the given implementation of "WatchClient".
+// Note: the gRPC framework executes interceptors within the gRPC handler. If the passed in "WatchClient"
+// doesn't go through the normal gRPC flow (creating a gRPC client etc.) then it will be up to the passed in
+// "WatchClient" to call the correct interceptors.
+func RegisterWatchHandlerClient(ctx context.Context, mux *runtime.ServeMux, client etcdserverpb.WatchClient) error {
+
+	mux.Handle("POST", pattern_Watch_Watch_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Watch_Watch_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Watch_Watch_0(ctx, mux, outboundMarshaler, w, req, func() (proto.Message, error) { return resp.Recv() }, mux.GetForwardResponseOptions()...)
+
+	})
+
+	return nil
+}
+
+var (
+	pattern_Watch_Watch_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1}, []string{"v3", "watch"}, ""))
+)
+
+var (
+	forward_Watch_Watch_0 = runtime.ForwardResponseStream
+)
+
+// RegisterLeaseHandlerFromEndpoint is same as RegisterLeaseHandler but
+// automatically dials to "endpoint" and closes the connection when "ctx" gets done.
+func RegisterLeaseHandlerFromEndpoint(ctx context.Context, mux *runtime.ServeMux, endpoint string, opts []grpc.DialOption) (err error) {
+	conn, err := grpc.Dial(endpoint, opts...)
+	if err != nil {
+		return err
+	}
+	defer func() {
+		if err != nil {
+			if cerr := conn.Close(); cerr != nil {
+				grpclog.Printf("Failed to close conn to %s: %v", endpoint, cerr)
+			}
+			return
+		}
+		go func() {
+			<-ctx.Done()
+			if cerr := conn.Close(); cerr != nil {
+				grpclog.Printf("Failed to close conn to %s: %v", endpoint, cerr)
+			}
+		}()
+	}()
+
+	return RegisterLeaseHandler(ctx, mux, conn)
+}
+
+// RegisterLeaseHandler registers the http handlers for service Lease to "mux".
+// The handlers forward requests to the grpc endpoint over "conn".
+func RegisterLeaseHandler(ctx context.Context, mux *runtime.ServeMux, conn *grpc.ClientConn) error {
+	return RegisterLeaseHandlerClient(ctx, mux, etcdserverpb.NewLeaseClient(conn))
+}
+
+// RegisterLeaseHandler registers the http handlers for service Lease to "mux".
+// The handlers forward requests to the grpc endpoint over the given implementation of "LeaseClient".
+// Note: the gRPC framework executes interceptors within the gRPC handler. If the passed in "LeaseClient"
+// doesn't go through the normal gRPC flow (creating a gRPC client etc.) then it will be up to the passed in
+// "LeaseClient" to call the correct interceptors.
+func RegisterLeaseHandlerClient(ctx context.Context, mux *runtime.ServeMux, client etcdserverpb.LeaseClient) error {
+
+	mux.Handle("POST", pattern_Lease_LeaseGrant_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Lease_LeaseGrant_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Lease_LeaseGrant_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Lease_LeaseRevoke_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Lease_LeaseRevoke_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Lease_LeaseRevoke_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Lease_LeaseRevoke_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Lease_LeaseRevoke_1(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Lease_LeaseRevoke_1(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Lease_LeaseKeepAlive_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Lease_LeaseKeepAlive_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Lease_LeaseKeepAlive_0(ctx, mux, outboundMarshaler, w, req, func() (proto.Message, error) { return resp.Recv() }, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Lease_LeaseTimeToLive_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Lease_LeaseTimeToLive_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Lease_LeaseTimeToLive_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Lease_LeaseTimeToLive_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Lease_LeaseTimeToLive_1(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Lease_LeaseTimeToLive_1(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Lease_LeaseLeases_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Lease_LeaseLeases_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Lease_LeaseLeases_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Lease_LeaseLeases_1, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Lease_LeaseLeases_1(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Lease_LeaseLeases_1(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	return nil
+}
+
+var (
+	pattern_Lease_LeaseGrant_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "lease", "grant"}, ""))
+
+	pattern_Lease_LeaseRevoke_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "lease", "revoke"}, ""))
+
+	pattern_Lease_LeaseRevoke_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "kv", "lease", "revoke"}, ""))
+
+	pattern_Lease_LeaseKeepAlive_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "lease", "keepalive"}, ""))
+
+	pattern_Lease_LeaseTimeToLive_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "lease", "timetolive"}, ""))
+
+	pattern_Lease_LeaseTimeToLive_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "kv", "lease", "timetolive"}, ""))
+
+	pattern_Lease_LeaseLeases_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "lease", "leases"}, ""))
+
+	pattern_Lease_LeaseLeases_1 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "kv", "lease", "leases"}, ""))
+)
+
+var (
+	forward_Lease_LeaseGrant_0 = runtime.ForwardResponseMessage
+
+	forward_Lease_LeaseRevoke_0 = runtime.ForwardResponseMessage
+
+	forward_Lease_LeaseRevoke_1 = runtime.ForwardResponseMessage
+
+	forward_Lease_LeaseKeepAlive_0 = runtime.ForwardResponseStream
+
+	forward_Lease_LeaseTimeToLive_0 = runtime.ForwardResponseMessage
+
+	forward_Lease_LeaseTimeToLive_1 = runtime.ForwardResponseMessage
+
+	forward_Lease_LeaseLeases_0 = runtime.ForwardResponseMessage
+
+	forward_Lease_LeaseLeases_1 = runtime.ForwardResponseMessage
+)
+
+// RegisterClusterHandlerFromEndpoint is same as RegisterClusterHandler but
+// automatically dials to "endpoint" and closes the connection when "ctx" gets done.
+func RegisterClusterHandlerFromEndpoint(ctx context.Context, mux *runtime.ServeMux, endpoint string, opts []grpc.DialOption) (err error) {
+	conn, err := grpc.Dial(endpoint, opts...)
+	if err != nil {
+		return err
+	}
+	defer func() {
+		if err != nil {
+			if cerr := conn.Close(); cerr != nil {
+				grpclog.Printf("Failed to close conn to %s: %v", endpoint, cerr)
+			}
+			return
+		}
+		go func() {
+			<-ctx.Done()
+			if cerr := conn.Close(); cerr != nil {
+				grpclog.Printf("Failed to close conn to %s: %v", endpoint, cerr)
+			}
+		}()
+	}()
+
+	return RegisterClusterHandler(ctx, mux, conn)
+}
+
+// RegisterClusterHandler registers the http handlers for service Cluster to "mux".
+// The handlers forward requests to the grpc endpoint over "conn".
+func RegisterClusterHandler(ctx context.Context, mux *runtime.ServeMux, conn *grpc.ClientConn) error {
+	return RegisterClusterHandlerClient(ctx, mux, etcdserverpb.NewClusterClient(conn))
+}
+
+// RegisterClusterHandler registers the http handlers for service Cluster to "mux".
+// The handlers forward requests to the grpc endpoint over the given implementation of "ClusterClient".
+// Note: the gRPC framework executes interceptors within the gRPC handler. If the passed in "ClusterClient"
+// doesn't go through the normal gRPC flow (creating a gRPC client etc.) then it will be up to the passed in
+// "ClusterClient" to call the correct interceptors.
+func RegisterClusterHandlerClient(ctx context.Context, mux *runtime.ServeMux, client etcdserverpb.ClusterClient) error {
+
+	mux.Handle("POST", pattern_Cluster_MemberAdd_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Cluster_MemberAdd_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Cluster_MemberAdd_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Cluster_MemberRemove_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Cluster_MemberRemove_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Cluster_MemberRemove_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Cluster_MemberUpdate_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Cluster_MemberUpdate_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Cluster_MemberUpdate_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Cluster_MemberList_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Cluster_MemberList_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Cluster_MemberList_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Cluster_MemberPromote_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Cluster_MemberPromote_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Cluster_MemberPromote_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	return nil
+}
+
+var (
+	pattern_Cluster_MemberAdd_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "cluster", "member", "add"}, ""))
+
+	pattern_Cluster_MemberRemove_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "cluster", "member", "remove"}, ""))
+
+	pattern_Cluster_MemberUpdate_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "cluster", "member", "update"}, ""))
+
+	pattern_Cluster_MemberList_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "cluster", "member", "list"}, ""))
+
+	pattern_Cluster_MemberPromote_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "cluster", "member", "promote"}, ""))
+)
+
+var (
+	forward_Cluster_MemberAdd_0 = runtime.ForwardResponseMessage
+
+	forward_Cluster_MemberRemove_0 = runtime.ForwardResponseMessage
+
+	forward_Cluster_MemberUpdate_0 = runtime.ForwardResponseMessage
+
+	forward_Cluster_MemberList_0 = runtime.ForwardResponseMessage
+
+	forward_Cluster_MemberPromote_0 = runtime.ForwardResponseMessage
+)
+
+// RegisterMaintenanceHandlerFromEndpoint is same as RegisterMaintenanceHandler but
+// automatically dials to "endpoint" and closes the connection when "ctx" gets done.
+func RegisterMaintenanceHandlerFromEndpoint(ctx context.Context, mux *runtime.ServeMux, endpoint string, opts []grpc.DialOption) (err error) {
+	conn, err := grpc.Dial(endpoint, opts...)
+	if err != nil {
+		return err
+	}
+	defer func() {
+		if err != nil {
+			if cerr := conn.Close(); cerr != nil {
+				grpclog.Printf("Failed to close conn to %s: %v", endpoint, cerr)
+			}
+			return
+		}
+		go func() {
+			<-ctx.Done()
+			if cerr := conn.Close(); cerr != nil {
+				grpclog.Printf("Failed to close conn to %s: %v", endpoint, cerr)
+			}
+		}()
+	}()
+
+	return RegisterMaintenanceHandler(ctx, mux, conn)
+}
+
+// RegisterMaintenanceHandler registers the http handlers for service Maintenance to "mux".
+// The handlers forward requests to the grpc endpoint over "conn".
+func RegisterMaintenanceHandler(ctx context.Context, mux *runtime.ServeMux, conn *grpc.ClientConn) error {
+	return RegisterMaintenanceHandlerClient(ctx, mux, etcdserverpb.NewMaintenanceClient(conn))
+}
+
+// RegisterMaintenanceHandler registers the http handlers for service Maintenance to "mux".
+// The handlers forward requests to the grpc endpoint over the given implementation of "MaintenanceClient".
+// Note: the gRPC framework executes interceptors within the gRPC handler. If the passed in "MaintenanceClient"
+// doesn't go through the normal gRPC flow (creating a gRPC client etc.) then it will be up to the passed in
+// "MaintenanceClient" to call the correct interceptors.
+func RegisterMaintenanceHandlerClient(ctx context.Context, mux *runtime.ServeMux, client etcdserverpb.MaintenanceClient) error {
+
+	mux.Handle("POST", pattern_Maintenance_Alarm_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Maintenance_Alarm_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Maintenance_Alarm_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Maintenance_Status_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Maintenance_Status_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Maintenance_Status_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Maintenance_Defragment_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Maintenance_Defragment_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Maintenance_Defragment_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Maintenance_Hash_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Maintenance_Hash_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Maintenance_Hash_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Maintenance_HashKV_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Maintenance_HashKV_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Maintenance_HashKV_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Maintenance_Snapshot_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Maintenance_Snapshot_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Maintenance_Snapshot_0(ctx, mux, outboundMarshaler, w, req, func() (proto.Message, error) { return resp.Recv() }, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Maintenance_MoveLeader_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Maintenance_MoveLeader_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Maintenance_MoveLeader_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	return nil
+}
+
+var (
+	pattern_Maintenance_Alarm_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "maintenance", "alarm"}, ""))
+
+	pattern_Maintenance_Status_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "maintenance", "status"}, ""))
+
+	pattern_Maintenance_Defragment_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "maintenance", "defragment"}, ""))
+
+	pattern_Maintenance_Hash_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "maintenance", "hash"}, ""))
+
+	pattern_Maintenance_HashKV_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "maintenance", "hash"}, ""))
+
+	pattern_Maintenance_Snapshot_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "maintenance", "snapshot"}, ""))
+
+	pattern_Maintenance_MoveLeader_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "maintenance", "transfer-leadership"}, ""))
+)
+
+var (
+	forward_Maintenance_Alarm_0 = runtime.ForwardResponseMessage
+
+	forward_Maintenance_Status_0 = runtime.ForwardResponseMessage
+
+	forward_Maintenance_Defragment_0 = runtime.ForwardResponseMessage
+
+	forward_Maintenance_Hash_0 = runtime.ForwardResponseMessage
+
+	forward_Maintenance_HashKV_0 = runtime.ForwardResponseMessage
+
+	forward_Maintenance_Snapshot_0 = runtime.ForwardResponseStream
+
+	forward_Maintenance_MoveLeader_0 = runtime.ForwardResponseMessage
+)
+
+// RegisterAuthHandlerFromEndpoint is same as RegisterAuthHandler but
+// automatically dials to "endpoint" and closes the connection when "ctx" gets done.
+func RegisterAuthHandlerFromEndpoint(ctx context.Context, mux *runtime.ServeMux, endpoint string, opts []grpc.DialOption) (err error) {
+	conn, err := grpc.Dial(endpoint, opts...)
+	if err != nil {
+		return err
+	}
+	defer func() {
+		if err != nil {
+			if cerr := conn.Close(); cerr != nil {
+				grpclog.Printf("Failed to close conn to %s: %v", endpoint, cerr)
+			}
+			return
+		}
+		go func() {
+			<-ctx.Done()
+			if cerr := conn.Close(); cerr != nil {
+				grpclog.Printf("Failed to close conn to %s: %v", endpoint, cerr)
+			}
+		}()
+	}()
+
+	return RegisterAuthHandler(ctx, mux, conn)
+}
+
+// RegisterAuthHandler registers the http handlers for service Auth to "mux".
+// The handlers forward requests to the grpc endpoint over "conn".
+func RegisterAuthHandler(ctx context.Context, mux *runtime.ServeMux, conn *grpc.ClientConn) error {
+	return RegisterAuthHandlerClient(ctx, mux, etcdserverpb.NewAuthClient(conn))
+}
+
+// RegisterAuthHandler registers the http handlers for service Auth to "mux".
+// The handlers forward requests to the grpc endpoint over the given implementation of "AuthClient".
+// Note: the gRPC framework executes interceptors within the gRPC handler. If the passed in "AuthClient"
+// doesn't go through the normal gRPC flow (creating a gRPC client etc.) then it will be up to the passed in
+// "AuthClient" to call the correct interceptors.
+func RegisterAuthHandlerClient(ctx context.Context, mux *runtime.ServeMux, client etcdserverpb.AuthClient) error {
+
+	mux.Handle("POST", pattern_Auth_AuthEnable_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Auth_AuthEnable_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Auth_AuthEnable_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Auth_AuthDisable_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Auth_AuthDisable_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Auth_AuthDisable_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Auth_Authenticate_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Auth_Authenticate_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Auth_Authenticate_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Auth_UserAdd_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Auth_UserAdd_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Auth_UserAdd_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Auth_UserGet_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Auth_UserGet_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Auth_UserGet_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Auth_UserList_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Auth_UserList_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Auth_UserList_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Auth_UserDelete_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Auth_UserDelete_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Auth_UserDelete_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Auth_UserChangePassword_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Auth_UserChangePassword_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Auth_UserChangePassword_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Auth_UserGrantRole_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Auth_UserGrantRole_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Auth_UserGrantRole_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Auth_UserRevokeRole_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Auth_UserRevokeRole_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Auth_UserRevokeRole_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Auth_RoleAdd_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Auth_RoleAdd_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Auth_RoleAdd_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Auth_RoleGet_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Auth_RoleGet_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Auth_RoleGet_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Auth_RoleList_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Auth_RoleList_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Auth_RoleList_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Auth_RoleDelete_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Auth_RoleDelete_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Auth_RoleDelete_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Auth_RoleGrantPermission_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Auth_RoleGrantPermission_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Auth_RoleGrantPermission_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	mux.Handle("POST", pattern_Auth_RoleRevokePermission_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(req.Context())
+		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, mux, req)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+		resp, md, err := request_Auth_RoleRevokePermission_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Auth_RoleRevokePermission_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
+	return nil
+}
+
+var (
+	pattern_Auth_AuthEnable_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "auth", "enable"}, ""))
+
+	pattern_Auth_AuthDisable_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "auth", "disable"}, ""))
+
+	pattern_Auth_Authenticate_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3", "auth", "authenticate"}, ""))
+
+	pattern_Auth_UserAdd_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "auth", "user", "add"}, ""))
+
+	pattern_Auth_UserGet_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "auth", "user", "get"}, ""))
+
+	pattern_Auth_UserList_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "auth", "user", "list"}, ""))
+
+	pattern_Auth_UserDelete_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "auth", "user", "delete"}, ""))
+
+	pattern_Auth_UserChangePassword_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "auth", "user", "changepw"}, ""))
+
+	pattern_Auth_UserGrantRole_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "auth", "user", "grant"}, ""))
+
+	pattern_Auth_UserRevokeRole_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "auth", "user", "revoke"}, ""))
+
+	pattern_Auth_RoleAdd_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "auth", "role", "add"}, ""))
+
+	pattern_Auth_RoleGet_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "auth", "role", "get"}, ""))
+
+	pattern_Auth_RoleList_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "auth", "role", "list"}, ""))
+
+	pattern_Auth_RoleDelete_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "auth", "role", "delete"}, ""))
+
+	pattern_Auth_RoleGrantPermission_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "auth", "role", "grant"}, ""))
+
+	pattern_Auth_RoleRevokePermission_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 2, 3}, []string{"v3", "auth", "role", "revoke"}, ""))
+)
+
+var (
+	forward_Auth_AuthEnable_0 = runtime.ForwardResponseMessage
+
+	forward_Auth_AuthDisable_0 = runtime.ForwardResponseMessage
+
+	forward_Auth_Authenticate_0 = runtime.ForwardResponseMessage
+
+	forward_Auth_UserAdd_0 = runtime.ForwardResponseMessage
+
+	forward_Auth_UserGet_0 = runtime.ForwardResponseMessage
+
+	forward_Auth_UserList_0 = runtime.ForwardResponseMessage
+
+	forward_Auth_UserDelete_0 = runtime.ForwardResponseMessage
+
+	forward_Auth_UserChangePassword_0 = runtime.ForwardResponseMessage
+
+	forward_Auth_UserGrantRole_0 = runtime.ForwardResponseMessage
+
+	forward_Auth_UserRevokeRole_0 = runtime.ForwardResponseMessage
+
+	forward_Auth_RoleAdd_0 = runtime.ForwardResponseMessage
+
+	forward_Auth_RoleGet_0 = runtime.ForwardResponseMessage
+
+	forward_Auth_RoleList_0 = runtime.ForwardResponseMessage
+
+	forward_Auth_RoleDelete_0 = runtime.ForwardResponseMessage
+
+	forward_Auth_RoleGrantPermission_0 = runtime.ForwardResponseMessage
+
+	forward_Auth_RoleRevokePermission_0 = runtime.ForwardResponseMessage
+)
diff --git a/vendor/go.etcd.io/etcd/etcdserver/metrics.go b/vendor/go.etcd.io/etcd/etcdserver/metrics.go
new file mode 100644
index 0000000..e0c0cde
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/metrics.go
@@ -0,0 +1,221 @@
+// Copyright 2015 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import (
+	goruntime "runtime"
+	"time"
+
+	"go.etcd.io/etcd/pkg/runtime"
+	"go.etcd.io/etcd/version"
+
+	"github.com/prometheus/client_golang/prometheus"
+	"go.uber.org/zap"
+)
+
+var (
+	hasLeader = prometheus.NewGauge(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "has_leader",
+		Help:      "Whether or not a leader exists. 1 is existence, 0 is not.",
+	})
+	isLeader = prometheus.NewGauge(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "is_leader",
+		Help:      "Whether or not this member is a leader. 1 if is, 0 otherwise.",
+	})
+	leaderChanges = prometheus.NewCounter(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "leader_changes_seen_total",
+		Help:      "The number of leader changes seen.",
+	})
+	isLearner = prometheus.NewGauge(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "is_learner",
+		Help:      "Whether or not this member is a learner. 1 if is, 0 otherwise.",
+	})
+	learnerPromoteFailed = prometheus.NewCounterVec(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "learner_promote_failures",
+		Help:      "The total number of failed learner promotions (likely learner not ready) while this member is leader.",
+	},
+		[]string{"Reason"},
+	)
+	learnerPromoteSucceed = prometheus.NewCounter(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "learner_promote_successes",
+		Help:      "The total number of successful learner promotions while this member is leader.",
+	})
+	heartbeatSendFailures = prometheus.NewCounter(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "heartbeat_send_failures_total",
+		Help:      "The total number of leader heartbeat send failures (likely overloaded from slow disk).",
+	})
+	slowApplies = prometheus.NewCounter(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "slow_apply_total",
+		Help:      "The total number of slow apply requests (likely overloaded from slow disk).",
+	})
+	applySnapshotInProgress = prometheus.NewGauge(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "snapshot_apply_in_progress_total",
+		Help:      "1 if the server is applying the incoming snapshot. 0 if none.",
+	})
+	proposalsCommitted = prometheus.NewGauge(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "proposals_committed_total",
+		Help:      "The total number of consensus proposals committed.",
+	})
+	proposalsApplied = prometheus.NewGauge(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "proposals_applied_total",
+		Help:      "The total number of consensus proposals applied.",
+	})
+	proposalsPending = prometheus.NewGauge(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "proposals_pending",
+		Help:      "The current number of pending proposals to commit.",
+	})
+	proposalsFailed = prometheus.NewCounter(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "proposals_failed_total",
+		Help:      "The total number of failed proposals seen.",
+	})
+	slowReadIndex = prometheus.NewCounter(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "slow_read_indexes_total",
+		Help:      "The total number of pending read indexes not in sync with leader's or timed out read index requests.",
+	})
+	readIndexFailed = prometheus.NewCounter(prometheus.CounterOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "read_indexes_failed_total",
+		Help:      "The total number of failed read indexes seen.",
+	})
+	leaseExpired = prometheus.NewCounter(prometheus.CounterOpts{
+		Namespace: "etcd_debugging",
+		Subsystem: "server",
+		Name:      "lease_expired_total",
+		Help:      "The total number of expired leases.",
+	})
+	quotaBackendBytes = prometheus.NewGauge(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "quota_backend_bytes",
+		Help:      "Current backend storage quota size in bytes.",
+	})
+	currentVersion = prometheus.NewGaugeVec(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "version",
+		Help:      "Which version is running. 1 for 'server_version' label with current version.",
+	},
+		[]string{"server_version"})
+	currentGoVersion = prometheus.NewGaugeVec(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "go_version",
+		Help:      "Which Go version server is running with. 1 for 'server_go_version' label with current version.",
+	},
+		[]string{"server_go_version"})
+	serverID = prometheus.NewGaugeVec(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "server",
+		Name:      "id",
+		Help:      "Server or member ID in hexadecimal format. 1 for 'server_id' label with current ID.",
+	},
+		[]string{"server_id"})
+)
+
+func init() {
+	prometheus.MustRegister(hasLeader)
+	prometheus.MustRegister(isLeader)
+	prometheus.MustRegister(leaderChanges)
+	prometheus.MustRegister(heartbeatSendFailures)
+	prometheus.MustRegister(slowApplies)
+	prometheus.MustRegister(applySnapshotInProgress)
+	prometheus.MustRegister(proposalsCommitted)
+	prometheus.MustRegister(proposalsApplied)
+	prometheus.MustRegister(proposalsPending)
+	prometheus.MustRegister(proposalsFailed)
+	prometheus.MustRegister(slowReadIndex)
+	prometheus.MustRegister(readIndexFailed)
+	prometheus.MustRegister(leaseExpired)
+	prometheus.MustRegister(quotaBackendBytes)
+	prometheus.MustRegister(currentVersion)
+	prometheus.MustRegister(currentGoVersion)
+	prometheus.MustRegister(serverID)
+	prometheus.MustRegister(isLearner)
+	prometheus.MustRegister(learnerPromoteSucceed)
+	prometheus.MustRegister(learnerPromoteFailed)
+
+	currentVersion.With(prometheus.Labels{
+		"server_version": version.Version,
+	}).Set(1)
+	currentGoVersion.With(prometheus.Labels{
+		"server_go_version": goruntime.Version(),
+	}).Set(1)
+}
+
+func monitorFileDescriptor(lg *zap.Logger, done <-chan struct{}) {
+	ticker := time.NewTicker(5 * time.Second)
+	defer ticker.Stop()
+	for {
+		used, err := runtime.FDUsage()
+		if err != nil {
+			if lg != nil {
+				lg.Warn("failed to get file descriptor usage", zap.Error(err))
+			} else {
+				plog.Errorf("cannot monitor file descriptor usage (%v)", err)
+			}
+			return
+		}
+		limit, err := runtime.FDLimit()
+		if err != nil {
+			if lg != nil {
+				lg.Warn("failed to get file descriptor limit", zap.Error(err))
+			} else {
+				plog.Errorf("cannot monitor file descriptor usage (%v)", err)
+			}
+			return
+		}
+		if used >= limit/5*4 {
+			if lg != nil {
+				lg.Warn("80% of file descriptors are used", zap.Uint64("used", used), zap.Uint64("limit", limit))
+			} else {
+				plog.Warningf("80%% of the file descriptor limit is used [used = %d, limit = %d]", used, limit)
+			}
+		}
+		select {
+		case <-ticker.C:
+		case <-done:
+			return
+		}
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/quota.go b/vendor/go.etcd.io/etcd/etcdserver/quota.go
new file mode 100644
index 0000000..6d70430
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/quota.go
@@ -0,0 +1,182 @@
+// Copyright 2016 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import (
+	"sync"
+
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+
+	humanize "github.com/dustin/go-humanize"
+	"go.uber.org/zap"
+)
+
+const (
+	// DefaultQuotaBytes is the number of bytes the backend Size may
+	// consume before exceeding the space quota.
+	DefaultQuotaBytes = int64(2 * 1024 * 1024 * 1024) // 2GB
+	// MaxQuotaBytes is the maximum number of bytes suggested for a backend
+	// quota. A larger quota may lead to degraded performance.
+	MaxQuotaBytes = int64(8 * 1024 * 1024 * 1024) // 8GB
+)
+
+// Quota represents an arbitrary quota against arbitrary requests. Each request
+// costs some charge; if there is not enough remaining charge, then there are
+// too few resources available within the quota to apply the request.
+type Quota interface {
+	// Available judges whether the given request fits within the quota.
+	Available(req interface{}) bool
+	// Cost computes the charge against the quota for a given request.
+	Cost(req interface{}) int
+	// Remaining is the amount of charge left for the quota.
+	Remaining() int64
+}
+
+type passthroughQuota struct{}
+
+func (*passthroughQuota) Available(interface{}) bool { return true }
+func (*passthroughQuota) Cost(interface{}) int       { return 0 }
+func (*passthroughQuota) Remaining() int64           { return 1 }
+
+type backendQuota struct {
+	s               *EtcdServer
+	maxBackendBytes int64
+}
+
+const (
+	// leaseOverhead is an estimate for the cost of storing a lease
+	leaseOverhead = 64
+	// kvOverhead is an estimate for the cost of storing a key's metadata
+	kvOverhead = 256
+)
+
+var (
+	// only log once
+	quotaLogOnce sync.Once
+
+	DefaultQuotaSize = humanize.Bytes(uint64(DefaultQuotaBytes))
+	maxQuotaSize     = humanize.Bytes(uint64(MaxQuotaBytes))
+)
+
+// NewBackendQuota creates a quota layer with the given storage limit.
+func NewBackendQuota(s *EtcdServer, name string) Quota {
+	lg := s.getLogger()
+	quotaBackendBytes.Set(float64(s.Cfg.QuotaBackendBytes))
+
+	if s.Cfg.QuotaBackendBytes < 0 {
+		// disable quotas if negative
+		quotaLogOnce.Do(func() {
+			if lg != nil {
+				lg.Info(
+					"disabled backend quota",
+					zap.String("quota-name", name),
+					zap.Int64("quota-size-bytes", s.Cfg.QuotaBackendBytes),
+				)
+			} else {
+				plog.Warningf("disabling backend quota")
+			}
+		})
+		return &passthroughQuota{}
+	}
+
+	if s.Cfg.QuotaBackendBytes == 0 {
+		// use default size if no quota size given
+		quotaLogOnce.Do(func() {
+			if lg != nil {
+				lg.Info(
+					"enabled backend quota with default value",
+					zap.String("quota-name", name),
+					zap.Int64("quota-size-bytes", DefaultQuotaBytes),
+					zap.String("quota-size", DefaultQuotaSize),
+				)
+			}
+		})
+		quotaBackendBytes.Set(float64(DefaultQuotaBytes))
+		return &backendQuota{s, DefaultQuotaBytes}
+	}
+
+	quotaLogOnce.Do(func() {
+		if s.Cfg.QuotaBackendBytes > MaxQuotaBytes {
+			if lg != nil {
+				lg.Warn(
+					"quota exceeds the maximum value",
+					zap.String("quota-name", name),
+					zap.Int64("quota-size-bytes", s.Cfg.QuotaBackendBytes),
+					zap.String("quota-size", humanize.Bytes(uint64(s.Cfg.QuotaBackendBytes))),
+					zap.Int64("quota-maximum-size-bytes", MaxQuotaBytes),
+					zap.String("quota-maximum-size", maxQuotaSize),
+				)
+			} else {
+				plog.Warningf("backend quota %v exceeds maximum recommended quota %v", s.Cfg.QuotaBackendBytes, MaxQuotaBytes)
+			}
+		}
+		if lg != nil {
+			lg.Info(
+				"enabled backend quota",
+				zap.String("quota-name", name),
+				zap.Int64("quota-size-bytes", s.Cfg.QuotaBackendBytes),
+				zap.String("quota-size", humanize.Bytes(uint64(s.Cfg.QuotaBackendBytes))),
+			)
+		}
+	})
+	return &backendQuota{s, s.Cfg.QuotaBackendBytes}
+}
+
+func (b *backendQuota) Available(v interface{}) bool {
+	// TODO: maybe optimize backend.Size()
+	return b.s.Backend().Size()+int64(b.Cost(v)) < b.maxBackendBytes
+}
+
+func (b *backendQuota) Cost(v interface{}) int {
+	switch r := v.(type) {
+	case *pb.PutRequest:
+		return costPut(r)
+	case *pb.TxnRequest:
+		return costTxn(r)
+	case *pb.LeaseGrantRequest:
+		return leaseOverhead
+	default:
+		panic("unexpected cost")
+	}
+}
+
+func costPut(r *pb.PutRequest) int { return kvOverhead + len(r.Key) + len(r.Value) }
+
+func costTxnReq(u *pb.RequestOp) int {
+	r := u.GetRequestPut()
+	if r == nil {
+		return 0
+	}
+	return costPut(r)
+}
+
+func costTxn(r *pb.TxnRequest) int {
+	sizeSuccess := 0
+	for _, u := range r.Success {
+		sizeSuccess += costTxnReq(u)
+	}
+	sizeFailure := 0
+	for _, u := range r.Failure {
+		sizeFailure += costTxnReq(u)
+	}
+	if sizeFailure > sizeSuccess {
+		return sizeFailure
+	}
+	return sizeSuccess
+}
+
+func (b *backendQuota) Remaining() int64 {
+	return b.maxBackendBytes - b.s.Backend().Size()
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/raft.go b/vendor/go.etcd.io/etcd/etcdserver/raft.go
new file mode 100644
index 0000000..c0fe979
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/raft.go
@@ -0,0 +1,753 @@
+// Copyright 2015 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import (
+	"encoding/json"
+	"expvar"
+	"fmt"
+	"log"
+	"sort"
+	"sync"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver/api/membership"
+	"go.etcd.io/etcd/etcdserver/api/rafthttp"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/pkg/contention"
+	"go.etcd.io/etcd/pkg/logutil"
+	"go.etcd.io/etcd/pkg/pbutil"
+	"go.etcd.io/etcd/pkg/types"
+	"go.etcd.io/etcd/raft"
+	"go.etcd.io/etcd/raft/raftpb"
+	"go.etcd.io/etcd/wal"
+	"go.etcd.io/etcd/wal/walpb"
+	"go.uber.org/zap"
+)
+
+const (
+	// The max throughput of etcd will not exceed 100MB/s (100K * 1KB value).
+	// Assuming the RTT is around 10ms, 1MB max size is large enough.
+	maxSizePerMsg = 1 * 1024 * 1024
+	// Never overflow the rafthttp buffer, which is 4096.
+	// TODO: a better const?
+	maxInflightMsgs = 4096 / 8
+)
+
+var (
+	// protects raftStatus
+	raftStatusMu sync.Mutex
+	// indirection for expvar func interface
+	// expvar panics when publishing duplicate name
+	// expvar does not support remove a registered name
+	// so only register a func that calls raftStatus
+	// and change raftStatus as we need.
+	raftStatus func() raft.Status
+)
+
+func init() {
+	expvar.Publish("raft.status", expvar.Func(func() interface{} {
+		raftStatusMu.Lock()
+		defer raftStatusMu.Unlock()
+		return raftStatus()
+	}))
+}
+
+// apply contains entries, snapshot to be applied. Once
+// an apply is consumed, the entries will be persisted to
+// to raft storage concurrently; the application must read
+// raftDone before assuming the raft messages are stable.
+type apply struct {
+	entries  []raftpb.Entry
+	snapshot raftpb.Snapshot
+	// notifyc synchronizes etcd server applies with the raft node
+	notifyc chan struct{}
+}
+
+type raftNode struct {
+	lg *zap.Logger
+
+	tickMu *sync.Mutex
+	raftNodeConfig
+
+	// a chan to send/receive snapshot
+	msgSnapC chan raftpb.Message
+
+	// a chan to send out apply
+	applyc chan apply
+
+	// a chan to send out readState
+	readStateC chan raft.ReadState
+
+	// utility
+	ticker *time.Ticker
+	// contention detectors for raft heartbeat message
+	td *contention.TimeoutDetector
+
+	stopped chan struct{}
+	done    chan struct{}
+}
+
+type raftNodeConfig struct {
+	lg *zap.Logger
+
+	// to check if msg receiver is removed from cluster
+	isIDRemoved func(id uint64) bool
+	raft.Node
+	raftStorage *raft.MemoryStorage
+	storage     Storage
+	heartbeat   time.Duration // for logging
+	// transport specifies the transport to send and receive msgs to members.
+	// Sending messages MUST NOT block. It is okay to drop messages, since
+	// clients should timeout and reissue their messages.
+	// If transport is nil, server will panic.
+	transport rafthttp.Transporter
+}
+
+func newRaftNode(cfg raftNodeConfig) *raftNode {
+	var lg raft.Logger
+	if cfg.lg != nil {
+		lg = logutil.NewRaftLoggerZap(cfg.lg)
+	} else {
+		lcfg := logutil.DefaultZapLoggerConfig
+		var err error
+		lg, err = logutil.NewRaftLogger(&lcfg)
+		if err != nil {
+			log.Fatalf("cannot create raft logger %v", err)
+		}
+	}
+	raft.SetLogger(lg)
+	r := &raftNode{
+		lg:             cfg.lg,
+		tickMu:         new(sync.Mutex),
+		raftNodeConfig: cfg,
+		// set up contention detectors for raft heartbeat message.
+		// expect to send a heartbeat within 2 heartbeat intervals.
+		td:         contention.NewTimeoutDetector(2 * cfg.heartbeat),
+		readStateC: make(chan raft.ReadState, 1),
+		msgSnapC:   make(chan raftpb.Message, maxInFlightMsgSnap),
+		applyc:     make(chan apply),
+		stopped:    make(chan struct{}),
+		done:       make(chan struct{}),
+	}
+	if r.heartbeat == 0 {
+		r.ticker = &time.Ticker{}
+	} else {
+		r.ticker = time.NewTicker(r.heartbeat)
+	}
+	return r
+}
+
+// raft.Node does not have locks in Raft package
+func (r *raftNode) tick() {
+	r.tickMu.Lock()
+	r.Tick()
+	r.tickMu.Unlock()
+}
+
+// start prepares and starts raftNode in a new goroutine. It is no longer safe
+// to modify the fields after it has been started.
+func (r *raftNode) start(rh *raftReadyHandler) {
+	internalTimeout := time.Second
+
+	go func() {
+		defer r.onStop()
+		islead := false
+
+		for {
+			select {
+			case <-r.ticker.C:
+				r.tick()
+			case rd := <-r.Ready():
+				if rd.SoftState != nil {
+					newLeader := rd.SoftState.Lead != raft.None && rh.getLead() != rd.SoftState.Lead
+					if newLeader {
+						leaderChanges.Inc()
+					}
+
+					if rd.SoftState.Lead == raft.None {
+						hasLeader.Set(0)
+					} else {
+						hasLeader.Set(1)
+					}
+
+					rh.updateLead(rd.SoftState.Lead)
+					islead = rd.RaftState == raft.StateLeader
+					if islead {
+						isLeader.Set(1)
+					} else {
+						isLeader.Set(0)
+					}
+					rh.updateLeadership(newLeader)
+					r.td.Reset()
+				}
+
+				if len(rd.ReadStates) != 0 {
+					select {
+					case r.readStateC <- rd.ReadStates[len(rd.ReadStates)-1]:
+					case <-time.After(internalTimeout):
+						if r.lg != nil {
+							r.lg.Warn("timed out sending read state", zap.Duration("timeout", internalTimeout))
+						} else {
+							plog.Warningf("timed out sending read state")
+						}
+					case <-r.stopped:
+						return
+					}
+				}
+
+				notifyc := make(chan struct{}, 1)
+				ap := apply{
+					entries:  rd.CommittedEntries,
+					snapshot: rd.Snapshot,
+					notifyc:  notifyc,
+				}
+
+				updateCommittedIndex(&ap, rh)
+
+				select {
+				case r.applyc <- ap:
+				case <-r.stopped:
+					return
+				}
+
+				// the leader can write to its disk in parallel with replicating to the followers and them
+				// writing to their disks.
+				// For more details, check raft thesis 10.2.1
+				if islead {
+					// gofail: var raftBeforeLeaderSend struct{}
+					r.transport.Send(r.processMessages(rd.Messages))
+				}
+
+				// gofail: var raftBeforeSave struct{}
+				if err := r.storage.Save(rd.HardState, rd.Entries); err != nil {
+					if r.lg != nil {
+						r.lg.Fatal("failed to save Raft hard state and entries", zap.Error(err))
+					} else {
+						plog.Fatalf("raft save state and entries error: %v", err)
+					}
+				}
+				if !raft.IsEmptyHardState(rd.HardState) {
+					proposalsCommitted.Set(float64(rd.HardState.Commit))
+				}
+				// gofail: var raftAfterSave struct{}
+
+				if !raft.IsEmptySnap(rd.Snapshot) {
+					// gofail: var raftBeforeSaveSnap struct{}
+					if err := r.storage.SaveSnap(rd.Snapshot); err != nil {
+						if r.lg != nil {
+							r.lg.Fatal("failed to save Raft snapshot", zap.Error(err))
+						} else {
+							plog.Fatalf("raft save snapshot error: %v", err)
+						}
+					}
+					// etcdserver now claim the snapshot has been persisted onto the disk
+					notifyc <- struct{}{}
+
+					// gofail: var raftAfterSaveSnap struct{}
+					r.raftStorage.ApplySnapshot(rd.Snapshot)
+					if r.lg != nil {
+						r.lg.Info("applied incoming Raft snapshot", zap.Uint64("snapshot-index", rd.Snapshot.Metadata.Index))
+					} else {
+						plog.Infof("raft applied incoming snapshot at index %d", rd.Snapshot.Metadata.Index)
+					}
+					// gofail: var raftAfterApplySnap struct{}
+				}
+
+				r.raftStorage.Append(rd.Entries)
+
+				if !islead {
+					// finish processing incoming messages before we signal raftdone chan
+					msgs := r.processMessages(rd.Messages)
+
+					// now unblocks 'applyAll' that waits on Raft log disk writes before triggering snapshots
+					notifyc <- struct{}{}
+
+					// Candidate or follower needs to wait for all pending configuration
+					// changes to be applied before sending messages.
+					// Otherwise we might incorrectly count votes (e.g. votes from removed members).
+					// Also slow machine's follower raft-layer could proceed to become the leader
+					// on its own single-node cluster, before apply-layer applies the config change.
+					// We simply wait for ALL pending entries to be applied for now.
+					// We might improve this later on if it causes unnecessary long blocking issues.
+					waitApply := false
+					for _, ent := range rd.CommittedEntries {
+						if ent.Type == raftpb.EntryConfChange {
+							waitApply = true
+							break
+						}
+					}
+					if waitApply {
+						// blocks until 'applyAll' calls 'applyWait.Trigger'
+						// to be in sync with scheduled config-change job
+						// (assume notifyc has cap of 1)
+						select {
+						case notifyc <- struct{}{}:
+						case <-r.stopped:
+							return
+						}
+					}
+
+					// gofail: var raftBeforeFollowerSend struct{}
+					r.transport.Send(msgs)
+				} else {
+					// leader already processed 'MsgSnap' and signaled
+					notifyc <- struct{}{}
+				}
+
+				r.Advance()
+			case <-r.stopped:
+				return
+			}
+		}
+	}()
+}
+
+func updateCommittedIndex(ap *apply, rh *raftReadyHandler) {
+	var ci uint64
+	if len(ap.entries) != 0 {
+		ci = ap.entries[len(ap.entries)-1].Index
+	}
+	if ap.snapshot.Metadata.Index > ci {
+		ci = ap.snapshot.Metadata.Index
+	}
+	if ci != 0 {
+		rh.updateCommittedIndex(ci)
+	}
+}
+
+func (r *raftNode) processMessages(ms []raftpb.Message) []raftpb.Message {
+	sentAppResp := false
+	for i := len(ms) - 1; i >= 0; i-- {
+		if r.isIDRemoved(ms[i].To) {
+			ms[i].To = 0
+		}
+
+		if ms[i].Type == raftpb.MsgAppResp {
+			if sentAppResp {
+				ms[i].To = 0
+			} else {
+				sentAppResp = true
+			}
+		}
+
+		if ms[i].Type == raftpb.MsgSnap {
+			// There are two separate data store: the store for v2, and the KV for v3.
+			// The msgSnap only contains the most recent snapshot of store without KV.
+			// So we need to redirect the msgSnap to etcd server main loop for merging in the
+			// current store snapshot and KV snapshot.
+			select {
+			case r.msgSnapC <- ms[i]:
+			default:
+				// drop msgSnap if the inflight chan if full.
+			}
+			ms[i].To = 0
+		}
+		if ms[i].Type == raftpb.MsgHeartbeat {
+			ok, exceed := r.td.Observe(ms[i].To)
+			if !ok {
+				// TODO: limit request rate.
+				if r.lg != nil {
+					r.lg.Warn(
+						"leader failed to send out heartbeat on time; took too long, leader is overloaded likely from slow disk",
+						zap.String("to", fmt.Sprintf("%x", ms[i].To)),
+						zap.Duration("heartbeat-interval", r.heartbeat),
+						zap.Duration("expected-duration", 2*r.heartbeat),
+						zap.Duration("exceeded-duration", exceed),
+					)
+				} else {
+					plog.Warningf("failed to send out heartbeat on time (exceeded the %v timeout for %v, to %x)", r.heartbeat, exceed, ms[i].To)
+					plog.Warningf("server is likely overloaded")
+				}
+				heartbeatSendFailures.Inc()
+			}
+		}
+	}
+	return ms
+}
+
+func (r *raftNode) apply() chan apply {
+	return r.applyc
+}
+
+func (r *raftNode) stop() {
+	r.stopped <- struct{}{}
+	<-r.done
+}
+
+func (r *raftNode) onStop() {
+	r.Stop()
+	r.ticker.Stop()
+	r.transport.Stop()
+	if err := r.storage.Close(); err != nil {
+		if r.lg != nil {
+			r.lg.Panic("failed to close Raft storage", zap.Error(err))
+		} else {
+			plog.Panicf("raft close storage error: %v", err)
+		}
+	}
+	close(r.done)
+}
+
+// for testing
+func (r *raftNode) pauseSending() {
+	p := r.transport.(rafthttp.Pausable)
+	p.Pause()
+}
+
+func (r *raftNode) resumeSending() {
+	p := r.transport.(rafthttp.Pausable)
+	p.Resume()
+}
+
+// advanceTicks advances ticks of Raft node.
+// This can be used for fast-forwarding election
+// ticks in multi data-center deployments, thus
+// speeding up election process.
+func (r *raftNode) advanceTicks(ticks int) {
+	for i := 0; i < ticks; i++ {
+		r.tick()
+	}
+}
+
+func startNode(cfg ServerConfig, cl *membership.RaftCluster, ids []types.ID) (id types.ID, n raft.Node, s *raft.MemoryStorage, w *wal.WAL) {
+	var err error
+	member := cl.MemberByName(cfg.Name)
+	metadata := pbutil.MustMarshal(
+		&pb.Metadata{
+			NodeID:    uint64(member.ID),
+			ClusterID: uint64(cl.ID()),
+		},
+	)
+	if w, err = wal.Create(cfg.Logger, cfg.WALDir(), metadata); err != nil {
+		if cfg.Logger != nil {
+			cfg.Logger.Panic("failed to create WAL", zap.Error(err))
+		} else {
+			plog.Panicf("create wal error: %v", err)
+		}
+	}
+	peers := make([]raft.Peer, len(ids))
+	for i, id := range ids {
+		var ctx []byte
+		ctx, err = json.Marshal((*cl).Member(id))
+		if err != nil {
+			if cfg.Logger != nil {
+				cfg.Logger.Panic("failed to marshal member", zap.Error(err))
+			} else {
+				plog.Panicf("marshal member should never fail: %v", err)
+			}
+		}
+		peers[i] = raft.Peer{ID: uint64(id), Context: ctx}
+	}
+	id = member.ID
+	if cfg.Logger != nil {
+		cfg.Logger.Info(
+			"starting local member",
+			zap.String("local-member-id", id.String()),
+			zap.String("cluster-id", cl.ID().String()),
+		)
+	} else {
+		plog.Infof("starting member %s in cluster %s", id, cl.ID())
+	}
+	s = raft.NewMemoryStorage()
+	c := &raft.Config{
+		ID:              uint64(id),
+		ElectionTick:    cfg.ElectionTicks,
+		HeartbeatTick:   1,
+		Storage:         s,
+		MaxSizePerMsg:   maxSizePerMsg,
+		MaxInflightMsgs: maxInflightMsgs,
+		CheckQuorum:     true,
+		PreVote:         cfg.PreVote,
+	}
+	if cfg.Logger != nil {
+		// called after capnslog setting in "init" function
+		if cfg.LoggerConfig != nil {
+			c.Logger, err = logutil.NewRaftLogger(cfg.LoggerConfig)
+			if err != nil {
+				log.Fatalf("cannot create raft logger %v", err)
+			}
+		} else if cfg.LoggerCore != nil && cfg.LoggerWriteSyncer != nil {
+			c.Logger = logutil.NewRaftLoggerFromZapCore(cfg.LoggerCore, cfg.LoggerWriteSyncer)
+		}
+	}
+
+	if len(peers) == 0 {
+		n = raft.RestartNode(c)
+	} else {
+		n = raft.StartNode(c, peers)
+	}
+	raftStatusMu.Lock()
+	raftStatus = n.Status
+	raftStatusMu.Unlock()
+	return id, n, s, w
+}
+
+func restartNode(cfg ServerConfig, snapshot *raftpb.Snapshot) (types.ID, *membership.RaftCluster, raft.Node, *raft.MemoryStorage, *wal.WAL) {
+	var walsnap walpb.Snapshot
+	if snapshot != nil {
+		walsnap.Index, walsnap.Term = snapshot.Metadata.Index, snapshot.Metadata.Term
+	}
+	w, id, cid, st, ents := readWAL(cfg.Logger, cfg.WALDir(), walsnap)
+
+	if cfg.Logger != nil {
+		cfg.Logger.Info(
+			"restarting local member",
+			zap.String("cluster-id", cid.String()),
+			zap.String("local-member-id", id.String()),
+			zap.Uint64("commit-index", st.Commit),
+		)
+	} else {
+		plog.Infof("restarting member %s in cluster %s at commit index %d", id, cid, st.Commit)
+	}
+	cl := membership.NewCluster(cfg.Logger, "")
+	cl.SetID(id, cid)
+	s := raft.NewMemoryStorage()
+	if snapshot != nil {
+		s.ApplySnapshot(*snapshot)
+	}
+	s.SetHardState(st)
+	s.Append(ents)
+	c := &raft.Config{
+		ID:              uint64(id),
+		ElectionTick:    cfg.ElectionTicks,
+		HeartbeatTick:   1,
+		Storage:         s,
+		MaxSizePerMsg:   maxSizePerMsg,
+		MaxInflightMsgs: maxInflightMsgs,
+		CheckQuorum:     true,
+		PreVote:         cfg.PreVote,
+	}
+	if cfg.Logger != nil {
+		// called after capnslog setting in "init" function
+		var err error
+		if cfg.LoggerConfig != nil {
+			c.Logger, err = logutil.NewRaftLogger(cfg.LoggerConfig)
+			if err != nil {
+				log.Fatalf("cannot create raft logger %v", err)
+			}
+		} else if cfg.LoggerCore != nil && cfg.LoggerWriteSyncer != nil {
+			c.Logger = logutil.NewRaftLoggerFromZapCore(cfg.LoggerCore, cfg.LoggerWriteSyncer)
+		}
+	}
+
+	n := raft.RestartNode(c)
+	raftStatusMu.Lock()
+	raftStatus = n.Status
+	raftStatusMu.Unlock()
+	return id, cl, n, s, w
+}
+
+func restartAsStandaloneNode(cfg ServerConfig, snapshot *raftpb.Snapshot) (types.ID, *membership.RaftCluster, raft.Node, *raft.MemoryStorage, *wal.WAL) {
+	var walsnap walpb.Snapshot
+	if snapshot != nil {
+		walsnap.Index, walsnap.Term = snapshot.Metadata.Index, snapshot.Metadata.Term
+	}
+	w, id, cid, st, ents := readWAL(cfg.Logger, cfg.WALDir(), walsnap)
+
+	// discard the previously uncommitted entries
+	for i, ent := range ents {
+		if ent.Index > st.Commit {
+			if cfg.Logger != nil {
+				cfg.Logger.Info(
+					"discarding uncommitted WAL entries",
+					zap.Uint64("entry-index", ent.Index),
+					zap.Uint64("commit-index-from-wal", st.Commit),
+					zap.Int("number-of-discarded-entries", len(ents)-i),
+				)
+			} else {
+				plog.Infof("discarding %d uncommitted WAL entries ", len(ents)-i)
+			}
+			ents = ents[:i]
+			break
+		}
+	}
+
+	// force append the configuration change entries
+	toAppEnts := createConfigChangeEnts(
+		cfg.Logger,
+		getIDs(cfg.Logger, snapshot, ents),
+		uint64(id),
+		st.Term,
+		st.Commit,
+	)
+	ents = append(ents, toAppEnts...)
+
+	// force commit newly appended entries
+	err := w.Save(raftpb.HardState{}, toAppEnts)
+	if err != nil {
+		if cfg.Logger != nil {
+			cfg.Logger.Fatal("failed to save hard state and entries", zap.Error(err))
+		} else {
+			plog.Fatalf("%v", err)
+		}
+	}
+	if len(ents) != 0 {
+		st.Commit = ents[len(ents)-1].Index
+	}
+
+	if cfg.Logger != nil {
+		cfg.Logger.Info(
+			"forcing restart member",
+			zap.String("cluster-id", cid.String()),
+			zap.String("local-member-id", id.String()),
+			zap.Uint64("commit-index", st.Commit),
+		)
+	} else {
+		plog.Printf("forcing restart of member %s in cluster %s at commit index %d", id, cid, st.Commit)
+	}
+
+	cl := membership.NewCluster(cfg.Logger, "")
+	cl.SetID(id, cid)
+	s := raft.NewMemoryStorage()
+	if snapshot != nil {
+		s.ApplySnapshot(*snapshot)
+	}
+	s.SetHardState(st)
+	s.Append(ents)
+	c := &raft.Config{
+		ID:              uint64(id),
+		ElectionTick:    cfg.ElectionTicks,
+		HeartbeatTick:   1,
+		Storage:         s,
+		MaxSizePerMsg:   maxSizePerMsg,
+		MaxInflightMsgs: maxInflightMsgs,
+		CheckQuorum:     true,
+		PreVote:         cfg.PreVote,
+	}
+	if cfg.Logger != nil {
+		// called after capnslog setting in "init" function
+		if cfg.LoggerConfig != nil {
+			c.Logger, err = logutil.NewRaftLogger(cfg.LoggerConfig)
+			if err != nil {
+				log.Fatalf("cannot create raft logger %v", err)
+			}
+		} else if cfg.LoggerCore != nil && cfg.LoggerWriteSyncer != nil {
+			c.Logger = logutil.NewRaftLoggerFromZapCore(cfg.LoggerCore, cfg.LoggerWriteSyncer)
+		}
+	}
+
+	n := raft.RestartNode(c)
+	raftStatus = n.Status
+	return id, cl, n, s, w
+}
+
+// getIDs returns an ordered set of IDs included in the given snapshot and
+// the entries. The given snapshot/entries can contain two kinds of
+// ID-related entry:
+// - ConfChangeAddNode, in which case the contained ID will be added into the set.
+// - ConfChangeRemoveNode, in which case the contained ID will be removed from the set.
+func getIDs(lg *zap.Logger, snap *raftpb.Snapshot, ents []raftpb.Entry) []uint64 {
+	ids := make(map[uint64]bool)
+	if snap != nil {
+		for _, id := range snap.Metadata.ConfState.Voters {
+			ids[id] = true
+		}
+	}
+	for _, e := range ents {
+		if e.Type != raftpb.EntryConfChange {
+			continue
+		}
+		var cc raftpb.ConfChange
+		pbutil.MustUnmarshal(&cc, e.Data)
+		switch cc.Type {
+		case raftpb.ConfChangeAddNode:
+			ids[cc.NodeID] = true
+		case raftpb.ConfChangeRemoveNode:
+			delete(ids, cc.NodeID)
+		case raftpb.ConfChangeUpdateNode:
+			// do nothing
+		default:
+			if lg != nil {
+				lg.Panic("unknown ConfChange Type", zap.String("type", cc.Type.String()))
+			} else {
+				plog.Panicf("ConfChange Type should be either ConfChangeAddNode or ConfChangeRemoveNode!")
+			}
+		}
+	}
+	sids := make(types.Uint64Slice, 0, len(ids))
+	for id := range ids {
+		sids = append(sids, id)
+	}
+	sort.Sort(sids)
+	return []uint64(sids)
+}
+
+// createConfigChangeEnts creates a series of Raft entries (i.e.
+// EntryConfChange) to remove the set of given IDs from the cluster. The ID
+// `self` is _not_ removed, even if present in the set.
+// If `self` is not inside the given ids, it creates a Raft entry to add a
+// default member with the given `self`.
+func createConfigChangeEnts(lg *zap.Logger, ids []uint64, self uint64, term, index uint64) []raftpb.Entry {
+	found := false
+	for _, id := range ids {
+		if id == self {
+			found = true
+		}
+	}
+
+	var ents []raftpb.Entry
+	next := index + 1
+
+	// NB: always add self first, then remove other nodes. Raft will panic if the
+	// set of voters ever becomes empty.
+	if !found {
+		m := membership.Member{
+			ID:             types.ID(self),
+			RaftAttributes: membership.RaftAttributes{PeerURLs: []string{"http://localhost:2380"}},
+		}
+		ctx, err := json.Marshal(m)
+		if err != nil {
+			if lg != nil {
+				lg.Panic("failed to marshal member", zap.Error(err))
+			} else {
+				plog.Panicf("marshal member should never fail: %v", err)
+			}
+		}
+		cc := &raftpb.ConfChange{
+			Type:    raftpb.ConfChangeAddNode,
+			NodeID:  self,
+			Context: ctx,
+		}
+		e := raftpb.Entry{
+			Type:  raftpb.EntryConfChange,
+			Data:  pbutil.MustMarshal(cc),
+			Term:  term,
+			Index: next,
+		}
+		ents = append(ents, e)
+		next++
+	}
+
+	for _, id := range ids {
+		if id == self {
+			continue
+		}
+		cc := &raftpb.ConfChange{
+			Type:   raftpb.ConfChangeRemoveNode,
+			NodeID: id,
+		}
+		e := raftpb.Entry{
+			Type:  raftpb.EntryConfChange,
+			Data:  pbutil.MustMarshal(cc),
+			Term:  term,
+			Index: next,
+		}
+		ents = append(ents, e)
+		next++
+	}
+
+	return ents
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/server.go b/vendor/go.etcd.io/etcd/etcdserver/server.go
new file mode 100644
index 0000000..78daa0e
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/server.go
@@ -0,0 +1,2656 @@
+// Copyright 2015 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import (
+	"context"
+	"encoding/json"
+	"expvar"
+	"fmt"
+	"math"
+	"math/rand"
+	"net/http"
+	"os"
+	"path"
+	"regexp"
+	"sync"
+	"sync/atomic"
+	"time"
+
+	"go.etcd.io/etcd/auth"
+	"go.etcd.io/etcd/etcdserver/api"
+	"go.etcd.io/etcd/etcdserver/api/membership"
+	"go.etcd.io/etcd/etcdserver/api/rafthttp"
+	"go.etcd.io/etcd/etcdserver/api/snap"
+	"go.etcd.io/etcd/etcdserver/api/v2discovery"
+	"go.etcd.io/etcd/etcdserver/api/v2http/httptypes"
+	stats "go.etcd.io/etcd/etcdserver/api/v2stats"
+	"go.etcd.io/etcd/etcdserver/api/v2store"
+	"go.etcd.io/etcd/etcdserver/api/v3alarm"
+	"go.etcd.io/etcd/etcdserver/api/v3compactor"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/lease"
+	"go.etcd.io/etcd/lease/leasehttp"
+	"go.etcd.io/etcd/mvcc"
+	"go.etcd.io/etcd/mvcc/backend"
+	"go.etcd.io/etcd/pkg/fileutil"
+	"go.etcd.io/etcd/pkg/idutil"
+	"go.etcd.io/etcd/pkg/pbutil"
+	"go.etcd.io/etcd/pkg/runtime"
+	"go.etcd.io/etcd/pkg/schedule"
+	"go.etcd.io/etcd/pkg/types"
+	"go.etcd.io/etcd/pkg/wait"
+	"go.etcd.io/etcd/raft"
+	"go.etcd.io/etcd/raft/raftpb"
+	"go.etcd.io/etcd/version"
+	"go.etcd.io/etcd/wal"
+
+	"github.com/coreos/go-semver/semver"
+	"github.com/coreos/pkg/capnslog"
+	humanize "github.com/dustin/go-humanize"
+	"github.com/prometheus/client_golang/prometheus"
+	"go.uber.org/zap"
+)
+
+const (
+	DefaultSnapshotCount = 100000
+
+	// DefaultSnapshotCatchUpEntries is the number of entries for a slow follower
+	// to catch-up after compacting the raft storage entries.
+	// We expect the follower has a millisecond level latency with the leader.
+	// The max throughput is around 10K. Keep a 5K entries is enough for helping
+	// follower to catch up.
+	DefaultSnapshotCatchUpEntries uint64 = 5000
+
+	StoreClusterPrefix = "/0"
+	StoreKeysPrefix    = "/1"
+
+	// HealthInterval is the minimum time the cluster should be healthy
+	// before accepting add member requests.
+	HealthInterval = 5 * time.Second
+
+	purgeFileInterval = 30 * time.Second
+	// monitorVersionInterval should be smaller than the timeout
+	// on the connection. Or we will not be able to reuse the connection
+	// (since it will timeout).
+	monitorVersionInterval = rafthttp.ConnWriteTimeout - time.Second
+
+	// max number of in-flight snapshot messages etcdserver allows to have
+	// This number is more than enough for most clusters with 5 machines.
+	maxInFlightMsgSnap = 16
+
+	releaseDelayAfterSnapshot = 30 * time.Second
+
+	// maxPendingRevokes is the maximum number of outstanding expired lease revocations.
+	maxPendingRevokes = 16
+
+	recommendedMaxRequestBytes = 10 * 1024 * 1024
+
+	readyPercent = 0.9
+)
+
+var (
+	plog = capnslog.NewPackageLogger("go.etcd.io/etcd", "etcdserver")
+
+	storeMemberAttributeRegexp = regexp.MustCompile(path.Join(membership.StoreMembersPrefix, "[[:xdigit:]]{1,16}", "attributes"))
+)
+
+func init() {
+	rand.Seed(time.Now().UnixNano())
+
+	expvar.Publish(
+		"file_descriptor_limit",
+		expvar.Func(
+			func() interface{} {
+				n, _ := runtime.FDLimit()
+				return n
+			},
+		),
+	)
+}
+
+type Response struct {
+	Term    uint64
+	Index   uint64
+	Event   *v2store.Event
+	Watcher v2store.Watcher
+	Err     error
+}
+
+type ServerV2 interface {
+	Server
+	Leader() types.ID
+
+	// Do takes a V2 request and attempts to fulfill it, returning a Response.
+	Do(ctx context.Context, r pb.Request) (Response, error)
+	stats.Stats
+	ClientCertAuthEnabled() bool
+}
+
+type ServerV3 interface {
+	Server
+	RaftStatusGetter
+}
+
+func (s *EtcdServer) ClientCertAuthEnabled() bool { return s.Cfg.ClientCertAuthEnabled }
+
+type Server interface {
+	// AddMember attempts to add a member into the cluster. It will return
+	// ErrIDRemoved if member ID is removed from the cluster, or return
+	// ErrIDExists if member ID exists in the cluster.
+	AddMember(ctx context.Context, memb membership.Member) ([]*membership.Member, error)
+	// RemoveMember attempts to remove a member from the cluster. It will
+	// return ErrIDRemoved if member ID is removed from the cluster, or return
+	// ErrIDNotFound if member ID is not in the cluster.
+	RemoveMember(ctx context.Context, id uint64) ([]*membership.Member, error)
+	// UpdateMember attempts to update an existing member in the cluster. It will
+	// return ErrIDNotFound if the member ID does not exist.
+	UpdateMember(ctx context.Context, updateMemb membership.Member) ([]*membership.Member, error)
+	// PromoteMember attempts to promote a non-voting node to a voting node. It will
+	// return ErrIDNotFound if the member ID does not exist.
+	// return ErrLearnerNotReady if the member are not ready.
+	// return ErrMemberNotLearner if the member is not a learner.
+	PromoteMember(ctx context.Context, id uint64) ([]*membership.Member, error)
+
+	// ClusterVersion is the cluster-wide minimum major.minor version.
+	// Cluster version is set to the min version that an etcd member is
+	// compatible with when first bootstrap.
+	//
+	// ClusterVersion is nil until the cluster is bootstrapped (has a quorum).
+	//
+	// During a rolling upgrades, the ClusterVersion will be updated
+	// automatically after a sync. (5 second by default)
+	//
+	// The API/raft component can utilize ClusterVersion to determine if
+	// it can accept a client request or a raft RPC.
+	// NOTE: ClusterVersion might be nil when etcd 2.1 works with etcd 2.0 and
+	// the leader is etcd 2.0. etcd 2.0 leader will not update clusterVersion since
+	// this feature is introduced post 2.0.
+	ClusterVersion() *semver.Version
+	Cluster() api.Cluster
+	Alarms() []*pb.AlarmMember
+}
+
+// EtcdServer is the production implementation of the Server interface
+type EtcdServer struct {
+	// inflightSnapshots holds count the number of snapshots currently inflight.
+	inflightSnapshots int64  // must use atomic operations to access; keep 64-bit aligned.
+	appliedIndex      uint64 // must use atomic operations to access; keep 64-bit aligned.
+	committedIndex    uint64 // must use atomic operations to access; keep 64-bit aligned.
+	term              uint64 // must use atomic operations to access; keep 64-bit aligned.
+	lead              uint64 // must use atomic operations to access; keep 64-bit aligned.
+
+	// consistIndex used to hold the offset of current executing entry
+	// It is initialized to 0 before executing any entry.
+	consistIndex consistentIndex // must use atomic operations to access; keep 64-bit aligned.
+	r            raftNode        // uses 64-bit atomics; keep 64-bit aligned.
+
+	readych chan struct{}
+	Cfg     ServerConfig
+
+	lgMu *sync.RWMutex
+	lg   *zap.Logger
+
+	w wait.Wait
+
+	readMu sync.RWMutex
+	// read routine notifies etcd server that it waits for reading by sending an empty struct to
+	// readwaitC
+	readwaitc chan struct{}
+	// readNotifier is used to notify the read routine that it can process the request
+	// when there is no error
+	readNotifier *notifier
+
+	// stop signals the run goroutine should shutdown.
+	stop chan struct{}
+	// stopping is closed by run goroutine on shutdown.
+	stopping chan struct{}
+	// done is closed when all goroutines from start() complete.
+	done chan struct{}
+	// leaderChanged is used to notify the linearizable read loop to drop the old read requests.
+	leaderChanged   chan struct{}
+	leaderChangedMu sync.RWMutex
+
+	errorc     chan error
+	id         types.ID
+	attributes membership.Attributes
+
+	cluster *membership.RaftCluster
+
+	v2store     v2store.Store
+	snapshotter *snap.Snapshotter
+
+	applyV2 ApplierV2
+
+	// applyV3 is the applier with auth and quotas
+	applyV3 applierV3
+	// applyV3Base is the core applier without auth or quotas
+	applyV3Base applierV3
+	applyWait   wait.WaitTime
+
+	kv         mvcc.ConsistentWatchableKV
+	lessor     lease.Lessor
+	bemu       sync.Mutex
+	be         backend.Backend
+	authStore  auth.AuthStore
+	alarmStore *v3alarm.AlarmStore
+
+	stats  *stats.ServerStats
+	lstats *stats.LeaderStats
+
+	SyncTicker *time.Ticker
+	// compactor is used to auto-compact the KV.
+	compactor v3compactor.Compactor
+
+	// peerRt used to send requests (version, lease) to peers.
+	peerRt   http.RoundTripper
+	reqIDGen *idutil.Generator
+
+	// forceVersionC is used to force the version monitor loop
+	// to detect the cluster version immediately.
+	forceVersionC chan struct{}
+
+	// wgMu blocks concurrent waitgroup mutation while server stopping
+	wgMu sync.RWMutex
+	// wg is used to wait for the go routines that depends on the server state
+	// to exit when stopping the server.
+	wg sync.WaitGroup
+
+	// ctx is used for etcd-initiated requests that may need to be canceled
+	// on etcd server shutdown.
+	ctx    context.Context
+	cancel context.CancelFunc
+
+	leadTimeMu      sync.RWMutex
+	leadElectedTime time.Time
+
+	*AccessController
+}
+
+// NewServer creates a new EtcdServer from the supplied configuration. The
+// configuration is considered static for the lifetime of the EtcdServer.
+func NewServer(cfg ServerConfig) (srv *EtcdServer, err error) {
+	st := v2store.New(StoreClusterPrefix, StoreKeysPrefix)
+
+	var (
+		w  *wal.WAL
+		n  raft.Node
+		s  *raft.MemoryStorage
+		id types.ID
+		cl *membership.RaftCluster
+	)
+
+	if cfg.MaxRequestBytes > recommendedMaxRequestBytes {
+		if cfg.Logger != nil {
+			cfg.Logger.Warn(
+				"exceeded recommended request limit",
+				zap.Uint("max-request-bytes", cfg.MaxRequestBytes),
+				zap.String("max-request-size", humanize.Bytes(uint64(cfg.MaxRequestBytes))),
+				zap.Int("recommended-request-bytes", recommendedMaxRequestBytes),
+				zap.String("recommended-request-size", humanize.Bytes(uint64(recommendedMaxRequestBytes))),
+			)
+		} else {
+			plog.Warningf("MaxRequestBytes %v exceeds maximum recommended size %v", cfg.MaxRequestBytes, recommendedMaxRequestBytes)
+		}
+	}
+
+	if terr := fileutil.TouchDirAll(cfg.DataDir); terr != nil {
+		return nil, fmt.Errorf("cannot access data directory: %v", terr)
+	}
+
+	haveWAL := wal.Exist(cfg.WALDir())
+
+	if err = fileutil.TouchDirAll(cfg.SnapDir()); err != nil {
+		if cfg.Logger != nil {
+			cfg.Logger.Fatal(
+				"failed to create snapshot directory",
+				zap.String("path", cfg.SnapDir()),
+				zap.Error(err),
+			)
+		} else {
+			plog.Fatalf("create snapshot directory error: %v", err)
+		}
+	}
+	ss := snap.New(cfg.Logger, cfg.SnapDir())
+
+	bepath := cfg.backendPath()
+	beExist := fileutil.Exist(bepath)
+	be := openBackend(cfg)
+
+	defer func() {
+		if err != nil {
+			be.Close()
+		}
+	}()
+
+	prt, err := rafthttp.NewRoundTripper(cfg.PeerTLSInfo, cfg.peerDialTimeout())
+	if err != nil {
+		return nil, err
+	}
+	var (
+		remotes  []*membership.Member
+		snapshot *raftpb.Snapshot
+	)
+
+	switch {
+	case !haveWAL && !cfg.NewCluster:
+		if err = cfg.VerifyJoinExisting(); err != nil {
+			return nil, err
+		}
+		cl, err = membership.NewClusterFromURLsMap(cfg.Logger, cfg.InitialClusterToken, cfg.InitialPeerURLsMap)
+		if err != nil {
+			return nil, err
+		}
+		existingCluster, gerr := GetClusterFromRemotePeers(cfg.Logger, getRemotePeerURLs(cl, cfg.Name), prt)
+		if gerr != nil {
+			return nil, fmt.Errorf("cannot fetch cluster info from peer urls: %v", gerr)
+		}
+		if err = membership.ValidateClusterAndAssignIDs(cfg.Logger, cl, existingCluster); err != nil {
+			return nil, fmt.Errorf("error validating peerURLs %s: %v", existingCluster, err)
+		}
+		if !isCompatibleWithCluster(cfg.Logger, cl, cl.MemberByName(cfg.Name).ID, prt) {
+			return nil, fmt.Errorf("incompatible with current running cluster")
+		}
+
+		remotes = existingCluster.Members()
+		cl.SetID(types.ID(0), existingCluster.ID())
+		cl.SetStore(st)
+		cl.SetBackend(be)
+		id, n, s, w = startNode(cfg, cl, nil)
+		cl.SetID(id, existingCluster.ID())
+
+	case !haveWAL && cfg.NewCluster:
+		if err = cfg.VerifyBootstrap(); err != nil {
+			return nil, err
+		}
+		cl, err = membership.NewClusterFromURLsMap(cfg.Logger, cfg.InitialClusterToken, cfg.InitialPeerURLsMap)
+		if err != nil {
+			return nil, err
+		}
+		m := cl.MemberByName(cfg.Name)
+		if isMemberBootstrapped(cfg.Logger, cl, cfg.Name, prt, cfg.bootstrapTimeout()) {
+			return nil, fmt.Errorf("member %s has already been bootstrapped", m.ID)
+		}
+		if cfg.ShouldDiscover() {
+			var str string
+			str, err = v2discovery.JoinCluster(cfg.Logger, cfg.DiscoveryURL, cfg.DiscoveryProxy, m.ID, cfg.InitialPeerURLsMap.String())
+			if err != nil {
+				return nil, &DiscoveryError{Op: "join", Err: err}
+			}
+			var urlsmap types.URLsMap
+			urlsmap, err = types.NewURLsMap(str)
+			if err != nil {
+				return nil, err
+			}
+			if checkDuplicateURL(urlsmap) {
+				return nil, fmt.Errorf("discovery cluster %s has duplicate url", urlsmap)
+			}
+			if cl, err = membership.NewClusterFromURLsMap(cfg.Logger, cfg.InitialClusterToken, urlsmap); err != nil {
+				return nil, err
+			}
+		}
+		cl.SetStore(st)
+		cl.SetBackend(be)
+		id, n, s, w = startNode(cfg, cl, cl.MemberIDs())
+		cl.SetID(id, cl.ID())
+
+	case haveWAL:
+		if err = fileutil.IsDirWriteable(cfg.MemberDir()); err != nil {
+			return nil, fmt.Errorf("cannot write to member directory: %v", err)
+		}
+
+		if err = fileutil.IsDirWriteable(cfg.WALDir()); err != nil {
+			return nil, fmt.Errorf("cannot write to WAL directory: %v", err)
+		}
+
+		if cfg.ShouldDiscover() {
+			if cfg.Logger != nil {
+				cfg.Logger.Warn(
+					"discovery token is ignored since cluster already initialized; valid logs are found",
+					zap.String("wal-dir", cfg.WALDir()),
+				)
+			} else {
+				plog.Warningf("discovery token ignored since a cluster has already been initialized. Valid log found at %q", cfg.WALDir())
+			}
+		}
+		snapshot, err = ss.Load()
+		if err != nil && err != snap.ErrNoSnapshot {
+			return nil, err
+		}
+		if snapshot != nil {
+			if err = st.Recovery(snapshot.Data); err != nil {
+				if cfg.Logger != nil {
+					cfg.Logger.Panic("failed to recover from snapshot")
+				} else {
+					plog.Panicf("recovered store from snapshot error: %v", err)
+				}
+			}
+
+			if cfg.Logger != nil {
+				cfg.Logger.Info(
+					"recovered v2 store from snapshot",
+					zap.Uint64("snapshot-index", snapshot.Metadata.Index),
+					zap.String("snapshot-size", humanize.Bytes(uint64(snapshot.Size()))),
+				)
+			} else {
+				plog.Infof("recovered store from snapshot at index %d", snapshot.Metadata.Index)
+			}
+
+			if be, err = recoverSnapshotBackend(cfg, be, *snapshot); err != nil {
+				if cfg.Logger != nil {
+					cfg.Logger.Panic("failed to recover v3 backend from snapshot", zap.Error(err))
+				} else {
+					plog.Panicf("recovering backend from snapshot error: %v", err)
+				}
+			}
+			if cfg.Logger != nil {
+				s1, s2 := be.Size(), be.SizeInUse()
+				cfg.Logger.Info(
+					"recovered v3 backend from snapshot",
+					zap.Int64("backend-size-bytes", s1),
+					zap.String("backend-size", humanize.Bytes(uint64(s1))),
+					zap.Int64("backend-size-in-use-bytes", s2),
+					zap.String("backend-size-in-use", humanize.Bytes(uint64(s2))),
+				)
+			}
+		}
+
+		if !cfg.ForceNewCluster {
+			id, cl, n, s, w = restartNode(cfg, snapshot)
+		} else {
+			id, cl, n, s, w = restartAsStandaloneNode(cfg, snapshot)
+		}
+
+		cl.SetStore(st)
+		cl.SetBackend(be)
+		cl.Recover(api.UpdateCapability)
+		if cl.Version() != nil && !cl.Version().LessThan(semver.Version{Major: 3}) && !beExist {
+			os.RemoveAll(bepath)
+			return nil, fmt.Errorf("database file (%v) of the backend is missing", bepath)
+		}
+
+	default:
+		return nil, fmt.Errorf("unsupported bootstrap config")
+	}
+
+	if terr := fileutil.TouchDirAll(cfg.MemberDir()); terr != nil {
+		return nil, fmt.Errorf("cannot access member directory: %v", terr)
+	}
+
+	sstats := stats.NewServerStats(cfg.Name, id.String())
+	lstats := stats.NewLeaderStats(id.String())
+
+	heartbeat := time.Duration(cfg.TickMs) * time.Millisecond
+	srv = &EtcdServer{
+		readych:     make(chan struct{}),
+		Cfg:         cfg,
+		lgMu:        new(sync.RWMutex),
+		lg:          cfg.Logger,
+		errorc:      make(chan error, 1),
+		v2store:     st,
+		snapshotter: ss,
+		r: *newRaftNode(
+			raftNodeConfig{
+				lg:          cfg.Logger,
+				isIDRemoved: func(id uint64) bool { return cl.IsIDRemoved(types.ID(id)) },
+				Node:        n,
+				heartbeat:   heartbeat,
+				raftStorage: s,
+				storage:     NewStorage(w, ss),
+			},
+		),
+		id:               id,
+		attributes:       membership.Attributes{Name: cfg.Name, ClientURLs: cfg.ClientURLs.StringSlice()},
+		cluster:          cl,
+		stats:            sstats,
+		lstats:           lstats,
+		SyncTicker:       time.NewTicker(500 * time.Millisecond),
+		peerRt:           prt,
+		reqIDGen:         idutil.NewGenerator(uint16(id), time.Now()),
+		forceVersionC:    make(chan struct{}),
+		AccessController: &AccessController{CORS: cfg.CORS, HostWhitelist: cfg.HostWhitelist},
+	}
+	serverID.With(prometheus.Labels{"server_id": id.String()}).Set(1)
+
+	srv.applyV2 = &applierV2store{store: srv.v2store, cluster: srv.cluster}
+
+	srv.be = be
+	minTTL := time.Duration((3*cfg.ElectionTicks)/2) * heartbeat
+
+	// always recover lessor before kv. When we recover the mvcc.KV it will reattach keys to its leases.
+	// If we recover mvcc.KV first, it will attach the keys to the wrong lessor before it recovers.
+	srv.lessor = lease.NewLessor(
+		srv.getLogger(),
+		srv.be,
+		lease.LessorConfig{
+			MinLeaseTTL:                int64(math.Ceil(minTTL.Seconds())),
+			CheckpointInterval:         cfg.LeaseCheckpointInterval,
+			ExpiredLeasesRetryInterval: srv.Cfg.ReqTimeout(),
+		})
+	srv.kv = mvcc.New(srv.getLogger(), srv.be, srv.lessor, &srv.consistIndex, mvcc.StoreConfig{CompactionBatchLimit: cfg.CompactionBatchLimit})
+	if beExist {
+		kvindex := srv.kv.ConsistentIndex()
+		// TODO: remove kvindex != 0 checking when we do not expect users to upgrade
+		// etcd from pre-3.0 release.
+		if snapshot != nil && kvindex < snapshot.Metadata.Index {
+			if kvindex != 0 {
+				return nil, fmt.Errorf("database file (%v index %d) does not match with snapshot (index %d)", bepath, kvindex, snapshot.Metadata.Index)
+			}
+			if cfg.Logger != nil {
+				cfg.Logger.Warn(
+					"consistent index was never saved",
+					zap.Uint64("snapshot-index", snapshot.Metadata.Index),
+				)
+			} else {
+				plog.Warningf("consistent index never saved (snapshot index=%d)", snapshot.Metadata.Index)
+			}
+		}
+	}
+	newSrv := srv // since srv == nil in defer if srv is returned as nil
+	defer func() {
+		// closing backend without first closing kv can cause
+		// resumed compactions to fail with closed tx errors
+		if err != nil {
+			newSrv.kv.Close()
+		}
+	}()
+
+	srv.consistIndex.setConsistentIndex(srv.kv.ConsistentIndex())
+	tp, err := auth.NewTokenProvider(cfg.Logger, cfg.AuthToken,
+		func(index uint64) <-chan struct{} {
+			return srv.applyWait.Wait(index)
+		},
+	)
+	if err != nil {
+		if cfg.Logger != nil {
+			cfg.Logger.Warn("failed to create token provider", zap.Error(err))
+		} else {
+			plog.Errorf("failed to create token provider: %s", err)
+		}
+		return nil, err
+	}
+	srv.authStore = auth.NewAuthStore(srv.getLogger(), srv.be, tp, int(cfg.BcryptCost))
+	if num := cfg.AutoCompactionRetention; num != 0 {
+		srv.compactor, err = v3compactor.New(cfg.Logger, cfg.AutoCompactionMode, num, srv.kv, srv)
+		if err != nil {
+			return nil, err
+		}
+		srv.compactor.Run()
+	}
+
+	srv.applyV3Base = srv.newApplierV3Backend()
+	if err = srv.restoreAlarms(); err != nil {
+		return nil, err
+	}
+
+	if srv.Cfg.EnableLeaseCheckpoint {
+		// setting checkpointer enables lease checkpoint feature.
+		srv.lessor.SetCheckpointer(func(ctx context.Context, cp *pb.LeaseCheckpointRequest) {
+			srv.raftRequestOnce(ctx, pb.InternalRaftRequest{LeaseCheckpoint: cp})
+		})
+	}
+
+	// TODO: move transport initialization near the definition of remote
+	tr := &rafthttp.Transport{
+		Logger:      cfg.Logger,
+		TLSInfo:     cfg.PeerTLSInfo,
+		DialTimeout: cfg.peerDialTimeout(),
+		ID:          id,
+		URLs:        cfg.PeerURLs,
+		ClusterID:   cl.ID(),
+		Raft:        srv,
+		Snapshotter: ss,
+		ServerStats: sstats,
+		LeaderStats: lstats,
+		ErrorC:      srv.errorc,
+	}
+	if err = tr.Start(); err != nil {
+		return nil, err
+	}
+	// add all remotes into transport
+	for _, m := range remotes {
+		if m.ID != id {
+			tr.AddRemote(m.ID, m.PeerURLs)
+		}
+	}
+	for _, m := range cl.Members() {
+		if m.ID != id {
+			tr.AddPeer(m.ID, m.PeerURLs)
+		}
+	}
+	srv.r.transport = tr
+
+	return srv, nil
+}
+
+func (s *EtcdServer) getLogger() *zap.Logger {
+	s.lgMu.RLock()
+	l := s.lg
+	s.lgMu.RUnlock()
+	return l
+}
+
+func tickToDur(ticks int, tickMs uint) string {
+	return fmt.Sprintf("%v", time.Duration(ticks)*time.Duration(tickMs)*time.Millisecond)
+}
+
+func (s *EtcdServer) adjustTicks() {
+	lg := s.getLogger()
+	clusterN := len(s.cluster.Members())
+
+	// single-node fresh start, or single-node recovers from snapshot
+	if clusterN == 1 {
+		ticks := s.Cfg.ElectionTicks - 1
+		if lg != nil {
+			lg.Info(
+				"started as single-node; fast-forwarding election ticks",
+				zap.String("local-member-id", s.ID().String()),
+				zap.Int("forward-ticks", ticks),
+				zap.String("forward-duration", tickToDur(ticks, s.Cfg.TickMs)),
+				zap.Int("election-ticks", s.Cfg.ElectionTicks),
+				zap.String("election-timeout", tickToDur(s.Cfg.ElectionTicks, s.Cfg.TickMs)),
+			)
+		} else {
+			plog.Infof("%s as single-node; fast-forwarding %d ticks (election ticks %d)", s.ID(), ticks, s.Cfg.ElectionTicks)
+		}
+		s.r.advanceTicks(ticks)
+		return
+	}
+
+	if !s.Cfg.InitialElectionTickAdvance {
+		if lg != nil {
+			lg.Info("skipping initial election tick advance", zap.Int("election-ticks", s.Cfg.ElectionTicks))
+		}
+		return
+	}
+	if lg != nil {
+		lg.Info("starting initial election tick advance", zap.Int("election-ticks", s.Cfg.ElectionTicks))
+	}
+
+	// retry up to "rafthttp.ConnReadTimeout", which is 5-sec
+	// until peer connection reports; otherwise:
+	// 1. all connections failed, or
+	// 2. no active peers, or
+	// 3. restarted single-node with no snapshot
+	// then, do nothing, because advancing ticks would have no effect
+	waitTime := rafthttp.ConnReadTimeout
+	itv := 50 * time.Millisecond
+	for i := int64(0); i < int64(waitTime/itv); i++ {
+		select {
+		case <-time.After(itv):
+		case <-s.stopping:
+			return
+		}
+
+		peerN := s.r.transport.ActivePeers()
+		if peerN > 1 {
+			// multi-node received peer connection reports
+			// adjust ticks, in case slow leader message receive
+			ticks := s.Cfg.ElectionTicks - 2
+
+			if lg != nil {
+				lg.Info(
+					"initialized peer connections; fast-forwarding election ticks",
+					zap.String("local-member-id", s.ID().String()),
+					zap.Int("forward-ticks", ticks),
+					zap.String("forward-duration", tickToDur(ticks, s.Cfg.TickMs)),
+					zap.Int("election-ticks", s.Cfg.ElectionTicks),
+					zap.String("election-timeout", tickToDur(s.Cfg.ElectionTicks, s.Cfg.TickMs)),
+					zap.Int("active-remote-members", peerN),
+				)
+			} else {
+				plog.Infof("%s initialized peer connection; fast-forwarding %d ticks (election ticks %d) with %d active peer(s)", s.ID(), ticks, s.Cfg.ElectionTicks, peerN)
+			}
+
+			s.r.advanceTicks(ticks)
+			return
+		}
+	}
+}
+
+// Start performs any initialization of the Server necessary for it to
+// begin serving requests. It must be called before Do or Process.
+// Start must be non-blocking; any long-running server functionality
+// should be implemented in goroutines.
+func (s *EtcdServer) Start() {
+	s.start()
+	s.goAttach(func() { s.adjustTicks() })
+	s.goAttach(func() { s.publish(s.Cfg.ReqTimeout()) })
+	s.goAttach(s.purgeFile)
+	s.goAttach(func() { monitorFileDescriptor(s.getLogger(), s.stopping) })
+	s.goAttach(s.monitorVersions)
+	s.goAttach(s.linearizableReadLoop)
+	s.goAttach(s.monitorKVHash)
+}
+
+// start prepares and starts server in a new goroutine. It is no longer safe to
+// modify a server's fields after it has been sent to Start.
+// This function is just used for testing.
+func (s *EtcdServer) start() {
+	lg := s.getLogger()
+
+	if s.Cfg.SnapshotCount == 0 {
+		if lg != nil {
+			lg.Info(
+				"updating snapshot-count to default",
+				zap.Uint64("given-snapshot-count", s.Cfg.SnapshotCount),
+				zap.Uint64("updated-snapshot-count", DefaultSnapshotCount),
+			)
+		} else {
+			plog.Infof("set snapshot count to default %d", DefaultSnapshotCount)
+		}
+		s.Cfg.SnapshotCount = DefaultSnapshotCount
+	}
+	if s.Cfg.SnapshotCatchUpEntries == 0 {
+		if lg != nil {
+			lg.Info(
+				"updating snapshot catch-up entries to default",
+				zap.Uint64("given-snapshot-catchup-entries", s.Cfg.SnapshotCatchUpEntries),
+				zap.Uint64("updated-snapshot-catchup-entries", DefaultSnapshotCatchUpEntries),
+			)
+		}
+		s.Cfg.SnapshotCatchUpEntries = DefaultSnapshotCatchUpEntries
+	}
+
+	s.w = wait.New()
+	s.applyWait = wait.NewTimeList()
+	s.done = make(chan struct{})
+	s.stop = make(chan struct{})
+	s.stopping = make(chan struct{})
+	s.ctx, s.cancel = context.WithCancel(context.Background())
+	s.readwaitc = make(chan struct{}, 1)
+	s.readNotifier = newNotifier()
+	s.leaderChanged = make(chan struct{})
+	if s.ClusterVersion() != nil {
+		if lg != nil {
+			lg.Info(
+				"starting etcd server",
+				zap.String("local-member-id", s.ID().String()),
+				zap.String("local-server-version", version.Version),
+				zap.String("cluster-id", s.Cluster().ID().String()),
+				zap.String("cluster-version", version.Cluster(s.ClusterVersion().String())),
+			)
+		} else {
+			plog.Infof("starting server... [version: %v, cluster version: %v]", version.Version, version.Cluster(s.ClusterVersion().String()))
+		}
+		membership.ClusterVersionMetrics.With(prometheus.Labels{"cluster_version": s.ClusterVersion().String()}).Set(1)
+	} else {
+		if lg != nil {
+			lg.Info(
+				"starting etcd server",
+				zap.String("local-member-id", s.ID().String()),
+				zap.String("local-server-version", version.Version),
+				zap.String("cluster-version", "to_be_decided"),
+			)
+		} else {
+			plog.Infof("starting server... [version: %v, cluster version: to_be_decided]", version.Version)
+		}
+	}
+
+	// TODO: if this is an empty log, writes all peer infos
+	// into the first entry
+	go s.run()
+}
+
+func (s *EtcdServer) purgeFile() {
+	var dberrc, serrc, werrc <-chan error
+	if s.Cfg.MaxSnapFiles > 0 {
+		dberrc = fileutil.PurgeFile(s.getLogger(), s.Cfg.SnapDir(), "snap.db", s.Cfg.MaxSnapFiles, purgeFileInterval, s.done)
+		serrc = fileutil.PurgeFile(s.getLogger(), s.Cfg.SnapDir(), "snap", s.Cfg.MaxSnapFiles, purgeFileInterval, s.done)
+	}
+	if s.Cfg.MaxWALFiles > 0 {
+		werrc = fileutil.PurgeFile(s.getLogger(), s.Cfg.WALDir(), "wal", s.Cfg.MaxWALFiles, purgeFileInterval, s.done)
+	}
+
+	lg := s.getLogger()
+	select {
+	case e := <-dberrc:
+		if lg != nil {
+			lg.Fatal("failed to purge snap db file", zap.Error(e))
+		} else {
+			plog.Fatalf("failed to purge snap db file %v", e)
+		}
+	case e := <-serrc:
+		if lg != nil {
+			lg.Fatal("failed to purge snap file", zap.Error(e))
+		} else {
+			plog.Fatalf("failed to purge snap file %v", e)
+		}
+	case e := <-werrc:
+		if lg != nil {
+			lg.Fatal("failed to purge wal file", zap.Error(e))
+		} else {
+			plog.Fatalf("failed to purge wal file %v", e)
+		}
+	case <-s.stopping:
+		return
+	}
+}
+
+func (s *EtcdServer) Cluster() api.Cluster { return s.cluster }
+
+func (s *EtcdServer) ApplyWait() <-chan struct{} { return s.applyWait.Wait(s.getCommittedIndex()) }
+
+type ServerPeer interface {
+	ServerV2
+	RaftHandler() http.Handler
+	LeaseHandler() http.Handler
+}
+
+func (s *EtcdServer) LeaseHandler() http.Handler {
+	if s.lessor == nil {
+		return nil
+	}
+	return leasehttp.NewHandler(s.lessor, s.ApplyWait)
+}
+
+func (s *EtcdServer) RaftHandler() http.Handler { return s.r.transport.Handler() }
+
+// Process takes a raft message and applies it to the server's raft state
+// machine, respecting any timeout of the given context.
+func (s *EtcdServer) Process(ctx context.Context, m raftpb.Message) error {
+	if s.cluster.IsIDRemoved(types.ID(m.From)) {
+		if lg := s.getLogger(); lg != nil {
+			lg.Warn(
+				"rejected Raft message from removed member",
+				zap.String("local-member-id", s.ID().String()),
+				zap.String("removed-member-id", types.ID(m.From).String()),
+			)
+		} else {
+			plog.Warningf("reject message from removed member %s", types.ID(m.From).String())
+		}
+		return httptypes.NewHTTPError(http.StatusForbidden, "cannot process message from removed member")
+	}
+	if m.Type == raftpb.MsgApp {
+		s.stats.RecvAppendReq(types.ID(m.From).String(), m.Size())
+	}
+	return s.r.Step(ctx, m)
+}
+
+func (s *EtcdServer) IsIDRemoved(id uint64) bool { return s.cluster.IsIDRemoved(types.ID(id)) }
+
+func (s *EtcdServer) ReportUnreachable(id uint64) { s.r.ReportUnreachable(id) }
+
+// ReportSnapshot reports snapshot sent status to the raft state machine,
+// and clears the used snapshot from the snapshot store.
+func (s *EtcdServer) ReportSnapshot(id uint64, status raft.SnapshotStatus) {
+	s.r.ReportSnapshot(id, status)
+}
+
+type etcdProgress struct {
+	confState raftpb.ConfState
+	snapi     uint64
+	appliedt  uint64
+	appliedi  uint64
+}
+
+// raftReadyHandler contains a set of EtcdServer operations to be called by raftNode,
+// and helps decouple state machine logic from Raft algorithms.
+// TODO: add a state machine interface to apply the commit entries and do snapshot/recover
+type raftReadyHandler struct {
+	getLead              func() (lead uint64)
+	updateLead           func(lead uint64)
+	updateLeadership     func(newLeader bool)
+	updateCommittedIndex func(uint64)
+}
+
+func (s *EtcdServer) run() {
+	lg := s.getLogger()
+
+	sn, err := s.r.raftStorage.Snapshot()
+	if err != nil {
+		if lg != nil {
+			lg.Panic("failed to get snapshot from Raft storage", zap.Error(err))
+		} else {
+			plog.Panicf("get snapshot from raft storage error: %v", err)
+		}
+	}
+
+	// asynchronously accept apply packets, dispatch progress in-order
+	sched := schedule.NewFIFOScheduler()
+
+	var (
+		smu   sync.RWMutex
+		syncC <-chan time.Time
+	)
+	setSyncC := func(ch <-chan time.Time) {
+		smu.Lock()
+		syncC = ch
+		smu.Unlock()
+	}
+	getSyncC := func() (ch <-chan time.Time) {
+		smu.RLock()
+		ch = syncC
+		smu.RUnlock()
+		return
+	}
+	rh := &raftReadyHandler{
+		getLead:    func() (lead uint64) { return s.getLead() },
+		updateLead: func(lead uint64) { s.setLead(lead) },
+		updateLeadership: func(newLeader bool) {
+			if !s.isLeader() {
+				if s.lessor != nil {
+					s.lessor.Demote()
+				}
+				if s.compactor != nil {
+					s.compactor.Pause()
+				}
+				setSyncC(nil)
+			} else {
+				if newLeader {
+					t := time.Now()
+					s.leadTimeMu.Lock()
+					s.leadElectedTime = t
+					s.leadTimeMu.Unlock()
+				}
+				setSyncC(s.SyncTicker.C)
+				if s.compactor != nil {
+					s.compactor.Resume()
+				}
+			}
+			if newLeader {
+				s.leaderChangedMu.Lock()
+				lc := s.leaderChanged
+				s.leaderChanged = make(chan struct{})
+				close(lc)
+				s.leaderChangedMu.Unlock()
+			}
+			// TODO: remove the nil checking
+			// current test utility does not provide the stats
+			if s.stats != nil {
+				s.stats.BecomeLeader()
+			}
+		},
+		updateCommittedIndex: func(ci uint64) {
+			cci := s.getCommittedIndex()
+			if ci > cci {
+				s.setCommittedIndex(ci)
+			}
+		},
+	}
+	s.r.start(rh)
+
+	ep := etcdProgress{
+		confState: sn.Metadata.ConfState,
+		snapi:     sn.Metadata.Index,
+		appliedt:  sn.Metadata.Term,
+		appliedi:  sn.Metadata.Index,
+	}
+
+	defer func() {
+		s.wgMu.Lock() // block concurrent waitgroup adds in goAttach while stopping
+		close(s.stopping)
+		s.wgMu.Unlock()
+		s.cancel()
+
+		sched.Stop()
+
+		// wait for gouroutines before closing raft so wal stays open
+		s.wg.Wait()
+
+		s.SyncTicker.Stop()
+
+		// must stop raft after scheduler-- etcdserver can leak rafthttp pipelines
+		// by adding a peer after raft stops the transport
+		s.r.stop()
+
+		// kv, lessor and backend can be nil if running without v3 enabled
+		// or running unit tests.
+		if s.lessor != nil {
+			s.lessor.Stop()
+		}
+		if s.kv != nil {
+			s.kv.Close()
+		}
+		if s.authStore != nil {
+			s.authStore.Close()
+		}
+		if s.be != nil {
+			s.be.Close()
+		}
+		if s.compactor != nil {
+			s.compactor.Stop()
+		}
+		close(s.done)
+	}()
+
+	var expiredLeaseC <-chan []*lease.Lease
+	if s.lessor != nil {
+		expiredLeaseC = s.lessor.ExpiredLeasesC()
+	}
+
+	for {
+		select {
+		case ap := <-s.r.apply():
+			f := func(context.Context) { s.applyAll(&ep, &ap) }
+			sched.Schedule(f)
+		case leases := <-expiredLeaseC:
+			s.goAttach(func() {
+				// Increases throughput of expired leases deletion process through parallelization
+				c := make(chan struct{}, maxPendingRevokes)
+				for _, lease := range leases {
+					select {
+					case c <- struct{}{}:
+					case <-s.stopping:
+						return
+					}
+					lid := lease.ID
+					s.goAttach(func() {
+						ctx := s.authStore.WithRoot(s.ctx)
+						_, lerr := s.LeaseRevoke(ctx, &pb.LeaseRevokeRequest{ID: int64(lid)})
+						if lerr == nil {
+							leaseExpired.Inc()
+						} else {
+							if lg != nil {
+								lg.Warn(
+									"failed to revoke lease",
+									zap.String("lease-id", fmt.Sprintf("%016x", lid)),
+									zap.Error(lerr),
+								)
+							} else {
+								plog.Warningf("failed to revoke %016x (%q)", lid, lerr.Error())
+							}
+						}
+
+						<-c
+					})
+				}
+			})
+		case err := <-s.errorc:
+			if lg != nil {
+				lg.Warn("server error", zap.Error(err))
+				lg.Warn("data-dir used by this member must be removed")
+			} else {
+				plog.Errorf("%s", err)
+				plog.Infof("the data-dir used by this member must be removed.")
+			}
+			return
+		case <-getSyncC():
+			if s.v2store.HasTTLKeys() {
+				s.sync(s.Cfg.ReqTimeout())
+			}
+		case <-s.stop:
+			return
+		}
+	}
+}
+
+func (s *EtcdServer) applyAll(ep *etcdProgress, apply *apply) {
+	s.applySnapshot(ep, apply)
+	s.applyEntries(ep, apply)
+
+	proposalsApplied.Set(float64(ep.appliedi))
+	s.applyWait.Trigger(ep.appliedi)
+
+	// wait for the raft routine to finish the disk writes before triggering a
+	// snapshot. or applied index might be greater than the last index in raft
+	// storage, since the raft routine might be slower than apply routine.
+	<-apply.notifyc
+
+	s.triggerSnapshot(ep)
+	select {
+	// snapshot requested via send()
+	case m := <-s.r.msgSnapC:
+		merged := s.createMergedSnapshotMessage(m, ep.appliedt, ep.appliedi, ep.confState)
+		s.sendMergedSnap(merged)
+	default:
+	}
+}
+
+func (s *EtcdServer) applySnapshot(ep *etcdProgress, apply *apply) {
+	if raft.IsEmptySnap(apply.snapshot) {
+		return
+	}
+	applySnapshotInProgress.Inc()
+
+	lg := s.getLogger()
+	if lg != nil {
+		lg.Info(
+			"applying snapshot",
+			zap.Uint64("current-snapshot-index", ep.snapi),
+			zap.Uint64("current-applied-index", ep.appliedi),
+			zap.Uint64("incoming-leader-snapshot-index", apply.snapshot.Metadata.Index),
+			zap.Uint64("incoming-leader-snapshot-term", apply.snapshot.Metadata.Term),
+		)
+	} else {
+		plog.Infof("applying snapshot at index %d...", ep.snapi)
+	}
+	defer func() {
+		if lg != nil {
+			lg.Info(
+				"applied snapshot",
+				zap.Uint64("current-snapshot-index", ep.snapi),
+				zap.Uint64("current-applied-index", ep.appliedi),
+				zap.Uint64("incoming-leader-snapshot-index", apply.snapshot.Metadata.Index),
+				zap.Uint64("incoming-leader-snapshot-term", apply.snapshot.Metadata.Term),
+			)
+		} else {
+			plog.Infof("finished applying incoming snapshot at index %d", ep.snapi)
+		}
+		applySnapshotInProgress.Dec()
+	}()
+
+	if apply.snapshot.Metadata.Index <= ep.appliedi {
+		if lg != nil {
+			lg.Panic(
+				"unexpected leader snapshot from outdated index",
+				zap.Uint64("current-snapshot-index", ep.snapi),
+				zap.Uint64("current-applied-index", ep.appliedi),
+				zap.Uint64("incoming-leader-snapshot-index", apply.snapshot.Metadata.Index),
+				zap.Uint64("incoming-leader-snapshot-term", apply.snapshot.Metadata.Term),
+			)
+		} else {
+			plog.Panicf("snapshot index [%d] should > appliedi[%d] + 1",
+				apply.snapshot.Metadata.Index, ep.appliedi)
+		}
+	}
+
+	// wait for raftNode to persist snapshot onto the disk
+	<-apply.notifyc
+
+	newbe, err := openSnapshotBackend(s.Cfg, s.snapshotter, apply.snapshot)
+	if err != nil {
+		if lg != nil {
+			lg.Panic("failed to open snapshot backend", zap.Error(err))
+		} else {
+			plog.Panic(err)
+		}
+	}
+
+	// always recover lessor before kv. When we recover the mvcc.KV it will reattach keys to its leases.
+	// If we recover mvcc.KV first, it will attach the keys to the wrong lessor before it recovers.
+	if s.lessor != nil {
+		if lg != nil {
+			lg.Info("restoring lease store")
+		} else {
+			plog.Info("recovering lessor...")
+		}
+
+		s.lessor.Recover(newbe, func() lease.TxnDelete { return s.kv.Write() })
+
+		if lg != nil {
+			lg.Info("restored lease store")
+		} else {
+			plog.Info("finished recovering lessor")
+		}
+	}
+
+	if lg != nil {
+		lg.Info("restoring mvcc store")
+	} else {
+		plog.Info("restoring mvcc store...")
+	}
+
+	if err := s.kv.Restore(newbe); err != nil {
+		if lg != nil {
+			lg.Panic("failed to restore mvcc store", zap.Error(err))
+		} else {
+			plog.Panicf("restore KV error: %v", err)
+		}
+	}
+
+	s.consistIndex.setConsistentIndex(s.kv.ConsistentIndex())
+	if lg != nil {
+		lg.Info("restored mvcc store")
+	} else {
+		plog.Info("finished restoring mvcc store")
+	}
+
+	// Closing old backend might block until all the txns
+	// on the backend are finished.
+	// We do not want to wait on closing the old backend.
+	s.bemu.Lock()
+	oldbe := s.be
+	go func() {
+		if lg != nil {
+			lg.Info("closing old backend file")
+		} else {
+			plog.Info("closing old backend...")
+		}
+		defer func() {
+			if lg != nil {
+				lg.Info("closed old backend file")
+			} else {
+				plog.Info("finished closing old backend")
+			}
+		}()
+		if err := oldbe.Close(); err != nil {
+			if lg != nil {
+				lg.Panic("failed to close old backend", zap.Error(err))
+			} else {
+				plog.Panicf("close backend error: %v", err)
+			}
+		}
+	}()
+
+	s.be = newbe
+	s.bemu.Unlock()
+
+	if lg != nil {
+		lg.Info("restoring alarm store")
+	} else {
+		plog.Info("recovering alarms...")
+	}
+
+	if err := s.restoreAlarms(); err != nil {
+		if lg != nil {
+			lg.Panic("failed to restore alarm store", zap.Error(err))
+		} else {
+			plog.Panicf("restore alarms error: %v", err)
+		}
+	}
+
+	if lg != nil {
+		lg.Info("restored alarm store")
+	} else {
+		plog.Info("finished recovering alarms")
+	}
+
+	if s.authStore != nil {
+		if lg != nil {
+			lg.Info("restoring auth store")
+		} else {
+			plog.Info("recovering auth store...")
+		}
+
+		s.authStore.Recover(newbe)
+
+		if lg != nil {
+			lg.Info("restored auth store")
+		} else {
+			plog.Info("finished recovering auth store")
+		}
+	}
+
+	if lg != nil {
+		lg.Info("restoring v2 store")
+	} else {
+		plog.Info("recovering store v2...")
+	}
+	if err := s.v2store.Recovery(apply.snapshot.Data); err != nil {
+		if lg != nil {
+			lg.Panic("failed to restore v2 store", zap.Error(err))
+		} else {
+			plog.Panicf("recovery store error: %v", err)
+		}
+	}
+
+	if lg != nil {
+		lg.Info("restored v2 store")
+	} else {
+		plog.Info("finished recovering store v2")
+	}
+
+	s.cluster.SetBackend(newbe)
+
+	if lg != nil {
+		lg.Info("restoring cluster configuration")
+	} else {
+		plog.Info("recovering cluster configuration...")
+	}
+
+	s.cluster.Recover(api.UpdateCapability)
+
+	if lg != nil {
+		lg.Info("restored cluster configuration")
+		lg.Info("removing old peers from network")
+	} else {
+		plog.Info("finished recovering cluster configuration")
+		plog.Info("removing old peers from network...")
+	}
+
+	// recover raft transport
+	s.r.transport.RemoveAllPeers()
+
+	if lg != nil {
+		lg.Info("removed old peers from network")
+		lg.Info("adding peers from new cluster configuration")
+	} else {
+		plog.Info("finished removing old peers from network")
+		plog.Info("adding peers from new cluster configuration into network...")
+	}
+
+	for _, m := range s.cluster.Members() {
+		if m.ID == s.ID() {
+			continue
+		}
+		s.r.transport.AddPeer(m.ID, m.PeerURLs)
+	}
+
+	if lg != nil {
+		lg.Info("added peers from new cluster configuration")
+	} else {
+		plog.Info("finished adding peers from new cluster configuration into network...")
+	}
+
+	ep.appliedt = apply.snapshot.Metadata.Term
+	ep.appliedi = apply.snapshot.Metadata.Index
+	ep.snapi = ep.appliedi
+	ep.confState = apply.snapshot.Metadata.ConfState
+}
+
+func (s *EtcdServer) applyEntries(ep *etcdProgress, apply *apply) {
+	if len(apply.entries) == 0 {
+		return
+	}
+	firsti := apply.entries[0].Index
+	if firsti > ep.appliedi+1 {
+		if lg := s.getLogger(); lg != nil {
+			lg.Panic(
+				"unexpected committed entry index",
+				zap.Uint64("current-applied-index", ep.appliedi),
+				zap.Uint64("first-committed-entry-index", firsti),
+			)
+		} else {
+			plog.Panicf("first index of committed entry[%d] should <= appliedi[%d] + 1", firsti, ep.appliedi)
+		}
+	}
+	var ents []raftpb.Entry
+	if ep.appliedi+1-firsti < uint64(len(apply.entries)) {
+		ents = apply.entries[ep.appliedi+1-firsti:]
+	}
+	if len(ents) == 0 {
+		return
+	}
+	var shouldstop bool
+	if ep.appliedt, ep.appliedi, shouldstop = s.apply(ents, &ep.confState); shouldstop {
+		go s.stopWithDelay(10*100*time.Millisecond, fmt.Errorf("the member has been permanently removed from the cluster"))
+	}
+}
+
+func (s *EtcdServer) triggerSnapshot(ep *etcdProgress) {
+	if ep.appliedi-ep.snapi <= s.Cfg.SnapshotCount {
+		return
+	}
+
+	if lg := s.getLogger(); lg != nil {
+		lg.Info(
+			"triggering snapshot",
+			zap.String("local-member-id", s.ID().String()),
+			zap.Uint64("local-member-applied-index", ep.appliedi),
+			zap.Uint64("local-member-snapshot-index", ep.snapi),
+			zap.Uint64("local-member-snapshot-count", s.Cfg.SnapshotCount),
+		)
+	} else {
+		plog.Infof("start to snapshot (applied: %d, lastsnap: %d)", ep.appliedi, ep.snapi)
+	}
+
+	s.snapshot(ep.appliedi, ep.confState)
+	ep.snapi = ep.appliedi
+}
+
+func (s *EtcdServer) hasMultipleVotingMembers() bool {
+	return s.cluster != nil && len(s.cluster.VotingMemberIDs()) > 1
+}
+
+func (s *EtcdServer) isLeader() bool {
+	return uint64(s.ID()) == s.Lead()
+}
+
+// MoveLeader transfers the leader to the given transferee.
+func (s *EtcdServer) MoveLeader(ctx context.Context, lead, transferee uint64) error {
+	if !s.cluster.IsMemberExist(types.ID(transferee)) || s.cluster.Member(types.ID(transferee)).IsLearner {
+		return ErrBadLeaderTransferee
+	}
+
+	now := time.Now()
+	interval := time.Duration(s.Cfg.TickMs) * time.Millisecond
+
+	if lg := s.getLogger(); lg != nil {
+		lg.Info(
+			"leadership transfer starting",
+			zap.String("local-member-id", s.ID().String()),
+			zap.String("current-leader-member-id", types.ID(lead).String()),
+			zap.String("transferee-member-id", types.ID(transferee).String()),
+		)
+	} else {
+		plog.Infof("%s starts leadership transfer from %s to %s", s.ID(), types.ID(lead), types.ID(transferee))
+	}
+
+	s.r.TransferLeadership(ctx, lead, transferee)
+	for s.Lead() != transferee {
+		select {
+		case <-ctx.Done(): // time out
+			return ErrTimeoutLeaderTransfer
+		case <-time.After(interval):
+		}
+	}
+
+	// TODO: drain all requests, or drop all messages to the old leader
+	if lg := s.getLogger(); lg != nil {
+		lg.Info(
+			"leadership transfer finished",
+			zap.String("local-member-id", s.ID().String()),
+			zap.String("old-leader-member-id", types.ID(lead).String()),
+			zap.String("new-leader-member-id", types.ID(transferee).String()),
+			zap.Duration("took", time.Since(now)),
+		)
+	} else {
+		plog.Infof("%s finished leadership transfer from %s to %s (took %v)", s.ID(), types.ID(lead), types.ID(transferee), time.Since(now))
+	}
+	return nil
+}
+
+// TransferLeadership transfers the leader to the chosen transferee.
+func (s *EtcdServer) TransferLeadership() error {
+	if !s.isLeader() {
+		if lg := s.getLogger(); lg != nil {
+			lg.Info(
+				"skipped leadership transfer; local server is not leader",
+				zap.String("local-member-id", s.ID().String()),
+				zap.String("current-leader-member-id", types.ID(s.Lead()).String()),
+			)
+		} else {
+			plog.Printf("skipped leadership transfer for stopping non-leader member")
+		}
+		return nil
+	}
+
+	if !s.hasMultipleVotingMembers() {
+		if lg := s.getLogger(); lg != nil {
+			lg.Info(
+				"skipped leadership transfer for single voting member cluster",
+				zap.String("local-member-id", s.ID().String()),
+				zap.String("current-leader-member-id", types.ID(s.Lead()).String()),
+			)
+		} else {
+			plog.Printf("skipped leadership transfer for single voting member cluster")
+		}
+		return nil
+	}
+
+	transferee, ok := longestConnected(s.r.transport, s.cluster.VotingMemberIDs())
+	if !ok {
+		return ErrUnhealthy
+	}
+
+	tm := s.Cfg.ReqTimeout()
+	ctx, cancel := context.WithTimeout(s.ctx, tm)
+	err := s.MoveLeader(ctx, s.Lead(), uint64(transferee))
+	cancel()
+	return err
+}
+
+// HardStop stops the server without coordination with other members in the cluster.
+func (s *EtcdServer) HardStop() {
+	select {
+	case s.stop <- struct{}{}:
+	case <-s.done:
+		return
+	}
+	<-s.done
+}
+
+// Stop stops the server gracefully, and shuts down the running goroutine.
+// Stop should be called after a Start(s), otherwise it will block forever.
+// When stopping leader, Stop transfers its leadership to one of its peers
+// before stopping the server.
+// Stop terminates the Server and performs any necessary finalization.
+// Do and Process cannot be called after Stop has been invoked.
+func (s *EtcdServer) Stop() {
+	if err := s.TransferLeadership(); err != nil {
+		if lg := s.getLogger(); lg != nil {
+			lg.Warn("leadership transfer failed", zap.String("local-member-id", s.ID().String()), zap.Error(err))
+		} else {
+			plog.Warningf("%s failed to transfer leadership (%v)", s.ID(), err)
+		}
+	}
+	s.HardStop()
+}
+
+// ReadyNotify returns a channel that will be closed when the server
+// is ready to serve client requests
+func (s *EtcdServer) ReadyNotify() <-chan struct{} { return s.readych }
+
+func (s *EtcdServer) stopWithDelay(d time.Duration, err error) {
+	select {
+	case <-time.After(d):
+	case <-s.done:
+	}
+	select {
+	case s.errorc <- err:
+	default:
+	}
+}
+
+// StopNotify returns a channel that receives a empty struct
+// when the server is stopped.
+func (s *EtcdServer) StopNotify() <-chan struct{} { return s.done }
+
+func (s *EtcdServer) SelfStats() []byte { return s.stats.JSON() }
+
+func (s *EtcdServer) LeaderStats() []byte {
+	lead := s.getLead()
+	if lead != uint64(s.id) {
+		return nil
+	}
+	return s.lstats.JSON()
+}
+
+func (s *EtcdServer) StoreStats() []byte { return s.v2store.JsonStats() }
+
+func (s *EtcdServer) checkMembershipOperationPermission(ctx context.Context) error {
+	if s.authStore == nil {
+		// In the context of ordinary etcd process, s.authStore will never be nil.
+		// This branch is for handling cases in server_test.go
+		return nil
+	}
+
+	// Note that this permission check is done in the API layer,
+	// so TOCTOU problem can be caused potentially in a schedule like this:
+	// update membership with user A -> revoke root role of A -> apply membership change
+	// in the state machine layer
+	// However, both of membership change and role management requires the root privilege.
+	// So careful operation by admins can prevent the problem.
+	authInfo, err := s.AuthInfoFromCtx(ctx)
+	if err != nil {
+		return err
+	}
+
+	return s.AuthStore().IsAdminPermitted(authInfo)
+}
+
+func (s *EtcdServer) AddMember(ctx context.Context, memb membership.Member) ([]*membership.Member, error) {
+	if err := s.checkMembershipOperationPermission(ctx); err != nil {
+		return nil, err
+	}
+
+	// TODO: move Member to protobuf type
+	b, err := json.Marshal(memb)
+	if err != nil {
+		return nil, err
+	}
+
+	// by default StrictReconfigCheck is enabled; reject new members if unhealthy.
+	if err := s.mayAddMember(memb); err != nil {
+		return nil, err
+	}
+
+	cc := raftpb.ConfChange{
+		Type:    raftpb.ConfChangeAddNode,
+		NodeID:  uint64(memb.ID),
+		Context: b,
+	}
+
+	if memb.IsLearner {
+		cc.Type = raftpb.ConfChangeAddLearnerNode
+	}
+
+	return s.configure(ctx, cc)
+}
+
+func (s *EtcdServer) mayAddMember(memb membership.Member) error {
+	if !s.Cfg.StrictReconfigCheck {
+		return nil
+	}
+
+	// protect quorum when adding voting member
+	if !memb.IsLearner && !s.cluster.IsReadyToAddVotingMember() {
+		if lg := s.getLogger(); lg != nil {
+			lg.Warn(
+				"rejecting member add request; not enough healthy members",
+				zap.String("local-member-id", s.ID().String()),
+				zap.String("requested-member-add", fmt.Sprintf("%+v", memb)),
+				zap.Error(ErrNotEnoughStartedMembers),
+			)
+		} else {
+			plog.Warningf("not enough started members, rejecting member add %+v", memb)
+		}
+		return ErrNotEnoughStartedMembers
+	}
+
+	if !isConnectedFullySince(s.r.transport, time.Now().Add(-HealthInterval), s.ID(), s.cluster.VotingMembers()) {
+		if lg := s.getLogger(); lg != nil {
+			lg.Warn(
+				"rejecting member add request; local member has not been connected to all peers, reconfigure breaks active quorum",
+				zap.String("local-member-id", s.ID().String()),
+				zap.String("requested-member-add", fmt.Sprintf("%+v", memb)),
+				zap.Error(ErrUnhealthy),
+			)
+		} else {
+			plog.Warningf("not healthy for reconfigure, rejecting member add %+v", memb)
+		}
+		return ErrUnhealthy
+	}
+
+	return nil
+}
+
+func (s *EtcdServer) RemoveMember(ctx context.Context, id uint64) ([]*membership.Member, error) {
+	if err := s.checkMembershipOperationPermission(ctx); err != nil {
+		return nil, err
+	}
+
+	// by default StrictReconfigCheck is enabled; reject removal if leads to quorum loss
+	if err := s.mayRemoveMember(types.ID(id)); err != nil {
+		return nil, err
+	}
+
+	cc := raftpb.ConfChange{
+		Type:   raftpb.ConfChangeRemoveNode,
+		NodeID: id,
+	}
+	return s.configure(ctx, cc)
+}
+
+// PromoteMember promotes a learner node to a voting node.
+func (s *EtcdServer) PromoteMember(ctx context.Context, id uint64) ([]*membership.Member, error) {
+	// only raft leader has information on whether the to-be-promoted learner node is ready. If promoteMember call
+	// fails with ErrNotLeader, forward the request to leader node via HTTP. If promoteMember call fails with error
+	// other than ErrNotLeader, return the error.
+	resp, err := s.promoteMember(ctx, id)
+	if err == nil {
+		learnerPromoteSucceed.Inc()
+		return resp, nil
+	}
+	if err != ErrNotLeader {
+		learnerPromoteFailed.WithLabelValues(err.Error()).Inc()
+		return resp, err
+	}
+
+	cctx, cancel := context.WithTimeout(ctx, s.Cfg.ReqTimeout())
+	defer cancel()
+	// forward to leader
+	for cctx.Err() == nil {
+		leader, err := s.waitLeader(cctx)
+		if err != nil {
+			return nil, err
+		}
+		for _, url := range leader.PeerURLs {
+			resp, err := promoteMemberHTTP(cctx, url, id, s.peerRt)
+			if err == nil {
+				return resp, nil
+			}
+			// If member promotion failed, return early. Otherwise keep retry.
+			if err == ErrLearnerNotReady || err == membership.ErrIDNotFound || err == membership.ErrMemberNotLearner {
+				return nil, err
+			}
+		}
+	}
+
+	if cctx.Err() == context.DeadlineExceeded {
+		return nil, ErrTimeout
+	}
+	return nil, ErrCanceled
+}
+
+// promoteMember checks whether the to-be-promoted learner node is ready before sending the promote
+// request to raft.
+// The function returns ErrNotLeader if the local node is not raft leader (therefore does not have
+// enough information to determine if the learner node is ready), returns ErrLearnerNotReady if the
+// local node is leader (therefore has enough information) but decided the learner node is not ready
+// to be promoted.
+func (s *EtcdServer) promoteMember(ctx context.Context, id uint64) ([]*membership.Member, error) {
+	if err := s.checkMembershipOperationPermission(ctx); err != nil {
+		return nil, err
+	}
+
+	// check if we can promote this learner.
+	if err := s.mayPromoteMember(types.ID(id)); err != nil {
+		return nil, err
+	}
+
+	// build the context for the promote confChange. mark IsLearner to false and IsPromote to true.
+	promoteChangeContext := membership.ConfigChangeContext{
+		Member: membership.Member{
+			ID: types.ID(id),
+		},
+		IsPromote: true,
+	}
+
+	b, err := json.Marshal(promoteChangeContext)
+	if err != nil {
+		return nil, err
+	}
+
+	cc := raftpb.ConfChange{
+		Type:    raftpb.ConfChangeAddNode,
+		NodeID:  id,
+		Context: b,
+	}
+
+	return s.configure(ctx, cc)
+}
+
+func (s *EtcdServer) mayPromoteMember(id types.ID) error {
+	err := s.isLearnerReady(uint64(id))
+	if err != nil {
+		return err
+	}
+
+	if !s.Cfg.StrictReconfigCheck {
+		return nil
+	}
+	if !s.cluster.IsReadyToPromoteMember(uint64(id)) {
+		if lg := s.getLogger(); lg != nil {
+			lg.Warn(
+				"rejecting member promote request; not enough healthy members",
+				zap.String("local-member-id", s.ID().String()),
+				zap.String("requested-member-remove-id", id.String()),
+				zap.Error(ErrNotEnoughStartedMembers),
+			)
+		} else {
+			plog.Warningf("not enough started members, rejecting promote member %s", id)
+		}
+		return ErrNotEnoughStartedMembers
+	}
+
+	return nil
+}
+
+// check whether the learner catches up with leader or not.
+// Note: it will return nil if member is not found in cluster or if member is not learner.
+// These two conditions will be checked before apply phase later.
+func (s *EtcdServer) isLearnerReady(id uint64) error {
+	rs := s.raftStatus()
+
+	// leader's raftStatus.Progress is not nil
+	if rs.Progress == nil {
+		return ErrNotLeader
+	}
+
+	var learnerMatch uint64
+	isFound := false
+	leaderID := rs.ID
+	for memberID, progress := range rs.Progress {
+		if id == memberID {
+			// check its status
+			learnerMatch = progress.Match
+			isFound = true
+			break
+		}
+	}
+
+	if isFound {
+		leaderMatch := rs.Progress[leaderID].Match
+		// the learner's Match not caught up with leader yet
+		if float64(learnerMatch) < float64(leaderMatch)*readyPercent {
+			return ErrLearnerNotReady
+		}
+	}
+
+	return nil
+}
+
+func (s *EtcdServer) mayRemoveMember(id types.ID) error {
+	if !s.Cfg.StrictReconfigCheck {
+		return nil
+	}
+
+	isLearner := s.cluster.IsMemberExist(id) && s.cluster.Member(id).IsLearner
+	// no need to check quorum when removing non-voting member
+	if isLearner {
+		return nil
+	}
+
+	if !s.cluster.IsReadyToRemoveVotingMember(uint64(id)) {
+		if lg := s.getLogger(); lg != nil {
+			lg.Warn(
+				"rejecting member remove request; not enough healthy members",
+				zap.String("local-member-id", s.ID().String()),
+				zap.String("requested-member-remove-id", id.String()),
+				zap.Error(ErrNotEnoughStartedMembers),
+			)
+		} else {
+			plog.Warningf("not enough started members, rejecting remove member %s", id)
+		}
+		return ErrNotEnoughStartedMembers
+	}
+
+	// downed member is safe to remove since it's not part of the active quorum
+	if t := s.r.transport.ActiveSince(id); id != s.ID() && t.IsZero() {
+		return nil
+	}
+
+	// protect quorum if some members are down
+	m := s.cluster.VotingMembers()
+	active := numConnectedSince(s.r.transport, time.Now().Add(-HealthInterval), s.ID(), m)
+	if (active - 1) < 1+((len(m)-1)/2) {
+		if lg := s.getLogger(); lg != nil {
+			lg.Warn(
+				"rejecting member remove request; local member has not been connected to all peers, reconfigure breaks active quorum",
+				zap.String("local-member-id", s.ID().String()),
+				zap.String("requested-member-remove", id.String()),
+				zap.Int("active-peers", active),
+				zap.Error(ErrUnhealthy),
+			)
+		} else {
+			plog.Warningf("reconfigure breaks active quorum, rejecting remove member %s", id)
+		}
+		return ErrUnhealthy
+	}
+
+	return nil
+}
+
+func (s *EtcdServer) UpdateMember(ctx context.Context, memb membership.Member) ([]*membership.Member, error) {
+	b, merr := json.Marshal(memb)
+	if merr != nil {
+		return nil, merr
+	}
+
+	if err := s.checkMembershipOperationPermission(ctx); err != nil {
+		return nil, err
+	}
+	cc := raftpb.ConfChange{
+		Type:    raftpb.ConfChangeUpdateNode,
+		NodeID:  uint64(memb.ID),
+		Context: b,
+	}
+	return s.configure(ctx, cc)
+}
+
+func (s *EtcdServer) setCommittedIndex(v uint64) {
+	atomic.StoreUint64(&s.committedIndex, v)
+}
+
+func (s *EtcdServer) getCommittedIndex() uint64 {
+	return atomic.LoadUint64(&s.committedIndex)
+}
+
+func (s *EtcdServer) setAppliedIndex(v uint64) {
+	atomic.StoreUint64(&s.appliedIndex, v)
+}
+
+func (s *EtcdServer) getAppliedIndex() uint64 {
+	return atomic.LoadUint64(&s.appliedIndex)
+}
+
+func (s *EtcdServer) setTerm(v uint64) {
+	atomic.StoreUint64(&s.term, v)
+}
+
+func (s *EtcdServer) getTerm() uint64 {
+	return atomic.LoadUint64(&s.term)
+}
+
+func (s *EtcdServer) setLead(v uint64) {
+	atomic.StoreUint64(&s.lead, v)
+}
+
+func (s *EtcdServer) getLead() uint64 {
+	return atomic.LoadUint64(&s.lead)
+}
+
+func (s *EtcdServer) leaderChangedNotify() <-chan struct{} {
+	s.leaderChangedMu.RLock()
+	defer s.leaderChangedMu.RUnlock()
+	return s.leaderChanged
+}
+
+// RaftStatusGetter represents etcd server and Raft progress.
+type RaftStatusGetter interface {
+	ID() types.ID
+	Leader() types.ID
+	CommittedIndex() uint64
+	AppliedIndex() uint64
+	Term() uint64
+}
+
+func (s *EtcdServer) ID() types.ID { return s.id }
+
+func (s *EtcdServer) Leader() types.ID { return types.ID(s.getLead()) }
+
+func (s *EtcdServer) Lead() uint64 { return s.getLead() }
+
+func (s *EtcdServer) CommittedIndex() uint64 { return s.getCommittedIndex() }
+
+func (s *EtcdServer) AppliedIndex() uint64 { return s.getAppliedIndex() }
+
+func (s *EtcdServer) Term() uint64 { return s.getTerm() }
+
+type confChangeResponse struct {
+	membs []*membership.Member
+	err   error
+}
+
+// configure sends a configuration change through consensus and
+// then waits for it to be applied to the server. It
+// will block until the change is performed or there is an error.
+func (s *EtcdServer) configure(ctx context.Context, cc raftpb.ConfChange) ([]*membership.Member, error) {
+	cc.ID = s.reqIDGen.Next()
+	ch := s.w.Register(cc.ID)
+
+	start := time.Now()
+	if err := s.r.ProposeConfChange(ctx, cc); err != nil {
+		s.w.Trigger(cc.ID, nil)
+		return nil, err
+	}
+
+	select {
+	case x := <-ch:
+		if x == nil {
+			if lg := s.getLogger(); lg != nil {
+				lg.Panic("failed to configure")
+			} else {
+				plog.Panicf("configure trigger value should never be nil")
+			}
+		}
+		resp := x.(*confChangeResponse)
+		if lg := s.getLogger(); lg != nil {
+			lg.Info(
+				"applied a configuration change through raft",
+				zap.String("local-member-id", s.ID().String()),
+				zap.String("raft-conf-change", cc.Type.String()),
+				zap.String("raft-conf-change-node-id", types.ID(cc.NodeID).String()),
+			)
+		}
+		return resp.membs, resp.err
+
+	case <-ctx.Done():
+		s.w.Trigger(cc.ID, nil) // GC wait
+		return nil, s.parseProposeCtxErr(ctx.Err(), start)
+
+	case <-s.stopping:
+		return nil, ErrStopped
+	}
+}
+
+// sync proposes a SYNC request and is non-blocking.
+// This makes no guarantee that the request will be proposed or performed.
+// The request will be canceled after the given timeout.
+func (s *EtcdServer) sync(timeout time.Duration) {
+	req := pb.Request{
+		Method: "SYNC",
+		ID:     s.reqIDGen.Next(),
+		Time:   time.Now().UnixNano(),
+	}
+	data := pbutil.MustMarshal(&req)
+	// There is no promise that node has leader when do SYNC request,
+	// so it uses goroutine to propose.
+	ctx, cancel := context.WithTimeout(s.ctx, timeout)
+	s.goAttach(func() {
+		s.r.Propose(ctx, data)
+		cancel()
+	})
+}
+
+// publish registers server information into the cluster. The information
+// is the JSON representation of this server's member struct, updated with the
+// static clientURLs of the server.
+// The function keeps attempting to register until it succeeds,
+// or its server is stopped.
+//
+// Use v2 store to encode member attributes, and apply through Raft
+// but does not go through v2 API endpoint, which means even with v2
+// client handler disabled (e.g. --enable-v2=false), cluster can still
+// process publish requests through rafthttp
+// TODO: Deprecate v2 store
+func (s *EtcdServer) publish(timeout time.Duration) {
+	b, err := json.Marshal(s.attributes)
+	if err != nil {
+		if lg := s.getLogger(); lg != nil {
+			lg.Panic("failed to marshal JSON", zap.Error(err))
+		} else {
+			plog.Panicf("json marshal error: %v", err)
+		}
+		return
+	}
+	req := pb.Request{
+		Method: "PUT",
+		Path:   membership.MemberAttributesStorePath(s.id),
+		Val:    string(b),
+	}
+
+	for {
+		ctx, cancel := context.WithTimeout(s.ctx, timeout)
+		_, err := s.Do(ctx, req)
+		cancel()
+		switch err {
+		case nil:
+			close(s.readych)
+			if lg := s.getLogger(); lg != nil {
+				lg.Info(
+					"published local member to cluster through raft",
+					zap.String("local-member-id", s.ID().String()),
+					zap.String("local-member-attributes", fmt.Sprintf("%+v", s.attributes)),
+					zap.String("request-path", req.Path),
+					zap.String("cluster-id", s.cluster.ID().String()),
+					zap.Duration("publish-timeout", timeout),
+				)
+			} else {
+				plog.Infof("published %+v to cluster %s", s.attributes, s.cluster.ID())
+			}
+			return
+
+		case ErrStopped:
+			if lg := s.getLogger(); lg != nil {
+				lg.Warn(
+					"stopped publish because server is stopped",
+					zap.String("local-member-id", s.ID().String()),
+					zap.String("local-member-attributes", fmt.Sprintf("%+v", s.attributes)),
+					zap.Duration("publish-timeout", timeout),
+					zap.Error(err),
+				)
+			} else {
+				plog.Infof("aborting publish because server is stopped")
+			}
+			return
+
+		default:
+			if lg := s.getLogger(); lg != nil {
+				lg.Warn(
+					"failed to publish local member to cluster through raft",
+					zap.String("local-member-id", s.ID().String()),
+					zap.String("local-member-attributes", fmt.Sprintf("%+v", s.attributes)),
+					zap.String("request-path", req.Path),
+					zap.Duration("publish-timeout", timeout),
+					zap.Error(err),
+				)
+			} else {
+				plog.Errorf("publish error: %v", err)
+			}
+		}
+	}
+}
+
+func (s *EtcdServer) sendMergedSnap(merged snap.Message) {
+	atomic.AddInt64(&s.inflightSnapshots, 1)
+
+	lg := s.getLogger()
+	fields := []zap.Field{
+		zap.String("from", s.ID().String()),
+		zap.String("to", types.ID(merged.To).String()),
+		zap.Int64("bytes", merged.TotalSize),
+		zap.String("size", humanize.Bytes(uint64(merged.TotalSize))),
+	}
+
+	now := time.Now()
+	s.r.transport.SendSnapshot(merged)
+	if lg != nil {
+		lg.Info("sending merged snapshot", fields...)
+	}
+
+	s.goAttach(func() {
+		select {
+		case ok := <-merged.CloseNotify():
+			// delay releasing inflight snapshot for another 30 seconds to
+			// block log compaction.
+			// If the follower still fails to catch up, it is probably just too slow
+			// to catch up. We cannot avoid the snapshot cycle anyway.
+			if ok {
+				select {
+				case <-time.After(releaseDelayAfterSnapshot):
+				case <-s.stopping:
+				}
+			}
+
+			atomic.AddInt64(&s.inflightSnapshots, -1)
+
+			if lg != nil {
+				lg.Info("sent merged snapshot", append(fields, zap.Duration("took", time.Since(now)))...)
+			}
+
+		case <-s.stopping:
+			if lg != nil {
+				lg.Warn("canceled sending merged snapshot; server stopping", fields...)
+			}
+			return
+		}
+	})
+}
+
+// apply takes entries received from Raft (after it has been committed) and
+// applies them to the current state of the EtcdServer.
+// The given entries should not be empty.
+func (s *EtcdServer) apply(
+	es []raftpb.Entry,
+	confState *raftpb.ConfState,
+) (appliedt uint64, appliedi uint64, shouldStop bool) {
+	for i := range es {
+		e := es[i]
+		switch e.Type {
+		case raftpb.EntryNormal:
+			s.applyEntryNormal(&e)
+			s.setAppliedIndex(e.Index)
+			s.setTerm(e.Term)
+
+		case raftpb.EntryConfChange:
+			// set the consistent index of current executing entry
+			if e.Index > s.consistIndex.ConsistentIndex() {
+				s.consistIndex.setConsistentIndex(e.Index)
+			}
+			var cc raftpb.ConfChange
+			pbutil.MustUnmarshal(&cc, e.Data)
+			removedSelf, err := s.applyConfChange(cc, confState)
+			s.setAppliedIndex(e.Index)
+			s.setTerm(e.Term)
+			shouldStop = shouldStop || removedSelf
+			s.w.Trigger(cc.ID, &confChangeResponse{s.cluster.Members(), err})
+
+		default:
+			if lg := s.getLogger(); lg != nil {
+				lg.Panic(
+					"unknown entry type; must be either EntryNormal or EntryConfChange",
+					zap.String("type", e.Type.String()),
+				)
+			} else {
+				plog.Panicf("entry type should be either EntryNormal or EntryConfChange")
+			}
+		}
+		appliedi, appliedt = e.Index, e.Term
+	}
+	return appliedt, appliedi, shouldStop
+}
+
+// applyEntryNormal apples an EntryNormal type raftpb request to the EtcdServer
+func (s *EtcdServer) applyEntryNormal(e *raftpb.Entry) {
+	shouldApplyV3 := false
+	if e.Index > s.consistIndex.ConsistentIndex() {
+		// set the consistent index of current executing entry
+		s.consistIndex.setConsistentIndex(e.Index)
+		shouldApplyV3 = true
+	}
+
+	// raft state machine may generate noop entry when leader confirmation.
+	// skip it in advance to avoid some potential bug in the future
+	if len(e.Data) == 0 {
+		select {
+		case s.forceVersionC <- struct{}{}:
+		default:
+		}
+		// promote lessor when the local member is leader and finished
+		// applying all entries from the last term.
+		if s.isLeader() {
+			s.lessor.Promote(s.Cfg.electionTimeout())
+		}
+		return
+	}
+
+	var raftReq pb.InternalRaftRequest
+	if !pbutil.MaybeUnmarshal(&raftReq, e.Data) { // backward compatible
+		var r pb.Request
+		rp := &r
+		pbutil.MustUnmarshal(rp, e.Data)
+		s.w.Trigger(r.ID, s.applyV2Request((*RequestV2)(rp)))
+		return
+	}
+	if raftReq.V2 != nil {
+		req := (*RequestV2)(raftReq.V2)
+		s.w.Trigger(req.ID, s.applyV2Request(req))
+		return
+	}
+
+	// do not re-apply applied entries.
+	if !shouldApplyV3 {
+		return
+	}
+
+	id := raftReq.ID
+	if id == 0 {
+		id = raftReq.Header.ID
+	}
+
+	var ar *applyResult
+	needResult := s.w.IsRegistered(id)
+	if needResult || !noSideEffect(&raftReq) {
+		if !needResult && raftReq.Txn != nil {
+			removeNeedlessRangeReqs(raftReq.Txn)
+		}
+		ar = s.applyV3.Apply(&raftReq)
+	}
+
+	if ar == nil {
+		return
+	}
+
+	if ar.err != ErrNoSpace || len(s.alarmStore.Get(pb.AlarmType_NOSPACE)) > 0 {
+		s.w.Trigger(id, ar)
+		return
+	}
+
+	if lg := s.getLogger(); lg != nil {
+		lg.Warn(
+			"message exceeded backend quota; raising alarm",
+			zap.Int64("quota-size-bytes", s.Cfg.QuotaBackendBytes),
+			zap.String("quota-size", humanize.Bytes(uint64(s.Cfg.QuotaBackendBytes))),
+			zap.Error(ar.err),
+		)
+	} else {
+		plog.Errorf("applying raft message exceeded backend quota")
+	}
+
+	s.goAttach(func() {
+		a := &pb.AlarmRequest{
+			MemberID: uint64(s.ID()),
+			Action:   pb.AlarmRequest_ACTIVATE,
+			Alarm:    pb.AlarmType_NOSPACE,
+		}
+		s.raftRequest(s.ctx, pb.InternalRaftRequest{Alarm: a})
+		s.w.Trigger(id, ar)
+	})
+}
+
+// applyConfChange applies a ConfChange to the server. It is only
+// invoked with a ConfChange that has already passed through Raft
+func (s *EtcdServer) applyConfChange(cc raftpb.ConfChange, confState *raftpb.ConfState) (bool, error) {
+	if err := s.cluster.ValidateConfigurationChange(cc); err != nil {
+		cc.NodeID = raft.None
+		s.r.ApplyConfChange(cc)
+		return false, err
+	}
+
+	lg := s.getLogger()
+	*confState = *s.r.ApplyConfChange(cc)
+	switch cc.Type {
+	case raftpb.ConfChangeAddNode, raftpb.ConfChangeAddLearnerNode:
+		confChangeContext := new(membership.ConfigChangeContext)
+		if err := json.Unmarshal(cc.Context, confChangeContext); err != nil {
+			if lg != nil {
+				lg.Panic("failed to unmarshal member", zap.Error(err))
+			} else {
+				plog.Panicf("unmarshal member should never fail: %v", err)
+			}
+		}
+		if cc.NodeID != uint64(confChangeContext.Member.ID) {
+			if lg != nil {
+				lg.Panic(
+					"got different member ID",
+					zap.String("member-id-from-config-change-entry", types.ID(cc.NodeID).String()),
+					zap.String("member-id-from-message", confChangeContext.Member.ID.String()),
+				)
+			} else {
+				plog.Panicf("nodeID should always be equal to member ID")
+			}
+		}
+		if confChangeContext.IsPromote {
+			s.cluster.PromoteMember(confChangeContext.Member.ID)
+		} else {
+			s.cluster.AddMember(&confChangeContext.Member)
+
+			if confChangeContext.Member.ID != s.id {
+				s.r.transport.AddPeer(confChangeContext.Member.ID, confChangeContext.PeerURLs)
+			}
+		}
+
+		// update the isLearner metric when this server id is equal to the id in raft member confChange
+		if confChangeContext.Member.ID == s.id {
+			if cc.Type == raftpb.ConfChangeAddLearnerNode {
+				isLearner.Set(1)
+			} else {
+				isLearner.Set(0)
+			}
+		}
+
+	case raftpb.ConfChangeRemoveNode:
+		id := types.ID(cc.NodeID)
+		s.cluster.RemoveMember(id)
+		if id == s.id {
+			return true, nil
+		}
+		s.r.transport.RemovePeer(id)
+
+	case raftpb.ConfChangeUpdateNode:
+		m := new(membership.Member)
+		if err := json.Unmarshal(cc.Context, m); err != nil {
+			if lg != nil {
+				lg.Panic("failed to unmarshal member", zap.Error(err))
+			} else {
+				plog.Panicf("unmarshal member should never fail: %v", err)
+			}
+		}
+		if cc.NodeID != uint64(m.ID) {
+			if lg != nil {
+				lg.Panic(
+					"got different member ID",
+					zap.String("member-id-from-config-change-entry", types.ID(cc.NodeID).String()),
+					zap.String("member-id-from-message", m.ID.String()),
+				)
+			} else {
+				plog.Panicf("nodeID should always be equal to member ID")
+			}
+		}
+		s.cluster.UpdateRaftAttributes(m.ID, m.RaftAttributes)
+		if m.ID != s.id {
+			s.r.transport.UpdatePeer(m.ID, m.PeerURLs)
+		}
+	}
+	return false, nil
+}
+
+// TODO: non-blocking snapshot
+func (s *EtcdServer) snapshot(snapi uint64, confState raftpb.ConfState) {
+	clone := s.v2store.Clone()
+	// commit kv to write metadata (for example: consistent index) to disk.
+	// KV().commit() updates the consistent index in backend.
+	// All operations that update consistent index must be called sequentially
+	// from applyAll function.
+	// So KV().Commit() cannot run in parallel with apply. It has to be called outside
+	// the go routine created below.
+	s.KV().Commit()
+
+	s.goAttach(func() {
+		lg := s.getLogger()
+
+		d, err := clone.SaveNoCopy()
+		// TODO: current store will never fail to do a snapshot
+		// what should we do if the store might fail?
+		if err != nil {
+			if lg != nil {
+				lg.Panic("failed to save v2 store", zap.Error(err))
+			} else {
+				plog.Panicf("store save should never fail: %v", err)
+			}
+		}
+		snap, err := s.r.raftStorage.CreateSnapshot(snapi, &confState, d)
+		if err != nil {
+			// the snapshot was done asynchronously with the progress of raft.
+			// raft might have already got a newer snapshot.
+			if err == raft.ErrSnapOutOfDate {
+				return
+			}
+			if lg != nil {
+				lg.Panic("failed to create snapshot", zap.Error(err))
+			} else {
+				plog.Panicf("unexpected create snapshot error %v", err)
+			}
+		}
+		// SaveSnap saves the snapshot and releases the locked wal files
+		// to the snapshot index.
+		if err = s.r.storage.SaveSnap(snap); err != nil {
+			if lg != nil {
+				lg.Panic("failed to save snapshot", zap.Error(err))
+			} else {
+				plog.Fatalf("save snapshot error: %v", err)
+			}
+		}
+		if lg != nil {
+			lg.Info(
+				"saved snapshot",
+				zap.Uint64("snapshot-index", snap.Metadata.Index),
+			)
+		} else {
+			plog.Infof("saved snapshot at index %d", snap.Metadata.Index)
+		}
+
+		// When sending a snapshot, etcd will pause compaction.
+		// After receives a snapshot, the slow follower needs to get all the entries right after
+		// the snapshot sent to catch up. If we do not pause compaction, the log entries right after
+		// the snapshot sent might already be compacted. It happens when the snapshot takes long time
+		// to send and save. Pausing compaction avoids triggering a snapshot sending cycle.
+		if atomic.LoadInt64(&s.inflightSnapshots) != 0 {
+			if lg != nil {
+				lg.Info("skip compaction since there is an inflight snapshot")
+			} else {
+				plog.Infof("skip compaction since there is an inflight snapshot")
+			}
+			return
+		}
+
+		// keep some in memory log entries for slow followers.
+		compacti := uint64(1)
+		if snapi > s.Cfg.SnapshotCatchUpEntries {
+			compacti = snapi - s.Cfg.SnapshotCatchUpEntries
+		}
+
+		err = s.r.raftStorage.Compact(compacti)
+		if err != nil {
+			// the compaction was done asynchronously with the progress of raft.
+			// raft log might already been compact.
+			if err == raft.ErrCompacted {
+				return
+			}
+			if lg != nil {
+				lg.Panic("failed to compact", zap.Error(err))
+			} else {
+				plog.Panicf("unexpected compaction error %v", err)
+			}
+		}
+		if lg != nil {
+			lg.Info(
+				"compacted Raft logs",
+				zap.Uint64("compact-index", compacti),
+			)
+		} else {
+			plog.Infof("compacted raft log at %d", compacti)
+		}
+	})
+}
+
+// CutPeer drops messages to the specified peer.
+func (s *EtcdServer) CutPeer(id types.ID) {
+	tr, ok := s.r.transport.(*rafthttp.Transport)
+	if ok {
+		tr.CutPeer(id)
+	}
+}
+
+// MendPeer recovers the message dropping behavior of the given peer.
+func (s *EtcdServer) MendPeer(id types.ID) {
+	tr, ok := s.r.transport.(*rafthttp.Transport)
+	if ok {
+		tr.MendPeer(id)
+	}
+}
+
+func (s *EtcdServer) PauseSending() { s.r.pauseSending() }
+
+func (s *EtcdServer) ResumeSending() { s.r.resumeSending() }
+
+func (s *EtcdServer) ClusterVersion() *semver.Version {
+	if s.cluster == nil {
+		return nil
+	}
+	return s.cluster.Version()
+}
+
+// monitorVersions checks the member's version every monitorVersionInterval.
+// It updates the cluster version if all members agrees on a higher one.
+// It prints out log if there is a member with a higher version than the
+// local version.
+func (s *EtcdServer) monitorVersions() {
+	for {
+		select {
+		case <-s.forceVersionC:
+		case <-time.After(monitorVersionInterval):
+		case <-s.stopping:
+			return
+		}
+
+		if s.Leader() != s.ID() {
+			continue
+		}
+
+		v := decideClusterVersion(s.getLogger(), getVersions(s.getLogger(), s.cluster, s.id, s.peerRt))
+		if v != nil {
+			// only keep major.minor version for comparison
+			v = &semver.Version{
+				Major: v.Major,
+				Minor: v.Minor,
+			}
+		}
+
+		// if the current version is nil:
+		// 1. use the decided version if possible
+		// 2. or use the min cluster version
+		if s.cluster.Version() == nil {
+			verStr := version.MinClusterVersion
+			if v != nil {
+				verStr = v.String()
+			}
+			s.goAttach(func() { s.updateClusterVersion(verStr) })
+			continue
+		}
+
+		// update cluster version only if the decided version is greater than
+		// the current cluster version
+		if v != nil && s.cluster.Version().LessThan(*v) {
+			s.goAttach(func() { s.updateClusterVersion(v.String()) })
+		}
+	}
+}
+
+func (s *EtcdServer) updateClusterVersion(ver string) {
+	lg := s.getLogger()
+
+	if s.cluster.Version() == nil {
+		if lg != nil {
+			lg.Info(
+				"setting up initial cluster version",
+				zap.String("cluster-version", version.Cluster(ver)),
+			)
+		} else {
+			plog.Infof("setting up the initial cluster version to %s", version.Cluster(ver))
+		}
+	} else {
+		if lg != nil {
+			lg.Info(
+				"updating cluster version",
+				zap.String("from", version.Cluster(s.cluster.Version().String())),
+				zap.String("to", version.Cluster(ver)),
+			)
+		} else {
+			plog.Infof("updating the cluster version from %s to %s", version.Cluster(s.cluster.Version().String()), version.Cluster(ver))
+		}
+	}
+
+	req := pb.Request{
+		Method: "PUT",
+		Path:   membership.StoreClusterVersionKey(),
+		Val:    ver,
+	}
+
+	ctx, cancel := context.WithTimeout(s.ctx, s.Cfg.ReqTimeout())
+	_, err := s.Do(ctx, req)
+	cancel()
+
+	switch err {
+	case nil:
+		if lg != nil {
+			lg.Info("cluster version is updated", zap.String("cluster-version", version.Cluster(ver)))
+		}
+		return
+
+	case ErrStopped:
+		if lg != nil {
+			lg.Warn("aborting cluster version update; server is stopped", zap.Error(err))
+		} else {
+			plog.Infof("aborting update cluster version because server is stopped")
+		}
+		return
+
+	default:
+		if lg != nil {
+			lg.Warn("failed to update cluster version", zap.Error(err))
+		} else {
+			plog.Errorf("error updating cluster version (%v)", err)
+		}
+	}
+}
+
+func (s *EtcdServer) parseProposeCtxErr(err error, start time.Time) error {
+	switch err {
+	case context.Canceled:
+		return ErrCanceled
+
+	case context.DeadlineExceeded:
+		s.leadTimeMu.RLock()
+		curLeadElected := s.leadElectedTime
+		s.leadTimeMu.RUnlock()
+		prevLeadLost := curLeadElected.Add(-2 * time.Duration(s.Cfg.ElectionTicks) * time.Duration(s.Cfg.TickMs) * time.Millisecond)
+		if start.After(prevLeadLost) && start.Before(curLeadElected) {
+			return ErrTimeoutDueToLeaderFail
+		}
+		lead := types.ID(s.getLead())
+		switch lead {
+		case types.ID(raft.None):
+			// TODO: return error to specify it happens because the cluster does not have leader now
+		case s.ID():
+			if !isConnectedToQuorumSince(s.r.transport, start, s.ID(), s.cluster.Members()) {
+				return ErrTimeoutDueToConnectionLost
+			}
+		default:
+			if !isConnectedSince(s.r.transport, start, lead) {
+				return ErrTimeoutDueToConnectionLost
+			}
+		}
+		return ErrTimeout
+
+	default:
+		return err
+	}
+}
+
+func (s *EtcdServer) KV() mvcc.ConsistentWatchableKV { return s.kv }
+func (s *EtcdServer) Backend() backend.Backend {
+	s.bemu.Lock()
+	defer s.bemu.Unlock()
+	return s.be
+}
+
+func (s *EtcdServer) AuthStore() auth.AuthStore { return s.authStore }
+
+func (s *EtcdServer) restoreAlarms() error {
+	s.applyV3 = s.newApplierV3()
+	as, err := v3alarm.NewAlarmStore(s)
+	if err != nil {
+		return err
+	}
+	s.alarmStore = as
+	if len(as.Get(pb.AlarmType_NOSPACE)) > 0 {
+		s.applyV3 = newApplierV3Capped(s.applyV3)
+	}
+	if len(as.Get(pb.AlarmType_CORRUPT)) > 0 {
+		s.applyV3 = newApplierV3Corrupt(s.applyV3)
+	}
+	return nil
+}
+
+// goAttach creates a goroutine on a given function and tracks it using
+// the etcdserver waitgroup.
+func (s *EtcdServer) goAttach(f func()) {
+	s.wgMu.RLock() // this blocks with ongoing close(s.stopping)
+	defer s.wgMu.RUnlock()
+	select {
+	case <-s.stopping:
+		if lg := s.getLogger(); lg != nil {
+			lg.Warn("server has stopped; skipping goAttach")
+		} else {
+			plog.Warning("server has stopped (skipping goAttach)")
+		}
+		return
+	default:
+	}
+
+	// now safe to add since waitgroup wait has not started yet
+	s.wg.Add(1)
+	go func() {
+		defer s.wg.Done()
+		f()
+	}()
+}
+
+func (s *EtcdServer) Alarms() []*pb.AlarmMember {
+	return s.alarmStore.Get(pb.AlarmType_NONE)
+}
+
+func (s *EtcdServer) Logger() *zap.Logger {
+	return s.lg
+}
+
+// IsLearner returns if the local member is raft learner
+func (s *EtcdServer) IsLearner() bool {
+	return s.cluster.IsLocalMemberLearner()
+}
+
+// IsMemberExist returns if the member with the given id exists in cluster.
+func (s *EtcdServer) IsMemberExist(id types.ID) bool {
+	return s.cluster.IsMemberExist(id)
+}
+
+// raftStatus returns the raft status of this etcd node.
+func (s *EtcdServer) raftStatus() raft.Status {
+	return s.r.Node.Status()
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/server_access_control.go b/vendor/go.etcd.io/etcd/etcdserver/server_access_control.go
new file mode 100644
index 0000000..09e2255
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/server_access_control.go
@@ -0,0 +1,65 @@
+// Copyright 2018 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import "sync"
+
+// AccessController controls etcd server HTTP request access.
+type AccessController struct {
+	corsMu          sync.RWMutex
+	CORS            map[string]struct{}
+	hostWhitelistMu sync.RWMutex
+	HostWhitelist   map[string]struct{}
+}
+
+// NewAccessController returns a new "AccessController" with default "*" values.
+func NewAccessController() *AccessController {
+	return &AccessController{
+		CORS:          map[string]struct{}{"*": {}},
+		HostWhitelist: map[string]struct{}{"*": {}},
+	}
+}
+
+// OriginAllowed determines whether the server will allow a given CORS origin.
+// If CORS is empty, allow all.
+func (ac *AccessController) OriginAllowed(origin string) bool {
+	ac.corsMu.RLock()
+	defer ac.corsMu.RUnlock()
+	if len(ac.CORS) == 0 { // allow all
+		return true
+	}
+	_, ok := ac.CORS["*"]
+	if ok {
+		return true
+	}
+	_, ok = ac.CORS[origin]
+	return ok
+}
+
+// IsHostWhitelisted returns true if the host is whitelisted.
+// If whitelist is empty, allow all.
+func (ac *AccessController) IsHostWhitelisted(host string) bool {
+	ac.hostWhitelistMu.RLock()
+	defer ac.hostWhitelistMu.RUnlock()
+	if len(ac.HostWhitelist) == 0 { // allow all
+		return true
+	}
+	_, ok := ac.HostWhitelist["*"]
+	if ok {
+		return true
+	}
+	_, ok = ac.HostWhitelist[host]
+	return ok
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/snapshot_merge.go b/vendor/go.etcd.io/etcd/etcdserver/snapshot_merge.go
new file mode 100644
index 0000000..4177768
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/snapshot_merge.go
@@ -0,0 +1,100 @@
+// Copyright 2015 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import (
+	"io"
+
+	"go.etcd.io/etcd/etcdserver/api/snap"
+	"go.etcd.io/etcd/mvcc/backend"
+	"go.etcd.io/etcd/raft/raftpb"
+
+	humanize "github.com/dustin/go-humanize"
+	"go.uber.org/zap"
+)
+
+// createMergedSnapshotMessage creates a snapshot message that contains: raft status (term, conf),
+// a snapshot of v2 store inside raft.Snapshot as []byte, a snapshot of v3 KV in the top level message
+// as ReadCloser.
+func (s *EtcdServer) createMergedSnapshotMessage(m raftpb.Message, snapt, snapi uint64, confState raftpb.ConfState) snap.Message {
+	// get a snapshot of v2 store as []byte
+	clone := s.v2store.Clone()
+	d, err := clone.SaveNoCopy()
+	if err != nil {
+		if lg := s.getLogger(); lg != nil {
+			lg.Panic("failed to save v2 store data", zap.Error(err))
+		} else {
+			plog.Panicf("store save should never fail: %v", err)
+		}
+	}
+
+	// commit kv to write metadata(for example: consistent index).
+	s.KV().Commit()
+	dbsnap := s.be.Snapshot()
+	// get a snapshot of v3 KV as readCloser
+	rc := newSnapshotReaderCloser(s.getLogger(), dbsnap)
+
+	// put the []byte snapshot of store into raft snapshot and return the merged snapshot with
+	// KV readCloser snapshot.
+	snapshot := raftpb.Snapshot{
+		Metadata: raftpb.SnapshotMetadata{
+			Index:     snapi,
+			Term:      snapt,
+			ConfState: confState,
+		},
+		Data: d,
+	}
+	m.Snapshot = snapshot
+
+	return *snap.NewMessage(m, rc, dbsnap.Size())
+}
+
+func newSnapshotReaderCloser(lg *zap.Logger, snapshot backend.Snapshot) io.ReadCloser {
+	pr, pw := io.Pipe()
+	go func() {
+		n, err := snapshot.WriteTo(pw)
+		if err == nil {
+			if lg != nil {
+				lg.Info(
+					"sent database snapshot to writer",
+					zap.Int64("bytes", n),
+					zap.String("size", humanize.Bytes(uint64(n))),
+				)
+			} else {
+				plog.Infof("wrote database snapshot out [total bytes: %d]", n)
+			}
+		} else {
+			if lg != nil {
+				lg.Warn(
+					"failed to send database snapshot to writer",
+					zap.String("size", humanize.Bytes(uint64(n))),
+					zap.Error(err),
+				)
+			} else {
+				plog.Warningf("failed to write database snapshot out [written bytes: %d]: %v", n, err)
+			}
+		}
+		pw.CloseWithError(err)
+		err = snapshot.Close()
+		if err != nil {
+			if lg != nil {
+				lg.Panic("failed to close database snapshot", zap.Error(err))
+			} else {
+				plog.Panicf("failed to close database snapshot: %v", err)
+			}
+		}
+	}()
+	return pr
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/storage.go b/vendor/go.etcd.io/etcd/etcdserver/storage.go
new file mode 100644
index 0000000..d57b6f9
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/storage.go
@@ -0,0 +1,116 @@
+// Copyright 2015 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import (
+	"io"
+
+	"go.etcd.io/etcd/etcdserver/api/snap"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/pkg/pbutil"
+	"go.etcd.io/etcd/pkg/types"
+	"go.etcd.io/etcd/raft/raftpb"
+	"go.etcd.io/etcd/wal"
+	"go.etcd.io/etcd/wal/walpb"
+
+	"go.uber.org/zap"
+)
+
+type Storage interface {
+	// Save function saves ents and state to the underlying stable storage.
+	// Save MUST block until st and ents are on stable storage.
+	Save(st raftpb.HardState, ents []raftpb.Entry) error
+	// SaveSnap function saves snapshot to the underlying stable storage.
+	SaveSnap(snap raftpb.Snapshot) error
+	// Close closes the Storage and performs finalization.
+	Close() error
+}
+
+type storage struct {
+	*wal.WAL
+	*snap.Snapshotter
+}
+
+func NewStorage(w *wal.WAL, s *snap.Snapshotter) Storage {
+	return &storage{w, s}
+}
+
+// SaveSnap saves the snapshot to disk and release the locked
+// wal files since they will not be used.
+func (st *storage) SaveSnap(snap raftpb.Snapshot) error {
+	walsnap := walpb.Snapshot{
+		Index: snap.Metadata.Index,
+		Term:  snap.Metadata.Term,
+	}
+	err := st.WAL.SaveSnapshot(walsnap)
+	if err != nil {
+		return err
+	}
+	err = st.Snapshotter.SaveSnap(snap)
+	if err != nil {
+		return err
+	}
+	return st.WAL.ReleaseLockTo(snap.Metadata.Index)
+}
+
+func readWAL(lg *zap.Logger, waldir string, snap walpb.Snapshot) (w *wal.WAL, id, cid types.ID, st raftpb.HardState, ents []raftpb.Entry) {
+	var (
+		err       error
+		wmetadata []byte
+	)
+
+	repaired := false
+	for {
+		if w, err = wal.Open(lg, waldir, snap); err != nil {
+			if lg != nil {
+				lg.Fatal("failed to open WAL", zap.Error(err))
+			} else {
+				plog.Fatalf("open wal error: %v", err)
+			}
+		}
+		if wmetadata, st, ents, err = w.ReadAll(); err != nil {
+			w.Close()
+			// we can only repair ErrUnexpectedEOF and we never repair twice.
+			if repaired || err != io.ErrUnexpectedEOF {
+				if lg != nil {
+					lg.Fatal("failed to read WAL, cannot be repaired", zap.Error(err))
+				} else {
+					plog.Fatalf("read wal error (%v) and cannot be repaired", err)
+				}
+			}
+			if !wal.Repair(lg, waldir) {
+				if lg != nil {
+					lg.Fatal("failed to repair WAL", zap.Error(err))
+				} else {
+					plog.Fatalf("WAL error (%v) cannot be repaired", err)
+				}
+			} else {
+				if lg != nil {
+					lg.Info("repaired WAL", zap.Error(err))
+				} else {
+					plog.Infof("repaired WAL error (%v)", err)
+				}
+				repaired = true
+			}
+			continue
+		}
+		break
+	}
+	var metadata pb.Metadata
+	pbutil.MustUnmarshal(&metadata, wmetadata)
+	id = types.ID(metadata.NodeID)
+	cid = types.ID(metadata.ClusterID)
+	return w, id, cid, st, ents
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/util.go b/vendor/go.etcd.io/etcd/etcdserver/util.go
new file mode 100644
index 0000000..fe5024e
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/util.go
@@ -0,0 +1,168 @@
+// Copyright 2015 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import (
+	"fmt"
+	"reflect"
+	"strings"
+	"time"
+
+	"github.com/golang/protobuf/proto"
+	"go.etcd.io/etcd/etcdserver/api/membership"
+	"go.etcd.io/etcd/etcdserver/api/rafthttp"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/pkg/types"
+
+	"go.uber.org/zap"
+)
+
+// isConnectedToQuorumSince checks whether the local member is connected to the
+// quorum of the cluster since the given time.
+func isConnectedToQuorumSince(transport rafthttp.Transporter, since time.Time, self types.ID, members []*membership.Member) bool {
+	return numConnectedSince(transport, since, self, members) >= (len(members)/2)+1
+}
+
+// isConnectedSince checks whether the local member is connected to the
+// remote member since the given time.
+func isConnectedSince(transport rafthttp.Transporter, since time.Time, remote types.ID) bool {
+	t := transport.ActiveSince(remote)
+	return !t.IsZero() && t.Before(since)
+}
+
+// isConnectedFullySince checks whether the local member is connected to all
+// members in the cluster since the given time.
+func isConnectedFullySince(transport rafthttp.Transporter, since time.Time, self types.ID, members []*membership.Member) bool {
+	return numConnectedSince(transport, since, self, members) == len(members)
+}
+
+// numConnectedSince counts how many members are connected to the local member
+// since the given time.
+func numConnectedSince(transport rafthttp.Transporter, since time.Time, self types.ID, members []*membership.Member) int {
+	connectedNum := 0
+	for _, m := range members {
+		if m.ID == self || isConnectedSince(transport, since, m.ID) {
+			connectedNum++
+		}
+	}
+	return connectedNum
+}
+
+// longestConnected chooses the member with longest active-since-time.
+// It returns false, if nothing is active.
+func longestConnected(tp rafthttp.Transporter, membs []types.ID) (types.ID, bool) {
+	var longest types.ID
+	var oldest time.Time
+	for _, id := range membs {
+		tm := tp.ActiveSince(id)
+		if tm.IsZero() { // inactive
+			continue
+		}
+
+		if oldest.IsZero() { // first longest candidate
+			oldest = tm
+			longest = id
+		}
+
+		if tm.Before(oldest) {
+			oldest = tm
+			longest = id
+		}
+	}
+	if uint64(longest) == 0 {
+		return longest, false
+	}
+	return longest, true
+}
+
+type notifier struct {
+	c   chan struct{}
+	err error
+}
+
+func newNotifier() *notifier {
+	return &notifier{
+		c: make(chan struct{}),
+	}
+}
+
+func (nc *notifier) notify(err error) {
+	nc.err = err
+	close(nc.c)
+}
+
+func warnOfExpensiveRequest(lg *zap.Logger, now time.Time, reqStringer fmt.Stringer, respMsg proto.Message, err error) {
+	var resp string
+	if !isNil(respMsg) {
+		resp = fmt.Sprintf("size:%d", proto.Size(respMsg))
+	}
+	warnOfExpensiveGenericRequest(lg, now, reqStringer, "", resp, err)
+}
+
+func warnOfExpensiveReadOnlyTxnRequest(lg *zap.Logger, now time.Time, r *pb.TxnRequest, txnResponse *pb.TxnResponse, err error) {
+	reqStringer := pb.NewLoggableTxnRequest(r)
+	var resp string
+	if !isNil(txnResponse) {
+		var resps []string
+		for _, r := range txnResponse.Responses {
+			switch op := r.Response.(type) {
+			case *pb.ResponseOp_ResponseRange:
+				resps = append(resps, fmt.Sprintf("range_response_count:%d", len(op.ResponseRange.Kvs)))
+			default:
+				// only range responses should be in a read only txn request
+			}
+		}
+		resp = fmt.Sprintf("responses:<%s> size:%d", strings.Join(resps, " "), proto.Size(txnResponse))
+	}
+	warnOfExpensiveGenericRequest(lg, now, reqStringer, "read-only range ", resp, err)
+}
+
+func warnOfExpensiveReadOnlyRangeRequest(lg *zap.Logger, now time.Time, reqStringer fmt.Stringer, rangeResponse *pb.RangeResponse, err error) {
+	var resp string
+	if !isNil(rangeResponse) {
+		resp = fmt.Sprintf("range_response_count:%d size:%d", len(rangeResponse.Kvs), proto.Size(rangeResponse))
+	}
+	warnOfExpensiveGenericRequest(lg, now, reqStringer, "read-only range ", resp, err)
+}
+
+func warnOfExpensiveGenericRequest(lg *zap.Logger, now time.Time, reqStringer fmt.Stringer, prefix string, resp string, err error) {
+	d := time.Since(now)
+	if d > warnApplyDuration {
+		if lg != nil {
+			lg.Warn(
+				"apply request took too long",
+				zap.Duration("took", d),
+				zap.Duration("expected-duration", warnApplyDuration),
+				zap.String("prefix", prefix),
+				zap.String("request", reqStringer.String()),
+				zap.String("response", resp),
+				zap.Error(err),
+			)
+		} else {
+			var result string
+			if err != nil {
+				result = fmt.Sprintf("error:%v", err)
+			} else {
+				result = resp
+			}
+			plog.Warningf("%srequest %q with result %q took too long (%v) to execute", prefix, reqStringer.String(), result, d)
+		}
+		slowApplies.Inc()
+	}
+}
+
+func isNil(msg proto.Message) bool {
+	return msg == nil || reflect.ValueOf(msg).IsNil()
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/v2_server.go b/vendor/go.etcd.io/etcd/etcdserver/v2_server.go
new file mode 100644
index 0000000..9238b2d
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/v2_server.go
@@ -0,0 +1,165 @@
+// Copyright 2016 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import (
+	"context"
+	"time"
+
+	"go.etcd.io/etcd/etcdserver/api/v2store"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+)
+
+type RequestV2 pb.Request
+
+type RequestV2Handler interface {
+	Post(ctx context.Context, r *RequestV2) (Response, error)
+	Put(ctx context.Context, r *RequestV2) (Response, error)
+	Delete(ctx context.Context, r *RequestV2) (Response, error)
+	QGet(ctx context.Context, r *RequestV2) (Response, error)
+	Get(ctx context.Context, r *RequestV2) (Response, error)
+	Head(ctx context.Context, r *RequestV2) (Response, error)
+}
+
+type reqV2HandlerEtcdServer struct {
+	reqV2HandlerStore
+	s *EtcdServer
+}
+
+type reqV2HandlerStore struct {
+	store   v2store.Store
+	applier ApplierV2
+}
+
+func NewStoreRequestV2Handler(s v2store.Store, applier ApplierV2) RequestV2Handler {
+	return &reqV2HandlerStore{s, applier}
+}
+
+func (a *reqV2HandlerStore) Post(ctx context.Context, r *RequestV2) (Response, error) {
+	return a.applier.Post(r), nil
+}
+
+func (a *reqV2HandlerStore) Put(ctx context.Context, r *RequestV2) (Response, error) {
+	return a.applier.Put(r), nil
+}
+
+func (a *reqV2HandlerStore) Delete(ctx context.Context, r *RequestV2) (Response, error) {
+	return a.applier.Delete(r), nil
+}
+
+func (a *reqV2HandlerStore) QGet(ctx context.Context, r *RequestV2) (Response, error) {
+	return a.applier.QGet(r), nil
+}
+
+func (a *reqV2HandlerStore) Get(ctx context.Context, r *RequestV2) (Response, error) {
+	if r.Wait {
+		wc, err := a.store.Watch(r.Path, r.Recursive, r.Stream, r.Since)
+		return Response{Watcher: wc}, err
+	}
+	ev, err := a.store.Get(r.Path, r.Recursive, r.Sorted)
+	return Response{Event: ev}, err
+}
+
+func (a *reqV2HandlerStore) Head(ctx context.Context, r *RequestV2) (Response, error) {
+	ev, err := a.store.Get(r.Path, r.Recursive, r.Sorted)
+	return Response{Event: ev}, err
+}
+
+func (a *reqV2HandlerEtcdServer) Post(ctx context.Context, r *RequestV2) (Response, error) {
+	return a.processRaftRequest(ctx, r)
+}
+
+func (a *reqV2HandlerEtcdServer) Put(ctx context.Context, r *RequestV2) (Response, error) {
+	return a.processRaftRequest(ctx, r)
+}
+
+func (a *reqV2HandlerEtcdServer) Delete(ctx context.Context, r *RequestV2) (Response, error) {
+	return a.processRaftRequest(ctx, r)
+}
+
+func (a *reqV2HandlerEtcdServer) QGet(ctx context.Context, r *RequestV2) (Response, error) {
+	return a.processRaftRequest(ctx, r)
+}
+
+func (a *reqV2HandlerEtcdServer) processRaftRequest(ctx context.Context, r *RequestV2) (Response, error) {
+	data, err := ((*pb.Request)(r)).Marshal()
+	if err != nil {
+		return Response{}, err
+	}
+	ch := a.s.w.Register(r.ID)
+
+	start := time.Now()
+	a.s.r.Propose(ctx, data)
+	proposalsPending.Inc()
+	defer proposalsPending.Dec()
+
+	select {
+	case x := <-ch:
+		resp := x.(Response)
+		return resp, resp.Err
+	case <-ctx.Done():
+		proposalsFailed.Inc()
+		a.s.w.Trigger(r.ID, nil) // GC wait
+		return Response{}, a.s.parseProposeCtxErr(ctx.Err(), start)
+	case <-a.s.stopping:
+	}
+	return Response{}, ErrStopped
+}
+
+func (s *EtcdServer) Do(ctx context.Context, r pb.Request) (Response, error) {
+	r.ID = s.reqIDGen.Next()
+	h := &reqV2HandlerEtcdServer{
+		reqV2HandlerStore: reqV2HandlerStore{
+			store:   s.v2store,
+			applier: s.applyV2,
+		},
+		s: s,
+	}
+	rp := &r
+	resp, err := ((*RequestV2)(rp)).Handle(ctx, h)
+	resp.Term, resp.Index = s.Term(), s.CommittedIndex()
+	return resp, err
+}
+
+// Handle interprets r and performs an operation on s.store according to r.Method
+// and other fields. If r.Method is "POST", "PUT", "DELETE", or a "GET" with
+// Quorum == true, r will be sent through consensus before performing its
+// respective operation. Do will block until an action is performed or there is
+// an error.
+func (r *RequestV2) Handle(ctx context.Context, v2api RequestV2Handler) (Response, error) {
+	if r.Method == "GET" && r.Quorum {
+		r.Method = "QGET"
+	}
+	switch r.Method {
+	case "POST":
+		return v2api.Post(ctx, r)
+	case "PUT":
+		return v2api.Put(ctx, r)
+	case "DELETE":
+		return v2api.Delete(ctx, r)
+	case "QGET":
+		return v2api.QGet(ctx, r)
+	case "GET":
+		return v2api.Get(ctx, r)
+	case "HEAD":
+		return v2api.Head(ctx, r)
+	}
+	return Response{}, ErrUnknownMethod
+}
+
+func (r *RequestV2) String() string {
+	rpb := pb.Request(*r)
+	return rpb.String()
+}
diff --git a/vendor/go.etcd.io/etcd/etcdserver/v3_server.go b/vendor/go.etcd.io/etcd/etcdserver/v3_server.go
new file mode 100644
index 0000000..b208461
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/etcdserver/v3_server.go
@@ -0,0 +1,768 @@
+// Copyright 2015 The etcd Authors
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package etcdserver
+
+import (
+	"bytes"
+	"context"
+	"encoding/binary"
+	"time"
+
+	"go.etcd.io/etcd/auth"
+	"go.etcd.io/etcd/etcdserver/api/membership"
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/lease"
+	"go.etcd.io/etcd/lease/leasehttp"
+	"go.etcd.io/etcd/mvcc"
+	"go.etcd.io/etcd/raft"
+
+	"github.com/gogo/protobuf/proto"
+	"go.uber.org/zap"
+)
+
+const (
+	// In the health case, there might be a small gap (10s of entries) between
+	// the applied index and committed index.
+	// However, if the committed entries are very heavy to apply, the gap might grow.
+	// We should stop accepting new proposals if the gap growing to a certain point.
+	maxGapBetweenApplyAndCommitIndex = 5000
+)
+
+type RaftKV interface {
+	Range(ctx context.Context, r *pb.RangeRequest) (*pb.RangeResponse, error)
+	Put(ctx context.Context, r *pb.PutRequest) (*pb.PutResponse, error)
+	DeleteRange(ctx context.Context, r *pb.DeleteRangeRequest) (*pb.DeleteRangeResponse, error)
+	Txn(ctx context.Context, r *pb.TxnRequest) (*pb.TxnResponse, error)
+	Compact(ctx context.Context, r *pb.CompactionRequest) (*pb.CompactionResponse, error)
+}
+
+type Lessor interface {
+	// LeaseGrant sends LeaseGrant request to raft and apply it after committed.
+	LeaseGrant(ctx context.Context, r *pb.LeaseGrantRequest) (*pb.LeaseGrantResponse, error)
+	// LeaseRevoke sends LeaseRevoke request to raft and apply it after committed.
+	LeaseRevoke(ctx context.Context, r *pb.LeaseRevokeRequest) (*pb.LeaseRevokeResponse, error)
+
+	// LeaseRenew renews the lease with given ID. The renewed TTL is returned. Or an error
+	// is returned.
+	LeaseRenew(ctx context.Context, id lease.LeaseID) (int64, error)
+
+	// LeaseTimeToLive retrieves lease information.
+	LeaseTimeToLive(ctx context.Context, r *pb.LeaseTimeToLiveRequest) (*pb.LeaseTimeToLiveResponse, error)
+
+	// LeaseLeases lists all leases.
+	LeaseLeases(ctx context.Context, r *pb.LeaseLeasesRequest) (*pb.LeaseLeasesResponse, error)
+}
+
+type Authenticator interface {
+	AuthEnable(ctx context.Context, r *pb.AuthEnableRequest) (*pb.AuthEnableResponse, error)
+	AuthDisable(ctx context.Context, r *pb.AuthDisableRequest) (*pb.AuthDisableResponse, error)
+	Authenticate(ctx context.Context, r *pb.AuthenticateRequest) (*pb.AuthenticateResponse, error)
+	UserAdd(ctx context.Context, r *pb.AuthUserAddRequest) (*pb.AuthUserAddResponse, error)
+	UserDelete(ctx context.Context, r *pb.AuthUserDeleteRequest) (*pb.AuthUserDeleteResponse, error)
+	UserChangePassword(ctx context.Context, r *pb.AuthUserChangePasswordRequest) (*pb.AuthUserChangePasswordResponse, error)
+	UserGrantRole(ctx context.Context, r *pb.AuthUserGrantRoleRequest) (*pb.AuthUserGrantRoleResponse, error)
+	UserGet(ctx context.Context, r *pb.AuthUserGetRequest) (*pb.AuthUserGetResponse, error)
+	UserRevokeRole(ctx context.Context, r *pb.AuthUserRevokeRoleRequest) (*pb.AuthUserRevokeRoleResponse, error)
+	RoleAdd(ctx context.Context, r *pb.AuthRoleAddRequest) (*pb.AuthRoleAddResponse, error)
+	RoleGrantPermission(ctx context.Context, r *pb.AuthRoleGrantPermissionRequest) (*pb.AuthRoleGrantPermissionResponse, error)
+	RoleGet(ctx context.Context, r *pb.AuthRoleGetRequest) (*pb.AuthRoleGetResponse, error)
+	RoleRevokePermission(ctx context.Context, r *pb.AuthRoleRevokePermissionRequest) (*pb.AuthRoleRevokePermissionResponse, error)
+	RoleDelete(ctx context.Context, r *pb.AuthRoleDeleteRequest) (*pb.AuthRoleDeleteResponse, error)
+	UserList(ctx context.Context, r *pb.AuthUserListRequest) (*pb.AuthUserListResponse, error)
+	RoleList(ctx context.Context, r *pb.AuthRoleListRequest) (*pb.AuthRoleListResponse, error)
+}
+
+func (s *EtcdServer) Range(ctx context.Context, r *pb.RangeRequest) (*pb.RangeResponse, error) {
+	var resp *pb.RangeResponse
+	var err error
+	defer func(start time.Time) {
+		warnOfExpensiveReadOnlyRangeRequest(s.getLogger(), start, r, resp, err)
+	}(time.Now())
+
+	if !r.Serializable {
+		err = s.linearizableReadNotify(ctx)
+		if err != nil {
+			return nil, err
+		}
+	}
+	chk := func(ai *auth.AuthInfo) error {
+		return s.authStore.IsRangePermitted(ai, r.Key, r.RangeEnd)
+	}
+
+	get := func() { resp, err = s.applyV3Base.Range(nil, r) }
+	if serr := s.doSerialize(ctx, chk, get); serr != nil {
+		err = serr
+		return nil, err
+	}
+	return resp, err
+}
+
+func (s *EtcdServer) Put(ctx context.Context, r *pb.PutRequest) (*pb.PutResponse, error) {
+	resp, err := s.raftRequest(ctx, pb.InternalRaftRequest{Put: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.PutResponse), nil
+}
+
+func (s *EtcdServer) DeleteRange(ctx context.Context, r *pb.DeleteRangeRequest) (*pb.DeleteRangeResponse, error) {
+	resp, err := s.raftRequest(ctx, pb.InternalRaftRequest{DeleteRange: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.DeleteRangeResponse), nil
+}
+
+func (s *EtcdServer) Txn(ctx context.Context, r *pb.TxnRequest) (*pb.TxnResponse, error) {
+	if isTxnReadonly(r) {
+		if !isTxnSerializable(r) {
+			err := s.linearizableReadNotify(ctx)
+			if err != nil {
+				return nil, err
+			}
+		}
+		var resp *pb.TxnResponse
+		var err error
+		chk := func(ai *auth.AuthInfo) error {
+			return checkTxnAuth(s.authStore, ai, r)
+		}
+
+		defer func(start time.Time) {
+			warnOfExpensiveReadOnlyTxnRequest(s.getLogger(), start, r, resp, err)
+		}(time.Now())
+
+		get := func() { resp, err = s.applyV3Base.Txn(r) }
+		if serr := s.doSerialize(ctx, chk, get); serr != nil {
+			return nil, serr
+		}
+		return resp, err
+	}
+
+	resp, err := s.raftRequest(ctx, pb.InternalRaftRequest{Txn: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.TxnResponse), nil
+}
+
+func isTxnSerializable(r *pb.TxnRequest) bool {
+	for _, u := range r.Success {
+		if r := u.GetRequestRange(); r == nil || !r.Serializable {
+			return false
+		}
+	}
+	for _, u := range r.Failure {
+		if r := u.GetRequestRange(); r == nil || !r.Serializable {
+			return false
+		}
+	}
+	return true
+}
+
+func isTxnReadonly(r *pb.TxnRequest) bool {
+	for _, u := range r.Success {
+		if r := u.GetRequestRange(); r == nil {
+			return false
+		}
+	}
+	for _, u := range r.Failure {
+		if r := u.GetRequestRange(); r == nil {
+			return false
+		}
+	}
+	return true
+}
+
+func (s *EtcdServer) Compact(ctx context.Context, r *pb.CompactionRequest) (*pb.CompactionResponse, error) {
+	result, err := s.processInternalRaftRequestOnce(ctx, pb.InternalRaftRequest{Compaction: r})
+	if r.Physical && result != nil && result.physc != nil {
+		<-result.physc
+		// The compaction is done deleting keys; the hash is now settled
+		// but the data is not necessarily committed. If there's a crash,
+		// the hash may revert to a hash prior to compaction completing
+		// if the compaction resumes. Force the finished compaction to
+		// commit so it won't resume following a crash.
+		s.be.ForceCommit()
+	}
+	if err != nil {
+		return nil, err
+	}
+	if result.err != nil {
+		return nil, result.err
+	}
+	resp := result.resp.(*pb.CompactionResponse)
+	if resp == nil {
+		resp = &pb.CompactionResponse{}
+	}
+	if resp.Header == nil {
+		resp.Header = &pb.ResponseHeader{}
+	}
+	resp.Header.Revision = s.kv.Rev()
+	return resp, nil
+}
+
+func (s *EtcdServer) LeaseGrant(ctx context.Context, r *pb.LeaseGrantRequest) (*pb.LeaseGrantResponse, error) {
+	// no id given? choose one
+	for r.ID == int64(lease.NoLease) {
+		// only use positive int64 id's
+		r.ID = int64(s.reqIDGen.Next() & ((1 << 63) - 1))
+	}
+	resp, err := s.raftRequestOnce(ctx, pb.InternalRaftRequest{LeaseGrant: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.LeaseGrantResponse), nil
+}
+
+func (s *EtcdServer) LeaseRevoke(ctx context.Context, r *pb.LeaseRevokeRequest) (*pb.LeaseRevokeResponse, error) {
+	resp, err := s.raftRequestOnce(ctx, pb.InternalRaftRequest{LeaseRevoke: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.LeaseRevokeResponse), nil
+}
+
+func (s *EtcdServer) LeaseRenew(ctx context.Context, id lease.LeaseID) (int64, error) {
+	ttl, err := s.lessor.Renew(id)
+	if err == nil { // already requested to primary lessor(leader)
+		return ttl, nil
+	}
+	if err != lease.ErrNotPrimary {
+		return -1, err
+	}
+
+	cctx, cancel := context.WithTimeout(ctx, s.Cfg.ReqTimeout())
+	defer cancel()
+
+	// renewals don't go through raft; forward to leader manually
+	for cctx.Err() == nil && err != nil {
+		leader, lerr := s.waitLeader(cctx)
+		if lerr != nil {
+			return -1, lerr
+		}
+		for _, url := range leader.PeerURLs {
+			lurl := url + leasehttp.LeasePrefix
+			ttl, err = leasehttp.RenewHTTP(cctx, id, lurl, s.peerRt)
+			if err == nil || err == lease.ErrLeaseNotFound {
+				return ttl, err
+			}
+		}
+	}
+
+	if cctx.Err() == context.DeadlineExceeded {
+		return -1, ErrTimeout
+	}
+	return -1, ErrCanceled
+}
+
+func (s *EtcdServer) LeaseTimeToLive(ctx context.Context, r *pb.LeaseTimeToLiveRequest) (*pb.LeaseTimeToLiveResponse, error) {
+	if s.Leader() == s.ID() {
+		// primary; timetolive directly from leader
+		le := s.lessor.Lookup(lease.LeaseID(r.ID))
+		if le == nil {
+			return nil, lease.ErrLeaseNotFound
+		}
+		// TODO: fill out ResponseHeader
+		resp := &pb.LeaseTimeToLiveResponse{Header: &pb.ResponseHeader{}, ID: r.ID, TTL: int64(le.Remaining().Seconds()), GrantedTTL: le.TTL()}
+		if r.Keys {
+			ks := le.Keys()
+			kbs := make([][]byte, len(ks))
+			for i := range ks {
+				kbs[i] = []byte(ks[i])
+			}
+			resp.Keys = kbs
+		}
+		return resp, nil
+	}
+
+	cctx, cancel := context.WithTimeout(ctx, s.Cfg.ReqTimeout())
+	defer cancel()
+
+	// forward to leader
+	for cctx.Err() == nil {
+		leader, err := s.waitLeader(cctx)
+		if err != nil {
+			return nil, err
+		}
+		for _, url := range leader.PeerURLs {
+			lurl := url + leasehttp.LeaseInternalPrefix
+			resp, err := leasehttp.TimeToLiveHTTP(cctx, lease.LeaseID(r.ID), r.Keys, lurl, s.peerRt)
+			if err == nil {
+				return resp.LeaseTimeToLiveResponse, nil
+			}
+			if err == lease.ErrLeaseNotFound {
+				return nil, err
+			}
+		}
+	}
+
+	if cctx.Err() == context.DeadlineExceeded {
+		return nil, ErrTimeout
+	}
+	return nil, ErrCanceled
+}
+
+func (s *EtcdServer) LeaseLeases(ctx context.Context, r *pb.LeaseLeasesRequest) (*pb.LeaseLeasesResponse, error) {
+	ls := s.lessor.Leases()
+	lss := make([]*pb.LeaseStatus, len(ls))
+	for i := range ls {
+		lss[i] = &pb.LeaseStatus{ID: int64(ls[i].ID)}
+	}
+	return &pb.LeaseLeasesResponse{Header: newHeader(s), Leases: lss}, nil
+}
+
+func (s *EtcdServer) waitLeader(ctx context.Context) (*membership.Member, error) {
+	leader := s.cluster.Member(s.Leader())
+	for leader == nil {
+		// wait an election
+		dur := time.Duration(s.Cfg.ElectionTicks) * time.Duration(s.Cfg.TickMs) * time.Millisecond
+		select {
+		case <-time.After(dur):
+			leader = s.cluster.Member(s.Leader())
+		case <-s.stopping:
+			return nil, ErrStopped
+		case <-ctx.Done():
+			return nil, ErrNoLeader
+		}
+	}
+	if leader == nil || len(leader.PeerURLs) == 0 {
+		return nil, ErrNoLeader
+	}
+	return leader, nil
+}
+
+func (s *EtcdServer) Alarm(ctx context.Context, r *pb.AlarmRequest) (*pb.AlarmResponse, error) {
+	resp, err := s.raftRequestOnce(ctx, pb.InternalRaftRequest{Alarm: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.AlarmResponse), nil
+}
+
+func (s *EtcdServer) AuthEnable(ctx context.Context, r *pb.AuthEnableRequest) (*pb.AuthEnableResponse, error) {
+	resp, err := s.raftRequestOnce(ctx, pb.InternalRaftRequest{AuthEnable: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.AuthEnableResponse), nil
+}
+
+func (s *EtcdServer) AuthDisable(ctx context.Context, r *pb.AuthDisableRequest) (*pb.AuthDisableResponse, error) {
+	resp, err := s.raftRequest(ctx, pb.InternalRaftRequest{AuthDisable: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.AuthDisableResponse), nil
+}
+
+func (s *EtcdServer) Authenticate(ctx context.Context, r *pb.AuthenticateRequest) (*pb.AuthenticateResponse, error) {
+	if err := s.linearizableReadNotify(ctx); err != nil {
+		return nil, err
+	}
+
+	lg := s.getLogger()
+
+	var resp proto.Message
+	for {
+		checkedRevision, err := s.AuthStore().CheckPassword(r.Name, r.Password)
+		if err != nil {
+			if err != auth.ErrAuthNotEnabled {
+				if lg != nil {
+					lg.Warn(
+						"invalid authentication was requested",
+						zap.String("user", r.Name),
+						zap.Error(err),
+					)
+				} else {
+					plog.Errorf("invalid authentication request to user %s was issued", r.Name)
+				}
+			}
+			return nil, err
+		}
+
+		st, err := s.AuthStore().GenTokenPrefix()
+		if err != nil {
+			return nil, err
+		}
+
+		internalReq := &pb.InternalAuthenticateRequest{
+			Name:        r.Name,
+			Password:    r.Password,
+			SimpleToken: st,
+		}
+
+		resp, err = s.raftRequestOnce(ctx, pb.InternalRaftRequest{Authenticate: internalReq})
+		if err != nil {
+			return nil, err
+		}
+		if checkedRevision == s.AuthStore().Revision() {
+			break
+		}
+
+		if lg != nil {
+			lg.Info("revision when password checked became stale; retrying")
+		} else {
+			plog.Infof("revision when password checked is obsolete, retrying")
+		}
+	}
+
+	return resp.(*pb.AuthenticateResponse), nil
+}
+
+func (s *EtcdServer) UserAdd(ctx context.Context, r *pb.AuthUserAddRequest) (*pb.AuthUserAddResponse, error) {
+	resp, err := s.raftRequest(ctx, pb.InternalRaftRequest{AuthUserAdd: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.AuthUserAddResponse), nil
+}
+
+func (s *EtcdServer) UserDelete(ctx context.Context, r *pb.AuthUserDeleteRequest) (*pb.AuthUserDeleteResponse, error) {
+	resp, err := s.raftRequest(ctx, pb.InternalRaftRequest{AuthUserDelete: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.AuthUserDeleteResponse), nil
+}
+
+func (s *EtcdServer) UserChangePassword(ctx context.Context, r *pb.AuthUserChangePasswordRequest) (*pb.AuthUserChangePasswordResponse, error) {
+	resp, err := s.raftRequest(ctx, pb.InternalRaftRequest{AuthUserChangePassword: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.AuthUserChangePasswordResponse), nil
+}
+
+func (s *EtcdServer) UserGrantRole(ctx context.Context, r *pb.AuthUserGrantRoleRequest) (*pb.AuthUserGrantRoleResponse, error) {
+	resp, err := s.raftRequest(ctx, pb.InternalRaftRequest{AuthUserGrantRole: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.AuthUserGrantRoleResponse), nil
+}
+
+func (s *EtcdServer) UserGet(ctx context.Context, r *pb.AuthUserGetRequest) (*pb.AuthUserGetResponse, error) {
+	resp, err := s.raftRequest(ctx, pb.InternalRaftRequest{AuthUserGet: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.AuthUserGetResponse), nil
+}
+
+func (s *EtcdServer) UserList(ctx context.Context, r *pb.AuthUserListRequest) (*pb.AuthUserListResponse, error) {
+	resp, err := s.raftRequest(ctx, pb.InternalRaftRequest{AuthUserList: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.AuthUserListResponse), nil
+}
+
+func (s *EtcdServer) UserRevokeRole(ctx context.Context, r *pb.AuthUserRevokeRoleRequest) (*pb.AuthUserRevokeRoleResponse, error) {
+	resp, err := s.raftRequest(ctx, pb.InternalRaftRequest{AuthUserRevokeRole: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.AuthUserRevokeRoleResponse), nil
+}
+
+func (s *EtcdServer) RoleAdd(ctx context.Context, r *pb.AuthRoleAddRequest) (*pb.AuthRoleAddResponse, error) {
+	resp, err := s.raftRequest(ctx, pb.InternalRaftRequest{AuthRoleAdd: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.AuthRoleAddResponse), nil
+}
+
+func (s *EtcdServer) RoleGrantPermission(ctx context.Context, r *pb.AuthRoleGrantPermissionRequest) (*pb.AuthRoleGrantPermissionResponse, error) {
+	resp, err := s.raftRequest(ctx, pb.InternalRaftRequest{AuthRoleGrantPermission: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.AuthRoleGrantPermissionResponse), nil
+}
+
+func (s *EtcdServer) RoleGet(ctx context.Context, r *pb.AuthRoleGetRequest) (*pb.AuthRoleGetResponse, error) {
+	resp, err := s.raftRequest(ctx, pb.InternalRaftRequest{AuthRoleGet: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.AuthRoleGetResponse), nil
+}
+
+func (s *EtcdServer) RoleList(ctx context.Context, r *pb.AuthRoleListRequest) (*pb.AuthRoleListResponse, error) {
+	resp, err := s.raftRequest(ctx, pb.InternalRaftRequest{AuthRoleList: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.AuthRoleListResponse), nil
+}
+
+func (s *EtcdServer) RoleRevokePermission(ctx context.Context, r *pb.AuthRoleRevokePermissionRequest) (*pb.AuthRoleRevokePermissionResponse, error) {
+	resp, err := s.raftRequest(ctx, pb.InternalRaftRequest{AuthRoleRevokePermission: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.AuthRoleRevokePermissionResponse), nil
+}
+
+func (s *EtcdServer) RoleDelete(ctx context.Context, r *pb.AuthRoleDeleteRequest) (*pb.AuthRoleDeleteResponse, error) {
+	resp, err := s.raftRequest(ctx, pb.InternalRaftRequest{AuthRoleDelete: r})
+	if err != nil {
+		return nil, err
+	}
+	return resp.(*pb.AuthRoleDeleteResponse), nil
+}
+
+func (s *EtcdServer) raftRequestOnce(ctx context.Context, r pb.InternalRaftRequest) (proto.Message, error) {
+	result, err := s.processInternalRaftRequestOnce(ctx, r)
+	if err != nil {
+		return nil, err
+	}
+	if result.err != nil {
+		return nil, result.err
+	}
+	return result.resp, nil
+}
+
+func (s *EtcdServer) raftRequest(ctx context.Context, r pb.InternalRaftRequest) (proto.Message, error) {
+	for {
+		resp, err := s.raftRequestOnce(ctx, r)
+		if err != auth.ErrAuthOldRevision {
+			return resp, err
+		}
+	}
+}
+
+// doSerialize handles the auth logic, with permissions checked by "chk", for a serialized request "get". Returns a non-nil error on authentication failure.
+func (s *EtcdServer) doSerialize(ctx context.Context, chk func(*auth.AuthInfo) error, get func()) error {
+	ai, err := s.AuthInfoFromCtx(ctx)
+	if err != nil {
+		return err
+	}
+	if ai == nil {
+		// chk expects non-nil AuthInfo; use empty credentials
+		ai = &auth.AuthInfo{}
+	}
+	if err = chk(ai); err != nil {
+		return err
+	}
+	// fetch response for serialized request
+	get()
+	// check for stale token revision in case the auth store was updated while
+	// the request has been handled.
+	if ai.Revision != 0 && ai.Revision != s.authStore.Revision() {
+		return auth.ErrAuthOldRevision
+	}
+	return nil
+}
+
+func (s *EtcdServer) processInternalRaftRequestOnce(ctx context.Context, r pb.InternalRaftRequest) (*applyResult, error) {
+	ai := s.getAppliedIndex()
+	ci := s.getCommittedIndex()
+	if ci > ai+maxGapBetweenApplyAndCommitIndex {
+		return nil, ErrTooManyRequests
+	}
+
+	r.Header = &pb.RequestHeader{
+		ID: s.reqIDGen.Next(),
+	}
+
+	authInfo, err := s.AuthInfoFromCtx(ctx)
+	if err != nil {
+		return nil, err
+	}
+	if authInfo != nil {
+		r.Header.Username = authInfo.Username
+		r.Header.AuthRevision = authInfo.Revision
+	}
+
+	data, err := r.Marshal()
+	if err != nil {
+		return nil, err
+	}
+
+	if len(data) > int(s.Cfg.MaxRequestBytes) {
+		return nil, ErrRequestTooLarge
+	}
+
+	id := r.ID
+	if id == 0 {
+		id = r.Header.ID
+	}
+	ch := s.w.Register(id)
+
+	cctx, cancel := context.WithTimeout(ctx, s.Cfg.ReqTimeout())
+	defer cancel()
+
+	start := time.Now()
+	err = s.r.Propose(cctx, data)
+	if err != nil {
+		proposalsFailed.Inc()
+		s.w.Trigger(id, nil) // GC wait
+		return nil, err
+	}
+	proposalsPending.Inc()
+	defer proposalsPending.Dec()
+
+	select {
+	case x := <-ch:
+		return x.(*applyResult), nil
+	case <-cctx.Done():
+		proposalsFailed.Inc()
+		s.w.Trigger(id, nil) // GC wait
+		return nil, s.parseProposeCtxErr(cctx.Err(), start)
+	case <-s.done:
+		return nil, ErrStopped
+	}
+}
+
+// Watchable returns a watchable interface attached to the etcdserver.
+func (s *EtcdServer) Watchable() mvcc.WatchableKV { return s.KV() }
+
+func (s *EtcdServer) linearizableReadLoop() {
+	var rs raft.ReadState
+
+	for {
+		ctxToSend := make([]byte, 8)
+		id1 := s.reqIDGen.Next()
+		binary.BigEndian.PutUint64(ctxToSend, id1)
+		leaderChangedNotifier := s.leaderChangedNotify()
+		select {
+		case <-leaderChangedNotifier:
+			continue
+		case <-s.readwaitc:
+		case <-s.stopping:
+			return
+		}
+
+		nextnr := newNotifier()
+
+		s.readMu.Lock()
+		nr := s.readNotifier
+		s.readNotifier = nextnr
+		s.readMu.Unlock()
+
+		lg := s.getLogger()
+		cctx, cancel := context.WithTimeout(context.Background(), s.Cfg.ReqTimeout())
+		if err := s.r.ReadIndex(cctx, ctxToSend); err != nil {
+			cancel()
+			if err == raft.ErrStopped {
+				return
+			}
+			if lg != nil {
+				lg.Warn("failed to get read index from Raft", zap.Error(err))
+			} else {
+				plog.Errorf("failed to get read index from raft: %v", err)
+			}
+			readIndexFailed.Inc()
+			nr.notify(err)
+			continue
+		}
+		cancel()
+
+		var (
+			timeout bool
+			done    bool
+		)
+		for !timeout && !done {
+			select {
+			case rs = <-s.r.readStateC:
+				done = bytes.Equal(rs.RequestCtx, ctxToSend)
+				if !done {
+					// a previous request might time out. now we should ignore the response of it and
+					// continue waiting for the response of the current requests.
+					id2 := uint64(0)
+					if len(rs.RequestCtx) == 8 {
+						id2 = binary.BigEndian.Uint64(rs.RequestCtx)
+					}
+					if lg != nil {
+						lg.Warn(
+							"ignored out-of-date read index response; local node read indexes queueing up and waiting to be in sync with leader",
+							zap.Uint64("sent-request-id", id1),
+							zap.Uint64("received-request-id", id2),
+						)
+					} else {
+						plog.Warningf("ignored out-of-date read index response; local node read indexes queueing up and waiting to be in sync with leader (request ID want %d, got %d)", id1, id2)
+					}
+					slowReadIndex.Inc()
+				}
+			case <-leaderChangedNotifier:
+				timeout = true
+				readIndexFailed.Inc()
+				// return a retryable error.
+				nr.notify(ErrLeaderChanged)
+			case <-time.After(s.Cfg.ReqTimeout()):
+				if lg != nil {
+					lg.Warn("timed out waiting for read index response (local node might have slow network)", zap.Duration("timeout", s.Cfg.ReqTimeout()))
+				} else {
+					plog.Warningf("timed out waiting for read index response (local node might have slow network)")
+				}
+				nr.notify(ErrTimeout)
+				timeout = true
+				slowReadIndex.Inc()
+			case <-s.stopping:
+				return
+			}
+		}
+		if !done {
+			continue
+		}
+
+		if ai := s.getAppliedIndex(); ai < rs.Index {
+			select {
+			case <-s.applyWait.Wait(rs.Index):
+			case <-s.stopping:
+				return
+			}
+		}
+		// unblock all l-reads requested at indices before rs.Index
+		nr.notify(nil)
+	}
+}
+
+func (s *EtcdServer) linearizableReadNotify(ctx context.Context) error {
+	s.readMu.RLock()
+	nc := s.readNotifier
+	s.readMu.RUnlock()
+
+	// signal linearizable loop for current notify if it hasn't been already
+	select {
+	case s.readwaitc <- struct{}{}:
+	default:
+	}
+
+	// wait for read state notification
+	select {
+	case <-nc.c:
+		return nc.err
+	case <-ctx.Done():
+		return ctx.Err()
+	case <-s.done:
+		return ErrStopped
+	}
+}
+
+func (s *EtcdServer) AuthInfoFromCtx(ctx context.Context) (*auth.AuthInfo, error) {
+	authInfo, err := s.AuthStore().AuthInfoFromCtx(ctx)
+	if authInfo != nil || err != nil {
+		return authInfo, err
+	}
+	if !s.Cfg.ClientCertAuthEnabled {
+		return nil, nil
+	}
+	authInfo = s.AuthStore().AuthInfoFromTLS(ctx)
+	return authInfo, nil
+
+}
diff --git a/vendor/go.etcd.io/etcd/lease/doc.go b/vendor/go.etcd.io/etcd/lease/doc.go
new file mode 100644
index 0000000..a74eaf7
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/lease/doc.go
@@ -0,0 +1,16 @@
+// Copyright 2016 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 lease provides an interface and implementation for time-limited leases over arbitrary resources.
+package lease
diff --git a/vendor/go.etcd.io/etcd/lease/lease_queue.go b/vendor/go.etcd.io/etcd/lease/lease_queue.go
new file mode 100644
index 0000000..17ddb35
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/lease/lease_queue.go
@@ -0,0 +1,106 @@
+// Copyright 2018 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 lease
+
+import "container/heap"
+
+// LeaseWithTime contains lease object with a time.
+// For the lessor's lease heap, time identifies the lease expiration time.
+// For the lessor's lease checkpoint heap, the time identifies the next lease checkpoint time.
+type LeaseWithTime struct {
+	id LeaseID
+	// Unix nanos timestamp.
+	time  int64
+	index int
+}
+
+type LeaseQueue []*LeaseWithTime
+
+func (pq LeaseQueue) Len() int { return len(pq) }
+
+func (pq LeaseQueue) Less(i, j int) bool {
+	return pq[i].time < pq[j].time
+}
+
+func (pq LeaseQueue) Swap(i, j int) {
+	pq[i], pq[j] = pq[j], pq[i]
+	pq[i].index = i
+	pq[j].index = j
+}
+
+func (pq *LeaseQueue) Push(x interface{}) {
+	n := len(*pq)
+	item := x.(*LeaseWithTime)
+	item.index = n
+	*pq = append(*pq, item)
+}
+
+func (pq *LeaseQueue) Pop() interface{} {
+	old := *pq
+	n := len(old)
+	item := old[n-1]
+	item.index = -1 // for safety
+	*pq = old[0 : n-1]
+	return item
+}
+
+// LeaseExpiredNotifier is a queue used to notify lessor to revoke expired lease.
+// Only save one item for a lease, `Register` will update time of the corresponding lease.
+type LeaseExpiredNotifier struct {
+	m     map[LeaseID]*LeaseWithTime
+	queue LeaseQueue
+}
+
+func newLeaseExpiredNotifier() *LeaseExpiredNotifier {
+	return &LeaseExpiredNotifier{
+		m:     make(map[LeaseID]*LeaseWithTime),
+		queue: make(LeaseQueue, 0),
+	}
+}
+
+func (mq *LeaseExpiredNotifier) Init() {
+	heap.Init(&mq.queue)
+	mq.m = make(map[LeaseID]*LeaseWithTime)
+	for _, item := range mq.queue {
+		mq.m[item.id] = item
+	}
+}
+
+func (mq *LeaseExpiredNotifier) RegisterOrUpdate(item *LeaseWithTime) {
+	if old, ok := mq.m[item.id]; ok {
+		old.time = item.time
+		heap.Fix(&mq.queue, old.index)
+	} else {
+		heap.Push(&mq.queue, item)
+		mq.m[item.id] = item
+	}
+}
+
+func (mq *LeaseExpiredNotifier) Unregister() *LeaseWithTime {
+	item := heap.Pop(&mq.queue).(*LeaseWithTime)
+	delete(mq.m, item.id)
+	return item
+}
+
+func (mq *LeaseExpiredNotifier) Poll() *LeaseWithTime {
+	if mq.Len() == 0 {
+		return nil
+	}
+	return mq.queue[0]
+}
+
+func (mq *LeaseExpiredNotifier) Len() int {
+	return len(mq.m)
+}
diff --git a/vendor/go.etcd.io/etcd/lease/leasehttp/doc.go b/vendor/go.etcd.io/etcd/lease/leasehttp/doc.go
new file mode 100644
index 0000000..8177a37
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/lease/leasehttp/doc.go
@@ -0,0 +1,16 @@
+// Copyright 2016 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 leasehttp serves lease renewals made through HTTP requests.
+package leasehttp
diff --git a/vendor/go.etcd.io/etcd/lease/leasehttp/http.go b/vendor/go.etcd.io/etcd/lease/leasehttp/http.go
new file mode 100644
index 0000000..67e916d
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/lease/leasehttp/http.go
@@ -0,0 +1,248 @@
+// Copyright 2016 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 leasehttp
+
+import (
+	"bytes"
+	"context"
+	"errors"
+	"fmt"
+	"io/ioutil"
+	"net/http"
+	"time"
+
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/lease"
+	"go.etcd.io/etcd/lease/leasepb"
+	"go.etcd.io/etcd/pkg/httputil"
+)
+
+var (
+	LeasePrefix         = "/leases"
+	LeaseInternalPrefix = "/leases/internal"
+	applyTimeout        = time.Second
+	ErrLeaseHTTPTimeout = errors.New("waiting for node to catch up its applied index has timed out")
+)
+
+// NewHandler returns an http Handler for lease renewals
+func NewHandler(l lease.Lessor, waitch func() <-chan struct{}) http.Handler {
+	return &leaseHandler{l, waitch}
+}
+
+type leaseHandler struct {
+	l      lease.Lessor
+	waitch func() <-chan struct{}
+}
+
+func (h *leaseHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) {
+	if r.Method != "POST" {
+		http.Error(w, "Method Not Allowed", http.StatusMethodNotAllowed)
+		return
+	}
+
+	defer r.Body.Close()
+	b, err := ioutil.ReadAll(r.Body)
+	if err != nil {
+		http.Error(w, "error reading body", http.StatusBadRequest)
+		return
+	}
+
+	var v []byte
+	switch r.URL.Path {
+	case LeasePrefix:
+		lreq := pb.LeaseKeepAliveRequest{}
+		if uerr := lreq.Unmarshal(b); uerr != nil {
+			http.Error(w, "error unmarshalling request", http.StatusBadRequest)
+			return
+		}
+		select {
+		case <-h.waitch():
+		case <-time.After(applyTimeout):
+			http.Error(w, ErrLeaseHTTPTimeout.Error(), http.StatusRequestTimeout)
+			return
+		}
+		ttl, rerr := h.l.Renew(lease.LeaseID(lreq.ID))
+		if rerr != nil {
+			if rerr == lease.ErrLeaseNotFound {
+				http.Error(w, rerr.Error(), http.StatusNotFound)
+				return
+			}
+
+			http.Error(w, rerr.Error(), http.StatusBadRequest)
+			return
+		}
+		// TODO: fill out ResponseHeader
+		resp := &pb.LeaseKeepAliveResponse{ID: lreq.ID, TTL: ttl}
+		v, err = resp.Marshal()
+		if err != nil {
+			http.Error(w, err.Error(), http.StatusInternalServerError)
+			return
+		}
+
+	case LeaseInternalPrefix:
+		lreq := leasepb.LeaseInternalRequest{}
+		if lerr := lreq.Unmarshal(b); lerr != nil {
+			http.Error(w, "error unmarshalling request", http.StatusBadRequest)
+			return
+		}
+		select {
+		case <-h.waitch():
+		case <-time.After(applyTimeout):
+			http.Error(w, ErrLeaseHTTPTimeout.Error(), http.StatusRequestTimeout)
+			return
+		}
+		l := h.l.Lookup(lease.LeaseID(lreq.LeaseTimeToLiveRequest.ID))
+		if l == nil {
+			http.Error(w, lease.ErrLeaseNotFound.Error(), http.StatusNotFound)
+			return
+		}
+		// TODO: fill out ResponseHeader
+		resp := &leasepb.LeaseInternalResponse{
+			LeaseTimeToLiveResponse: &pb.LeaseTimeToLiveResponse{
+				Header:     &pb.ResponseHeader{},
+				ID:         lreq.LeaseTimeToLiveRequest.ID,
+				TTL:        int64(l.Remaining().Seconds()),
+				GrantedTTL: l.TTL(),
+			},
+		}
+		if lreq.LeaseTimeToLiveRequest.Keys {
+			ks := l.Keys()
+			kbs := make([][]byte, len(ks))
+			for i := range ks {
+				kbs[i] = []byte(ks[i])
+			}
+			resp.LeaseTimeToLiveResponse.Keys = kbs
+		}
+
+		v, err = resp.Marshal()
+		if err != nil {
+			http.Error(w, err.Error(), http.StatusInternalServerError)
+			return
+		}
+
+	default:
+		http.Error(w, fmt.Sprintf("unknown request path %q", r.URL.Path), http.StatusBadRequest)
+		return
+	}
+
+	w.Header().Set("Content-Type", "application/protobuf")
+	w.Write(v)
+}
+
+// RenewHTTP renews a lease at a given primary server.
+// TODO: Batch request in future?
+func RenewHTTP(ctx context.Context, id lease.LeaseID, url string, rt http.RoundTripper) (int64, error) {
+	// will post lreq protobuf to leader
+	lreq, err := (&pb.LeaseKeepAliveRequest{ID: int64(id)}).Marshal()
+	if err != nil {
+		return -1, err
+	}
+
+	cc := &http.Client{Transport: rt}
+	req, err := http.NewRequest("POST", url, bytes.NewReader(lreq))
+	if err != nil {
+		return -1, err
+	}
+	req.Header.Set("Content-Type", "application/protobuf")
+	req.Cancel = ctx.Done()
+
+	resp, err := cc.Do(req)
+	if err != nil {
+		return -1, err
+	}
+	b, err := readResponse(resp)
+	if err != nil {
+		return -1, err
+	}
+
+	if resp.StatusCode == http.StatusRequestTimeout {
+		return -1, ErrLeaseHTTPTimeout
+	}
+
+	if resp.StatusCode == http.StatusNotFound {
+		return -1, lease.ErrLeaseNotFound
+	}
+
+	if resp.StatusCode != http.StatusOK {
+		return -1, fmt.Errorf("lease: unknown error(%s)", string(b))
+	}
+
+	lresp := &pb.LeaseKeepAliveResponse{}
+	if err := lresp.Unmarshal(b); err != nil {
+		return -1, fmt.Errorf(`lease: %v. data = "%s"`, err, string(b))
+	}
+	if lresp.ID != int64(id) {
+		return -1, fmt.Errorf("lease: renew id mismatch")
+	}
+	return lresp.TTL, nil
+}
+
+// TimeToLiveHTTP retrieves lease information of the given lease ID.
+func TimeToLiveHTTP(ctx context.Context, id lease.LeaseID, keys bool, url string, rt http.RoundTripper) (*leasepb.LeaseInternalResponse, error) {
+	// will post lreq protobuf to leader
+	lreq, err := (&leasepb.LeaseInternalRequest{
+		LeaseTimeToLiveRequest: &pb.LeaseTimeToLiveRequest{
+			ID:   int64(id),
+			Keys: keys,
+		},
+	}).Marshal()
+	if err != nil {
+		return nil, err
+	}
+
+	req, err := http.NewRequest("POST", url, bytes.NewReader(lreq))
+	if err != nil {
+		return nil, err
+	}
+	req.Header.Set("Content-Type", "application/protobuf")
+
+	req = req.WithContext(ctx)
+
+	cc := &http.Client{Transport: rt}
+	var b []byte
+	// buffer errc channel so that errc don't block inside the go routinue
+	resp, err := cc.Do(req)
+	if err != nil {
+		return nil, err
+	}
+	b, err = readResponse(resp)
+	if err != nil {
+		return nil, err
+	}
+	if resp.StatusCode == http.StatusRequestTimeout {
+		return nil, ErrLeaseHTTPTimeout
+	}
+	if resp.StatusCode == http.StatusNotFound {
+		return nil, lease.ErrLeaseNotFound
+	}
+	if resp.StatusCode != http.StatusOK {
+		return nil, fmt.Errorf("lease: unknown error(%s)", string(b))
+	}
+
+	lresp := &leasepb.LeaseInternalResponse{}
+	if err := lresp.Unmarshal(b); err != nil {
+		return nil, fmt.Errorf(`lease: %v. data = "%s"`, err, string(b))
+	}
+	if lresp.LeaseTimeToLiveResponse.ID != int64(id) {
+		return nil, fmt.Errorf("lease: renew id mismatch")
+	}
+	return lresp, nil
+}
+
+func readResponse(resp *http.Response) (b []byte, err error) {
+	b, err = ioutil.ReadAll(resp.Body)
+	httputil.GracefulClose(resp)
+	return
+}
diff --git a/vendor/go.etcd.io/etcd/lease/leasepb/lease.pb.go b/vendor/go.etcd.io/etcd/lease/leasepb/lease.pb.go
new file mode 100644
index 0000000..16637ee
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/lease/leasepb/lease.pb.go
@@ -0,0 +1,620 @@
+// Code generated by protoc-gen-gogo. DO NOT EDIT.
+// source: lease.proto
+
+/*
+	Package leasepb is a generated protocol buffer package.
+
+	It is generated from these files:
+		lease.proto
+
+	It has these top-level messages:
+		Lease
+		LeaseInternalRequest
+		LeaseInternalResponse
+*/
+package leasepb
+
+import (
+	"fmt"
+
+	proto "github.com/golang/protobuf/proto"
+
+	math "math"
+
+	_ "github.com/gogo/protobuf/gogoproto"
+
+	etcdserverpb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+
+	io "io"
+)
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
+
+type Lease struct {
+	ID           int64 `protobuf:"varint,1,opt,name=ID,proto3" json:"ID,omitempty"`
+	TTL          int64 `protobuf:"varint,2,opt,name=TTL,proto3" json:"TTL,omitempty"`
+	RemainingTTL int64 `protobuf:"varint,3,opt,name=RemainingTTL,proto3" json:"RemainingTTL,omitempty"`
+}
+
+func (m *Lease) Reset()                    { *m = Lease{} }
+func (m *Lease) String() string            { return proto.CompactTextString(m) }
+func (*Lease) ProtoMessage()               {}
+func (*Lease) Descriptor() ([]byte, []int) { return fileDescriptorLease, []int{0} }
+
+type LeaseInternalRequest struct {
+	LeaseTimeToLiveRequest *etcdserverpb.LeaseTimeToLiveRequest `protobuf:"bytes,1,opt,name=LeaseTimeToLiveRequest" json:"LeaseTimeToLiveRequest,omitempty"`
+}
+
+func (m *LeaseInternalRequest) Reset()                    { *m = LeaseInternalRequest{} }
+func (m *LeaseInternalRequest) String() string            { return proto.CompactTextString(m) }
+func (*LeaseInternalRequest) ProtoMessage()               {}
+func (*LeaseInternalRequest) Descriptor() ([]byte, []int) { return fileDescriptorLease, []int{1} }
+
+type LeaseInternalResponse struct {
+	LeaseTimeToLiveResponse *etcdserverpb.LeaseTimeToLiveResponse `protobuf:"bytes,1,opt,name=LeaseTimeToLiveResponse" json:"LeaseTimeToLiveResponse,omitempty"`
+}
+
+func (m *LeaseInternalResponse) Reset()                    { *m = LeaseInternalResponse{} }
+func (m *LeaseInternalResponse) String() string            { return proto.CompactTextString(m) }
+func (*LeaseInternalResponse) ProtoMessage()               {}
+func (*LeaseInternalResponse) Descriptor() ([]byte, []int) { return fileDescriptorLease, []int{2} }
+
+func init() {
+	proto.RegisterType((*Lease)(nil), "leasepb.Lease")
+	proto.RegisterType((*LeaseInternalRequest)(nil), "leasepb.LeaseInternalRequest")
+	proto.RegisterType((*LeaseInternalResponse)(nil), "leasepb.LeaseInternalResponse")
+}
+func (m *Lease) 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 *Lease) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if m.ID != 0 {
+		dAtA[i] = 0x8
+		i++
+		i = encodeVarintLease(dAtA, i, uint64(m.ID))
+	}
+	if m.TTL != 0 {
+		dAtA[i] = 0x10
+		i++
+		i = encodeVarintLease(dAtA, i, uint64(m.TTL))
+	}
+	if m.RemainingTTL != 0 {
+		dAtA[i] = 0x18
+		i++
+		i = encodeVarintLease(dAtA, i, uint64(m.RemainingTTL))
+	}
+	return i, nil
+}
+
+func (m *LeaseInternalRequest) 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 *LeaseInternalRequest) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if m.LeaseTimeToLiveRequest != nil {
+		dAtA[i] = 0xa
+		i++
+		i = encodeVarintLease(dAtA, i, uint64(m.LeaseTimeToLiveRequest.Size()))
+		n1, err := m.LeaseTimeToLiveRequest.MarshalTo(dAtA[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n1
+	}
+	return i, nil
+}
+
+func (m *LeaseInternalResponse) 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 *LeaseInternalResponse) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if m.LeaseTimeToLiveResponse != nil {
+		dAtA[i] = 0xa
+		i++
+		i = encodeVarintLease(dAtA, i, uint64(m.LeaseTimeToLiveResponse.Size()))
+		n2, err := m.LeaseTimeToLiveResponse.MarshalTo(dAtA[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n2
+	}
+	return i, nil
+}
+
+func encodeVarintLease(dAtA []byte, offset int, v uint64) int {
+	for v >= 1<<7 {
+		dAtA[offset] = uint8(v&0x7f | 0x80)
+		v >>= 7
+		offset++
+	}
+	dAtA[offset] = uint8(v)
+	return offset + 1
+}
+func (m *Lease) Size() (n int) {
+	var l int
+	_ = l
+	if m.ID != 0 {
+		n += 1 + sovLease(uint64(m.ID))
+	}
+	if m.TTL != 0 {
+		n += 1 + sovLease(uint64(m.TTL))
+	}
+	if m.RemainingTTL != 0 {
+		n += 1 + sovLease(uint64(m.RemainingTTL))
+	}
+	return n
+}
+
+func (m *LeaseInternalRequest) Size() (n int) {
+	var l int
+	_ = l
+	if m.LeaseTimeToLiveRequest != nil {
+		l = m.LeaseTimeToLiveRequest.Size()
+		n += 1 + l + sovLease(uint64(l))
+	}
+	return n
+}
+
+func (m *LeaseInternalResponse) Size() (n int) {
+	var l int
+	_ = l
+	if m.LeaseTimeToLiveResponse != nil {
+		l = m.LeaseTimeToLiveResponse.Size()
+		n += 1 + l + sovLease(uint64(l))
+	}
+	return n
+}
+
+func sovLease(x uint64) (n int) {
+	for {
+		n++
+		x >>= 7
+		if x == 0 {
+			break
+		}
+	}
+	return n
+}
+func sozLease(x uint64) (n int) {
+	return sovLease(uint64((x << 1) ^ uint64((int64(x) >> 63))))
+}
+func (m *Lease) 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 ErrIntOverflowLease
+			}
+			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: Lease: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: Lease: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field ID", wireType)
+			}
+			m.ID = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLease
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ID |= (int64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field TTL", wireType)
+			}
+			m.TTL = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLease
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.TTL |= (int64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field RemainingTTL", wireType)
+			}
+			m.RemainingTTL = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLease
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.RemainingTTL |= (int64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipLease(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthLease
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *LeaseInternalRequest) 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 ErrIntOverflowLease
+			}
+			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: LeaseInternalRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LeaseInternalRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field LeaseTimeToLiveRequest", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLease
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthLease
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.LeaseTimeToLiveRequest == nil {
+				m.LeaseTimeToLiveRequest = &etcdserverpb.LeaseTimeToLiveRequest{}
+			}
+			if err := m.LeaseTimeToLiveRequest.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipLease(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthLease
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *LeaseInternalResponse) 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 ErrIntOverflowLease
+			}
+			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: LeaseInternalResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LeaseInternalResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field LeaseTimeToLiveResponse", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLease
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthLease
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.LeaseTimeToLiveResponse == nil {
+				m.LeaseTimeToLiveResponse = &etcdserverpb.LeaseTimeToLiveResponse{}
+			}
+			if err := m.LeaseTimeToLiveResponse.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipLease(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthLease
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func skipLease(dAtA []byte) (n int, err error) {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return 0, ErrIntOverflowLease
+			}
+			if iNdEx >= l {
+				return 0, io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= (uint64(b) & 0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		wireType := int(wire & 0x7)
+		switch wireType {
+		case 0:
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return 0, ErrIntOverflowLease
+				}
+				if iNdEx >= l {
+					return 0, io.ErrUnexpectedEOF
+				}
+				iNdEx++
+				if dAtA[iNdEx-1] < 0x80 {
+					break
+				}
+			}
+			return iNdEx, nil
+		case 1:
+			iNdEx += 8
+			return iNdEx, nil
+		case 2:
+			var length int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return 0, ErrIntOverflowLease
+				}
+				if iNdEx >= l {
+					return 0, io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				length |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			iNdEx += length
+			if length < 0 {
+				return 0, ErrInvalidLengthLease
+			}
+			return iNdEx, nil
+		case 3:
+			for {
+				var innerWire uint64
+				var start int = iNdEx
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return 0, ErrIntOverflowLease
+					}
+					if iNdEx >= l {
+						return 0, io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					innerWire |= (uint64(b) & 0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				innerWireType := int(innerWire & 0x7)
+				if innerWireType == 4 {
+					break
+				}
+				next, err := skipLease(dAtA[start:])
+				if err != nil {
+					return 0, err
+				}
+				iNdEx = start + next
+			}
+			return iNdEx, nil
+		case 4:
+			return iNdEx, nil
+		case 5:
+			iNdEx += 4
+			return iNdEx, nil
+		default:
+			return 0, fmt.Errorf("proto: illegal wireType %d", wireType)
+		}
+	}
+	panic("unreachable")
+}
+
+var (
+	ErrInvalidLengthLease = fmt.Errorf("proto: negative length found during unmarshaling")
+	ErrIntOverflowLease   = fmt.Errorf("proto: integer overflow")
+)
+
+func init() { proto.RegisterFile("lease.proto", fileDescriptorLease) }
+
+var fileDescriptorLease = []byte{
+	// 253 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0xe2, 0xce, 0x49, 0x4d, 0x2c,
+	0x4e, 0xd5, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x62, 0x07, 0x73, 0x0a, 0x92, 0xa4, 0x44, 0xd2,
+	0xf3, 0xd3, 0xf3, 0xc1, 0x62, 0xfa, 0x20, 0x16, 0x44, 0x5a, 0x4a, 0x2d, 0xb5, 0x24, 0x39, 0x45,
+	0x1f, 0x44, 0x14, 0xa7, 0x16, 0x95, 0xa5, 0x16, 0x21, 0x31, 0x0b, 0x92, 0xf4, 0x8b, 0x0a, 0x92,
+	0x21, 0xea, 0x94, 0x7c, 0xb9, 0x58, 0x7d, 0x40, 0x06, 0x09, 0xf1, 0x71, 0x31, 0x79, 0xba, 0x48,
+	0x30, 0x2a, 0x30, 0x6a, 0x30, 0x07, 0x31, 0x79, 0xba, 0x08, 0x09, 0x70, 0x31, 0x87, 0x84, 0xf8,
+	0x48, 0x30, 0x81, 0x05, 0x40, 0x4c, 0x21, 0x25, 0x2e, 0x9e, 0xa0, 0xd4, 0xdc, 0xc4, 0xcc, 0xbc,
+	0xcc, 0xbc, 0x74, 0x90, 0x14, 0x33, 0x58, 0x0a, 0x45, 0x4c, 0xa9, 0x84, 0x4b, 0x04, 0x6c, 0x9c,
+	0x67, 0x5e, 0x49, 0x6a, 0x51, 0x5e, 0x62, 0x4e, 0x50, 0x6a, 0x61, 0x69, 0x6a, 0x71, 0x89, 0x50,
+	0x0c, 0x97, 0x18, 0x58, 0x3c, 0x24, 0x33, 0x37, 0x35, 0x24, 0xdf, 0x27, 0xb3, 0x2c, 0x15, 0x2a,
+	0x03, 0xb6, 0x91, 0xdb, 0x48, 0x45, 0x0f, 0xd9, 0x7d, 0x7a, 0xd8, 0xd5, 0x06, 0xe1, 0x30, 0x43,
+	0xa9, 0x82, 0x4b, 0x14, 0xcd, 0xd6, 0xe2, 0x82, 0xfc, 0xbc, 0xe2, 0x54, 0xa1, 0x78, 0x2e, 0x71,
+	0x0c, 0x2d, 0x10, 0x29, 0xa8, 0xbd, 0xaa, 0x04, 0xec, 0x85, 0x28, 0x0e, 0xc2, 0x65, 0x8a, 0x93,
+	0xc4, 0x89, 0x87, 0x72, 0x0c, 0x17, 0x1e, 0xca, 0x31, 0x9c, 0x78, 0x24, 0xc7, 0x78, 0xe1, 0x91,
+	0x1c, 0xe3, 0x83, 0x47, 0x72, 0x8c, 0x33, 0x1e, 0xcb, 0x31, 0x24, 0xb1, 0x81, 0xc3, 0xd7, 0x18,
+	0x10, 0x00, 0x00, 0xff, 0xff, 0xa9, 0x9f, 0x8b, 0x6c, 0xb5, 0x01, 0x00, 0x00,
+}
diff --git a/vendor/go.etcd.io/etcd/lease/leasepb/lease.proto b/vendor/go.etcd.io/etcd/lease/leasepb/lease.proto
new file mode 100644
index 0000000..1169d9f
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/lease/leasepb/lease.proto
@@ -0,0 +1,25 @@
+syntax = "proto3";
+package leasepb;
+
+import "gogoproto/gogo.proto";
+import "etcd/etcdserver/etcdserverpb/rpc.proto";
+
+option (gogoproto.marshaler_all) = true;
+option (gogoproto.sizer_all) = true;
+option (gogoproto.unmarshaler_all) = true;
+option (gogoproto.goproto_getters_all) = false;
+option (gogoproto.goproto_enum_prefix_all) = false;
+
+message Lease {
+  int64 ID = 1;
+  int64 TTL = 2;
+  int64 RemainingTTL = 3;
+}
+
+message LeaseInternalRequest {
+  etcdserverpb.LeaseTimeToLiveRequest LeaseTimeToLiveRequest = 1;
+}
+
+message LeaseInternalResponse {
+  etcdserverpb.LeaseTimeToLiveResponse LeaseTimeToLiveResponse = 1;
+}
diff --git a/vendor/go.etcd.io/etcd/lease/lessor.go b/vendor/go.etcd.io/etcd/lease/lessor.go
new file mode 100644
index 0000000..cc20283
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/lease/lessor.go
@@ -0,0 +1,933 @@
+// Copyright 2015 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 lease
+
+import (
+	"container/heap"
+	"context"
+	"encoding/binary"
+	"errors"
+	"math"
+	"sort"
+	"sync"
+	"time"
+
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"go.etcd.io/etcd/lease/leasepb"
+	"go.etcd.io/etcd/mvcc/backend"
+	"go.uber.org/zap"
+)
+
+// NoLease is a special LeaseID representing the absence of a lease.
+const NoLease = LeaseID(0)
+
+// MaxLeaseTTL is the maximum lease TTL value
+const MaxLeaseTTL = 9000000000
+
+var (
+	forever = time.Time{}
+
+	leaseBucketName = []byte("lease")
+
+	// maximum number of leases to revoke per second; configurable for tests
+	leaseRevokeRate = 1000
+
+	// maximum number of lease checkpoints recorded to the consensus log per second; configurable for tests
+	leaseCheckpointRate = 1000
+
+	// the default interval of lease checkpoint
+	defaultLeaseCheckpointInterval = 5 * time.Minute
+
+	// maximum number of lease checkpoints to batch into a single consensus log entry
+	maxLeaseCheckpointBatchSize = 1000
+
+	// the default interval to check if the expired lease is revoked
+	defaultExpiredleaseRetryInterval = 3 * time.Second
+
+	ErrNotPrimary       = errors.New("not a primary lessor")
+	ErrLeaseNotFound    = errors.New("lease not found")
+	ErrLeaseExists      = errors.New("lease already exists")
+	ErrLeaseTTLTooLarge = errors.New("too large lease TTL")
+)
+
+// TxnDelete is a TxnWrite that only permits deletes. Defined here
+// to avoid circular dependency with mvcc.
+type TxnDelete interface {
+	DeleteRange(key, end []byte) (n, rev int64)
+	End()
+}
+
+// RangeDeleter is a TxnDelete constructor.
+type RangeDeleter func() TxnDelete
+
+// Checkpointer permits checkpointing of lease remaining TTLs to the consensus log. Defined here to
+// avoid circular dependency with mvcc.
+type Checkpointer func(ctx context.Context, lc *pb.LeaseCheckpointRequest)
+
+type LeaseID int64
+
+// Lessor owns leases. It can grant, revoke, renew and modify leases for lessee.
+type Lessor interface {
+	// SetRangeDeleter lets the lessor create TxnDeletes to the store.
+	// Lessor deletes the items in the revoked or expired lease by creating
+	// new TxnDeletes.
+	SetRangeDeleter(rd RangeDeleter)
+
+	SetCheckpointer(cp Checkpointer)
+
+	// Grant grants a lease that expires at least after TTL seconds.
+	Grant(id LeaseID, ttl int64) (*Lease, error)
+	// Revoke revokes a lease with given ID. The item attached to the
+	// given lease will be removed. If the ID does not exist, an error
+	// will be returned.
+	Revoke(id LeaseID) error
+
+	// Checkpoint applies the remainingTTL of a lease. The remainingTTL is used in Promote to set
+	// the expiry of leases to less than the full TTL when possible.
+	Checkpoint(id LeaseID, remainingTTL int64) error
+
+	// Attach attaches given leaseItem to the lease with given LeaseID.
+	// If the lease does not exist, an error will be returned.
+	Attach(id LeaseID, items []LeaseItem) error
+
+	// GetLease returns LeaseID for given item.
+	// If no lease found, NoLease value will be returned.
+	GetLease(item LeaseItem) LeaseID
+
+	// Detach detaches given leaseItem from the lease with given LeaseID.
+	// If the lease does not exist, an error will be returned.
+	Detach(id LeaseID, items []LeaseItem) error
+
+	// Promote promotes the lessor to be the primary lessor. Primary lessor manages
+	// the expiration and renew of leases.
+	// Newly promoted lessor renew the TTL of all lease to extend + previous TTL.
+	Promote(extend time.Duration)
+
+	// Demote demotes the lessor from being the primary lessor.
+	Demote()
+
+	// Renew renews a lease with given ID. It returns the renewed TTL. If the ID does not exist,
+	// an error will be returned.
+	Renew(id LeaseID) (int64, error)
+
+	// Lookup gives the lease at a given lease id, if any
+	Lookup(id LeaseID) *Lease
+
+	// Leases lists all leases.
+	Leases() []*Lease
+
+	// ExpiredLeasesC returns a chan that is used to receive expired leases.
+	ExpiredLeasesC() <-chan []*Lease
+
+	// Recover recovers the lessor state from the given backend and RangeDeleter.
+	Recover(b backend.Backend, rd RangeDeleter)
+
+	// Stop stops the lessor for managing leases. The behavior of calling Stop multiple
+	// times is undefined.
+	Stop()
+}
+
+// lessor implements Lessor interface.
+// TODO: use clockwork for testability.
+type lessor struct {
+	mu sync.RWMutex
+
+	// demotec is set when the lessor is the primary.
+	// demotec will be closed if the lessor is demoted.
+	demotec chan struct{}
+
+	leaseMap             map[LeaseID]*Lease
+	leaseExpiredNotifier *LeaseExpiredNotifier
+	leaseCheckpointHeap  LeaseQueue
+	itemMap              map[LeaseItem]LeaseID
+
+	// When a lease expires, the lessor will delete the
+	// leased range (or key) by the RangeDeleter.
+	rd RangeDeleter
+
+	// When a lease's deadline should be persisted to preserve the remaining TTL across leader
+	// elections and restarts, the lessor will checkpoint the lease by the Checkpointer.
+	cp Checkpointer
+
+	// backend to persist leases. We only persist lease ID and expiry for now.
+	// The leased items can be recovered by iterating all the keys in kv.
+	b backend.Backend
+
+	// minLeaseTTL is the minimum lease TTL that can be granted for a lease. Any
+	// requests for shorter TTLs are extended to the minimum TTL.
+	minLeaseTTL int64
+
+	expiredC chan []*Lease
+	// stopC is a channel whose closure indicates that the lessor should be stopped.
+	stopC chan struct{}
+	// doneC is a channel whose closure indicates that the lessor is stopped.
+	doneC chan struct{}
+
+	lg *zap.Logger
+
+	// Wait duration between lease checkpoints.
+	checkpointInterval time.Duration
+	// the interval to check if the expired lease is revoked
+	expiredLeaseRetryInterval time.Duration
+}
+
+type LessorConfig struct {
+	MinLeaseTTL                int64
+	CheckpointInterval         time.Duration
+	ExpiredLeasesRetryInterval time.Duration
+}
+
+func NewLessor(lg *zap.Logger, b backend.Backend, cfg LessorConfig) Lessor {
+	return newLessor(lg, b, cfg)
+}
+
+func newLessor(lg *zap.Logger, b backend.Backend, cfg LessorConfig) *lessor {
+	checkpointInterval := cfg.CheckpointInterval
+	expiredLeaseRetryInterval := cfg.ExpiredLeasesRetryInterval
+	if checkpointInterval == 0 {
+		checkpointInterval = defaultLeaseCheckpointInterval
+	}
+	if expiredLeaseRetryInterval == 0 {
+		expiredLeaseRetryInterval = defaultExpiredleaseRetryInterval
+	}
+	l := &lessor{
+		leaseMap:                  make(map[LeaseID]*Lease),
+		itemMap:                   make(map[LeaseItem]LeaseID),
+		leaseExpiredNotifier:      newLeaseExpiredNotifier(),
+		leaseCheckpointHeap:       make(LeaseQueue, 0),
+		b:                         b,
+		minLeaseTTL:               cfg.MinLeaseTTL,
+		checkpointInterval:        checkpointInterval,
+		expiredLeaseRetryInterval: expiredLeaseRetryInterval,
+		// expiredC is a small buffered chan to avoid unnecessary blocking.
+		expiredC: make(chan []*Lease, 16),
+		stopC:    make(chan struct{}),
+		doneC:    make(chan struct{}),
+		lg:       lg,
+	}
+	l.initAndRecover()
+
+	go l.runLoop()
+
+	return l
+}
+
+// isPrimary indicates if this lessor is the primary lessor. The primary
+// lessor manages lease expiration and renew.
+//
+// in etcd, raft leader is the primary. Thus there might be two primary
+// leaders at the same time (raft allows concurrent leader but with different term)
+// for at most a leader election timeout.
+// The old primary leader cannot affect the correctness since its proposal has a
+// smaller term and will not be committed.
+//
+// TODO: raft follower do not forward lease management proposals. There might be a
+// very small window (within second normally which depends on go scheduling) that
+// a raft follow is the primary between the raft leader demotion and lessor demotion.
+// Usually this should not be a problem. Lease should not be that sensitive to timing.
+func (le *lessor) isPrimary() bool {
+	return le.demotec != nil
+}
+
+func (le *lessor) SetRangeDeleter(rd RangeDeleter) {
+	le.mu.Lock()
+	defer le.mu.Unlock()
+
+	le.rd = rd
+}
+
+func (le *lessor) SetCheckpointer(cp Checkpointer) {
+	le.mu.Lock()
+	defer le.mu.Unlock()
+
+	le.cp = cp
+}
+
+func (le *lessor) Grant(id LeaseID, ttl int64) (*Lease, error) {
+	if id == NoLease {
+		return nil, ErrLeaseNotFound
+	}
+
+	if ttl > MaxLeaseTTL {
+		return nil, ErrLeaseTTLTooLarge
+	}
+
+	// TODO: when lessor is under high load, it should give out lease
+	// with longer TTL to reduce renew load.
+	l := &Lease{
+		ID:      id,
+		ttl:     ttl,
+		itemSet: make(map[LeaseItem]struct{}),
+		revokec: make(chan struct{}),
+	}
+
+	le.mu.Lock()
+	defer le.mu.Unlock()
+
+	if _, ok := le.leaseMap[id]; ok {
+		return nil, ErrLeaseExists
+	}
+
+	if l.ttl < le.minLeaseTTL {
+		l.ttl = le.minLeaseTTL
+	}
+
+	if le.isPrimary() {
+		l.refresh(0)
+	} else {
+		l.forever()
+	}
+
+	le.leaseMap[id] = l
+	item := &LeaseWithTime{id: l.ID, time: l.expiry.UnixNano()}
+	le.leaseExpiredNotifier.RegisterOrUpdate(item)
+	l.persistTo(le.b)
+
+	leaseTotalTTLs.Observe(float64(l.ttl))
+	leaseGranted.Inc()
+
+	if le.isPrimary() {
+		le.scheduleCheckpointIfNeeded(l)
+	}
+
+	return l, nil
+}
+
+func (le *lessor) Revoke(id LeaseID) error {
+	le.mu.Lock()
+
+	l := le.leaseMap[id]
+	if l == nil {
+		le.mu.Unlock()
+		return ErrLeaseNotFound
+	}
+	defer close(l.revokec)
+	// unlock before doing external work
+	le.mu.Unlock()
+
+	if le.rd == nil {
+		return nil
+	}
+
+	txn := le.rd()
+
+	// sort keys so deletes are in same order among all members,
+	// otherwise the backend hashes will be different
+	keys := l.Keys()
+	sort.StringSlice(keys).Sort()
+	for _, key := range keys {
+		txn.DeleteRange([]byte(key), nil)
+	}
+
+	le.mu.Lock()
+	defer le.mu.Unlock()
+	delete(le.leaseMap, l.ID)
+	// lease deletion needs to be in the same backend transaction with the
+	// kv deletion. Or we might end up with not executing the revoke or not
+	// deleting the keys if etcdserver fails in between.
+	le.b.BatchTx().UnsafeDelete(leaseBucketName, int64ToBytes(int64(l.ID)))
+
+	txn.End()
+
+	leaseRevoked.Inc()
+	return nil
+}
+
+func (le *lessor) Checkpoint(id LeaseID, remainingTTL int64) error {
+	le.mu.Lock()
+	defer le.mu.Unlock()
+
+	if l, ok := le.leaseMap[id]; ok {
+		// when checkpointing, we only update the remainingTTL, Promote is responsible for applying this to lease expiry
+		l.remainingTTL = remainingTTL
+		if le.isPrimary() {
+			// schedule the next checkpoint as needed
+			le.scheduleCheckpointIfNeeded(l)
+		}
+	}
+	return nil
+}
+
+// Renew renews an existing lease. If the given lease does not exist or
+// has expired, an error will be returned.
+func (le *lessor) Renew(id LeaseID) (int64, error) {
+	le.mu.RLock()
+	if !le.isPrimary() {
+		// forward renew request to primary instead of returning error.
+		le.mu.RUnlock()
+		return -1, ErrNotPrimary
+	}
+
+	demotec := le.demotec
+
+	l := le.leaseMap[id]
+	if l == nil {
+		le.mu.RUnlock()
+		return -1, ErrLeaseNotFound
+	}
+	// Clear remaining TTL when we renew if it is set
+	clearRemainingTTL := le.cp != nil && l.remainingTTL > 0
+
+	le.mu.RUnlock()
+	if l.expired() {
+		select {
+		// A expired lease might be pending for revoking or going through
+		// quorum to be revoked. To be accurate, renew request must wait for the
+		// deletion to complete.
+		case <-l.revokec:
+			return -1, ErrLeaseNotFound
+		// The expired lease might fail to be revoked if the primary changes.
+		// The caller will retry on ErrNotPrimary.
+		case <-demotec:
+			return -1, ErrNotPrimary
+		case <-le.stopC:
+			return -1, ErrNotPrimary
+		}
+	}
+
+	// Clear remaining TTL when we renew if it is set
+	// By applying a RAFT entry only when the remainingTTL is already set, we limit the number
+	// of RAFT entries written per lease to a max of 2 per checkpoint interval.
+	if clearRemainingTTL {
+		le.cp(context.Background(), &pb.LeaseCheckpointRequest{Checkpoints: []*pb.LeaseCheckpoint{{ID: int64(l.ID), Remaining_TTL: 0}}})
+	}
+
+	le.mu.Lock()
+	l.refresh(0)
+	item := &LeaseWithTime{id: l.ID, time: l.expiry.UnixNano()}
+	le.leaseExpiredNotifier.RegisterOrUpdate(item)
+	le.mu.Unlock()
+
+	leaseRenewed.Inc()
+	return l.ttl, nil
+}
+
+func (le *lessor) Lookup(id LeaseID) *Lease {
+	le.mu.RLock()
+	defer le.mu.RUnlock()
+	return le.leaseMap[id]
+}
+
+func (le *lessor) unsafeLeases() []*Lease {
+	leases := make([]*Lease, 0, len(le.leaseMap))
+	for _, l := range le.leaseMap {
+		leases = append(leases, l)
+	}
+	return leases
+}
+
+func (le *lessor) Leases() []*Lease {
+	le.mu.RLock()
+	ls := le.unsafeLeases()
+	le.mu.RUnlock()
+	sort.Sort(leasesByExpiry(ls))
+	return ls
+}
+
+func (le *lessor) Promote(extend time.Duration) {
+	le.mu.Lock()
+	defer le.mu.Unlock()
+
+	le.demotec = make(chan struct{})
+
+	// refresh the expiries of all leases.
+	for _, l := range le.leaseMap {
+		l.refresh(extend)
+		item := &LeaseWithTime{id: l.ID, time: l.expiry.UnixNano()}
+		le.leaseExpiredNotifier.RegisterOrUpdate(item)
+	}
+
+	if len(le.leaseMap) < leaseRevokeRate {
+		// no possibility of lease pile-up
+		return
+	}
+
+	// adjust expiries in case of overlap
+	leases := le.unsafeLeases()
+	sort.Sort(leasesByExpiry(leases))
+
+	baseWindow := leases[0].Remaining()
+	nextWindow := baseWindow + time.Second
+	expires := 0
+	// have fewer expires than the total revoke rate so piled up leases
+	// don't consume the entire revoke limit
+	targetExpiresPerSecond := (3 * leaseRevokeRate) / 4
+	for _, l := range leases {
+		remaining := l.Remaining()
+		if remaining > nextWindow {
+			baseWindow = remaining
+			nextWindow = baseWindow + time.Second
+			expires = 1
+			continue
+		}
+		expires++
+		if expires <= targetExpiresPerSecond {
+			continue
+		}
+		rateDelay := float64(time.Second) * (float64(expires) / float64(targetExpiresPerSecond))
+		// If leases are extended by n seconds, leases n seconds ahead of the
+		// base window should be extended by only one second.
+		rateDelay -= float64(remaining - baseWindow)
+		delay := time.Duration(rateDelay)
+		nextWindow = baseWindow + delay
+		l.refresh(delay + extend)
+		item := &LeaseWithTime{id: l.ID, time: l.expiry.UnixNano()}
+		le.leaseExpiredNotifier.RegisterOrUpdate(item)
+		le.scheduleCheckpointIfNeeded(l)
+	}
+}
+
+type leasesByExpiry []*Lease
+
+func (le leasesByExpiry) Len() int           { return len(le) }
+func (le leasesByExpiry) Less(i, j int) bool { return le[i].Remaining() < le[j].Remaining() }
+func (le leasesByExpiry) Swap(i, j int)      { le[i], le[j] = le[j], le[i] }
+
+func (le *lessor) Demote() {
+	le.mu.Lock()
+	defer le.mu.Unlock()
+
+	// set the expiries of all leases to forever
+	for _, l := range le.leaseMap {
+		l.forever()
+	}
+
+	le.clearScheduledLeasesCheckpoints()
+
+	if le.demotec != nil {
+		close(le.demotec)
+		le.demotec = nil
+	}
+}
+
+// Attach attaches items to the lease with given ID. When the lease
+// expires, the attached items will be automatically removed.
+// If the given lease does not exist, an error will be returned.
+func (le *lessor) Attach(id LeaseID, items []LeaseItem) error {
+	le.mu.Lock()
+	defer le.mu.Unlock()
+
+	l := le.leaseMap[id]
+	if l == nil {
+		return ErrLeaseNotFound
+	}
+
+	l.mu.Lock()
+	for _, it := range items {
+		l.itemSet[it] = struct{}{}
+		le.itemMap[it] = id
+	}
+	l.mu.Unlock()
+	return nil
+}
+
+func (le *lessor) GetLease(item LeaseItem) LeaseID {
+	le.mu.RLock()
+	id := le.itemMap[item]
+	le.mu.RUnlock()
+	return id
+}
+
+// Detach detaches items from the lease with given ID.
+// If the given lease does not exist, an error will be returned.
+func (le *lessor) Detach(id LeaseID, items []LeaseItem) error {
+	le.mu.Lock()
+	defer le.mu.Unlock()
+
+	l := le.leaseMap[id]
+	if l == nil {
+		return ErrLeaseNotFound
+	}
+
+	l.mu.Lock()
+	for _, it := range items {
+		delete(l.itemSet, it)
+		delete(le.itemMap, it)
+	}
+	l.mu.Unlock()
+	return nil
+}
+
+func (le *lessor) Recover(b backend.Backend, rd RangeDeleter) {
+	le.mu.Lock()
+	defer le.mu.Unlock()
+
+	le.b = b
+	le.rd = rd
+	le.leaseMap = make(map[LeaseID]*Lease)
+	le.itemMap = make(map[LeaseItem]LeaseID)
+	le.initAndRecover()
+}
+
+func (le *lessor) ExpiredLeasesC() <-chan []*Lease {
+	return le.expiredC
+}
+
+func (le *lessor) Stop() {
+	close(le.stopC)
+	<-le.doneC
+}
+
+func (le *lessor) runLoop() {
+	defer close(le.doneC)
+
+	for {
+		le.revokeExpiredLeases()
+		le.checkpointScheduledLeases()
+
+		select {
+		case <-time.After(500 * time.Millisecond):
+		case <-le.stopC:
+			return
+		}
+	}
+}
+
+// revokeExpiredLeases finds all leases past their expiry and sends them to expired channel for
+// to be revoked.
+func (le *lessor) revokeExpiredLeases() {
+	var ls []*Lease
+
+	// rate limit
+	revokeLimit := leaseRevokeRate / 2
+
+	le.mu.RLock()
+	if le.isPrimary() {
+		ls = le.findExpiredLeases(revokeLimit)
+	}
+	le.mu.RUnlock()
+
+	if len(ls) != 0 {
+		select {
+		case <-le.stopC:
+			return
+		case le.expiredC <- ls:
+		default:
+			// the receiver of expiredC is probably busy handling
+			// other stuff
+			// let's try this next time after 500ms
+		}
+	}
+}
+
+// checkpointScheduledLeases finds all scheduled lease checkpoints that are due and
+// submits them to the checkpointer to persist them to the consensus log.
+func (le *lessor) checkpointScheduledLeases() {
+	var cps []*pb.LeaseCheckpoint
+
+	// rate limit
+	for i := 0; i < leaseCheckpointRate/2; i++ {
+		le.mu.Lock()
+		if le.isPrimary() {
+			cps = le.findDueScheduledCheckpoints(maxLeaseCheckpointBatchSize)
+		}
+		le.mu.Unlock()
+
+		if len(cps) != 0 {
+			le.cp(context.Background(), &pb.LeaseCheckpointRequest{Checkpoints: cps})
+		}
+		if len(cps) < maxLeaseCheckpointBatchSize {
+			return
+		}
+	}
+}
+
+func (le *lessor) clearScheduledLeasesCheckpoints() {
+	le.leaseCheckpointHeap = make(LeaseQueue, 0)
+}
+
+// expireExists returns true if expiry items exist.
+// It pops only when expiry item exists.
+// "next" is true, to indicate that it may exist in next attempt.
+func (le *lessor) expireExists() (l *Lease, ok bool, next bool) {
+	if le.leaseExpiredNotifier.Len() == 0 {
+		return nil, false, false
+	}
+
+	item := le.leaseExpiredNotifier.Poll()
+	l = le.leaseMap[item.id]
+	if l == nil {
+		// lease has expired or been revoked
+		// no need to revoke (nothing is expiry)
+		le.leaseExpiredNotifier.Unregister() // O(log N)
+		return nil, false, true
+	}
+	now := time.Now()
+	if now.UnixNano() < item.time /* expiration time */ {
+		// Candidate expirations are caught up, reinsert this item
+		// and no need to revoke (nothing is expiry)
+		return l, false, false
+	}
+
+	// recheck if revoke is complete after retry interval
+	item.time = now.Add(le.expiredLeaseRetryInterval).UnixNano()
+	le.leaseExpiredNotifier.RegisterOrUpdate(item)
+	return l, true, false
+}
+
+// findExpiredLeases loops leases in the leaseMap until reaching expired limit
+// and returns the expired leases that needed to be revoked.
+func (le *lessor) findExpiredLeases(limit int) []*Lease {
+	leases := make([]*Lease, 0, 16)
+
+	for {
+		l, ok, next := le.expireExists()
+		if !ok && !next {
+			break
+		}
+		if !ok {
+			continue
+		}
+		if next {
+			continue
+		}
+
+		if l.expired() {
+			leases = append(leases, l)
+
+			// reach expired limit
+			if len(leases) == limit {
+				break
+			}
+		}
+	}
+
+	return leases
+}
+
+func (le *lessor) scheduleCheckpointIfNeeded(lease *Lease) {
+	if le.cp == nil {
+		return
+	}
+
+	if lease.RemainingTTL() > int64(le.checkpointInterval.Seconds()) {
+		if le.lg != nil {
+			le.lg.Debug("Scheduling lease checkpoint",
+				zap.Int64("leaseID", int64(lease.ID)),
+				zap.Duration("intervalSeconds", le.checkpointInterval),
+			)
+		}
+		heap.Push(&le.leaseCheckpointHeap, &LeaseWithTime{
+			id:   lease.ID,
+			time: time.Now().Add(le.checkpointInterval).UnixNano(),
+		})
+	}
+}
+
+func (le *lessor) findDueScheduledCheckpoints(checkpointLimit int) []*pb.LeaseCheckpoint {
+	if le.cp == nil {
+		return nil
+	}
+
+	now := time.Now()
+	cps := []*pb.LeaseCheckpoint{}
+	for le.leaseCheckpointHeap.Len() > 0 && len(cps) < checkpointLimit {
+		lt := le.leaseCheckpointHeap[0]
+		if lt.time /* next checkpoint time */ > now.UnixNano() {
+			return cps
+		}
+		heap.Pop(&le.leaseCheckpointHeap)
+		var l *Lease
+		var ok bool
+		if l, ok = le.leaseMap[lt.id]; !ok {
+			continue
+		}
+		if !now.Before(l.expiry) {
+			continue
+		}
+		remainingTTL := int64(math.Ceil(l.expiry.Sub(now).Seconds()))
+		if remainingTTL >= l.ttl {
+			continue
+		}
+		if le.lg != nil {
+			le.lg.Debug("Checkpointing lease",
+				zap.Int64("leaseID", int64(lt.id)),
+				zap.Int64("remainingTTL", remainingTTL),
+			)
+		}
+		cps = append(cps, &pb.LeaseCheckpoint{ID: int64(lt.id), Remaining_TTL: remainingTTL})
+	}
+	return cps
+}
+
+func (le *lessor) initAndRecover() {
+	tx := le.b.BatchTx()
+	tx.Lock()
+
+	tx.UnsafeCreateBucket(leaseBucketName)
+	_, vs := tx.UnsafeRange(leaseBucketName, int64ToBytes(0), int64ToBytes(math.MaxInt64), 0)
+	// TODO: copy vs and do decoding outside tx lock if lock contention becomes an issue.
+	for i := range vs {
+		var lpb leasepb.Lease
+		err := lpb.Unmarshal(vs[i])
+		if err != nil {
+			tx.Unlock()
+			panic("failed to unmarshal lease proto item")
+		}
+		ID := LeaseID(lpb.ID)
+		if lpb.TTL < le.minLeaseTTL {
+			lpb.TTL = le.minLeaseTTL
+		}
+		le.leaseMap[ID] = &Lease{
+			ID:  ID,
+			ttl: lpb.TTL,
+			// itemSet will be filled in when recover key-value pairs
+			// set expiry to forever, refresh when promoted
+			itemSet: make(map[LeaseItem]struct{}),
+			expiry:  forever,
+			revokec: make(chan struct{}),
+		}
+	}
+	le.leaseExpiredNotifier.Init()
+	heap.Init(&le.leaseCheckpointHeap)
+	tx.Unlock()
+
+	le.b.ForceCommit()
+}
+
+type Lease struct {
+	ID           LeaseID
+	ttl          int64 // time to live of the lease in seconds
+	remainingTTL int64 // remaining time to live in seconds, if zero valued it is considered unset and the full ttl should be used
+	// expiryMu protects concurrent accesses to expiry
+	expiryMu sync.RWMutex
+	// expiry is time when lease should expire. no expiration when expiry.IsZero() is true
+	expiry time.Time
+
+	// mu protects concurrent accesses to itemSet
+	mu      sync.RWMutex
+	itemSet map[LeaseItem]struct{}
+	revokec chan struct{}
+}
+
+func (l *Lease) expired() bool {
+	return l.Remaining() <= 0
+}
+
+func (l *Lease) persistTo(b backend.Backend) {
+	key := int64ToBytes(int64(l.ID))
+
+	lpb := leasepb.Lease{ID: int64(l.ID), TTL: l.ttl, RemainingTTL: l.remainingTTL}
+	val, err := lpb.Marshal()
+	if err != nil {
+		panic("failed to marshal lease proto item")
+	}
+
+	b.BatchTx().Lock()
+	b.BatchTx().UnsafePut(leaseBucketName, key, val)
+	b.BatchTx().Unlock()
+}
+
+// TTL returns the TTL of the Lease.
+func (l *Lease) TTL() int64 {
+	return l.ttl
+}
+
+// RemainingTTL returns the last checkpointed remaining TTL of the lease.
+// TODO(jpbetz): do not expose this utility method
+func (l *Lease) RemainingTTL() int64 {
+	if l.remainingTTL > 0 {
+		return l.remainingTTL
+	}
+	return l.ttl
+}
+
+// refresh refreshes the expiry of the lease.
+func (l *Lease) refresh(extend time.Duration) {
+	newExpiry := time.Now().Add(extend + time.Duration(l.RemainingTTL())*time.Second)
+	l.expiryMu.Lock()
+	defer l.expiryMu.Unlock()
+	l.expiry = newExpiry
+}
+
+// forever sets the expiry of lease to be forever.
+func (l *Lease) forever() {
+	l.expiryMu.Lock()
+	defer l.expiryMu.Unlock()
+	l.expiry = forever
+}
+
+// Keys returns all the keys attached to the lease.
+func (l *Lease) Keys() []string {
+	l.mu.RLock()
+	keys := make([]string, 0, len(l.itemSet))
+	for k := range l.itemSet {
+		keys = append(keys, k.Key)
+	}
+	l.mu.RUnlock()
+	return keys
+}
+
+// Remaining returns the remaining time of the lease.
+func (l *Lease) Remaining() time.Duration {
+	l.expiryMu.RLock()
+	defer l.expiryMu.RUnlock()
+	if l.expiry.IsZero() {
+		return time.Duration(math.MaxInt64)
+	}
+	return time.Until(l.expiry)
+}
+
+type LeaseItem struct {
+	Key string
+}
+
+func int64ToBytes(n int64) []byte {
+	bytes := make([]byte, 8)
+	binary.BigEndian.PutUint64(bytes, uint64(n))
+	return bytes
+}
+
+// FakeLessor is a fake implementation of Lessor interface.
+// Used for testing only.
+type FakeLessor struct{}
+
+func (fl *FakeLessor) SetRangeDeleter(dr RangeDeleter) {}
+
+func (fl *FakeLessor) SetCheckpointer(cp Checkpointer) {}
+
+func (fl *FakeLessor) Grant(id LeaseID, ttl int64) (*Lease, error) { return nil, nil }
+
+func (fl *FakeLessor) Revoke(id LeaseID) error { return nil }
+
+func (fl *FakeLessor) Checkpoint(id LeaseID, remainingTTL int64) error { return nil }
+
+func (fl *FakeLessor) Attach(id LeaseID, items []LeaseItem) error { return nil }
+
+func (fl *FakeLessor) GetLease(item LeaseItem) LeaseID            { return 0 }
+func (fl *FakeLessor) Detach(id LeaseID, items []LeaseItem) error { return nil }
+
+func (fl *FakeLessor) Promote(extend time.Duration) {}
+
+func (fl *FakeLessor) Demote() {}
+
+func (fl *FakeLessor) Renew(id LeaseID) (int64, error) { return 10, nil }
+
+func (fl *FakeLessor) Lookup(id LeaseID) *Lease { return nil }
+
+func (fl *FakeLessor) Leases() []*Lease { return nil }
+
+func (fl *FakeLessor) ExpiredLeasesC() <-chan []*Lease { return nil }
+
+func (fl *FakeLessor) Recover(b backend.Backend, rd RangeDeleter) {}
+
+func (fl *FakeLessor) Stop() {}
+
+type FakeTxnDelete struct {
+	backend.BatchTx
+}
+
+func (ftd *FakeTxnDelete) DeleteRange(key, end []byte) (n, rev int64) { return 0, 0 }
+func (ftd *FakeTxnDelete) End()                                       { ftd.Unlock() }
diff --git a/vendor/go.etcd.io/etcd/lease/metrics.go b/vendor/go.etcd.io/etcd/lease/metrics.go
new file mode 100644
index 0000000..06f8b58
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/lease/metrics.go
@@ -0,0 +1,59 @@
+// Copyright 2018 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 lease
+
+import (
+	"github.com/prometheus/client_golang/prometheus"
+)
+
+var (
+	leaseGranted = prometheus.NewCounter(prometheus.CounterOpts{
+		Namespace: "etcd_debugging",
+		Subsystem: "lease",
+		Name:      "granted_total",
+		Help:      "The total number of granted leases.",
+	})
+
+	leaseRevoked = prometheus.NewCounter(prometheus.CounterOpts{
+		Namespace: "etcd_debugging",
+		Subsystem: "lease",
+		Name:      "revoked_total",
+		Help:      "The total number of revoked leases.",
+	})
+
+	leaseRenewed = prometheus.NewCounter(prometheus.CounterOpts{
+		Namespace: "etcd_debugging",
+		Subsystem: "lease",
+		Name:      "renewed_total",
+		Help:      "The number of renewed leases seen by the leader.",
+	})
+
+	leaseTotalTTLs = prometheus.NewHistogram(
+		prometheus.HistogramOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "lease",
+			Name:      "ttl_total",
+			Help:      "Bucketed histogram of lease TTLs.",
+			// 1 second -> 3 months
+			Buckets: prometheus.ExponentialBuckets(1, 2, 24),
+		})
+)
+
+func init() {
+	prometheus.MustRegister(leaseGranted)
+	prometheus.MustRegister(leaseRevoked)
+	prometheus.MustRegister(leaseRenewed)
+	prometheus.MustRegister(leaseTotalTTLs)
+}
diff --git a/vendor/go.etcd.io/etcd/mvcc/backend/backend.go b/vendor/go.etcd.io/etcd/mvcc/backend/backend.go
new file mode 100644
index 0000000..bffd749
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/backend/backend.go
@@ -0,0 +1,570 @@
+// Copyright 2015 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 backend
+
+import (
+	"fmt"
+	"hash/crc32"
+	"io"
+	"io/ioutil"
+	"os"
+	"path/filepath"
+	"sync"
+	"sync/atomic"
+	"time"
+
+	"github.com/coreos/pkg/capnslog"
+	humanize "github.com/dustin/go-humanize"
+	bolt "go.etcd.io/bbolt"
+	"go.uber.org/zap"
+)
+
+var (
+	defaultBatchLimit    = 10000
+	defaultBatchInterval = 100 * time.Millisecond
+
+	defragLimit = 10000
+
+	// initialMmapSize is the initial size of the mmapped region. Setting this larger than
+	// the potential max db size can prevent writer from blocking reader.
+	// This only works for linux.
+	initialMmapSize = uint64(10 * 1024 * 1024 * 1024)
+
+	plog = capnslog.NewPackageLogger("go.etcd.io/etcd", "mvcc/backend")
+
+	// minSnapshotWarningTimeout is the minimum threshold to trigger a long running snapshot warning.
+	minSnapshotWarningTimeout = 30 * time.Second
+)
+
+type Backend interface {
+	// ReadTx returns a read transaction. It is replaced by ConcurrentReadTx in the main data path, see #10523.
+	ReadTx() ReadTx
+	BatchTx() BatchTx
+	// ConcurrentReadTx returns a non-blocking read transaction.
+	ConcurrentReadTx() ReadTx
+
+	Snapshot() Snapshot
+	Hash(ignores map[IgnoreKey]struct{}) (uint32, error)
+	// Size returns the current size of the backend physically allocated.
+	// The backend can hold DB space that is not utilized at the moment,
+	// since it can conduct pre-allocation or spare unused space for recycling.
+	// Use SizeInUse() instead for the actual DB size.
+	Size() int64
+	// SizeInUse returns the current size of the backend logically in use.
+	// Since the backend can manage free space in a non-byte unit such as
+	// number of pages, the returned value can be not exactly accurate in bytes.
+	SizeInUse() int64
+	// OpenReadTxN returns the number of currently open read transactions in the backend.
+	OpenReadTxN() int64
+	Defrag() error
+	ForceCommit()
+	Close() error
+}
+
+type Snapshot interface {
+	// Size gets the size of the snapshot.
+	Size() int64
+	// WriteTo writes the snapshot into the given writer.
+	WriteTo(w io.Writer) (n int64, err error)
+	// Close closes the snapshot.
+	Close() error
+}
+
+type backend struct {
+	// size and commits are used with atomic operations so they must be
+	// 64-bit aligned, otherwise 32-bit tests will crash
+
+	// size is the number of bytes allocated in the backend
+	size int64
+	// sizeInUse is the number of bytes actually used in the backend
+	sizeInUse int64
+	// commits counts number of commits since start
+	commits int64
+	// openReadTxN is the number of currently open read transactions in the backend
+	openReadTxN int64
+
+	mu sync.RWMutex
+	db *bolt.DB
+
+	batchInterval time.Duration
+	batchLimit    int
+	batchTx       *batchTxBuffered
+
+	readTx *readTx
+
+	stopc chan struct{}
+	donec chan struct{}
+
+	lg *zap.Logger
+}
+
+type BackendConfig struct {
+	// Path is the file path to the backend file.
+	Path string
+	// BatchInterval is the maximum time before flushing the BatchTx.
+	BatchInterval time.Duration
+	// BatchLimit is the maximum puts before flushing the BatchTx.
+	BatchLimit int
+	// BackendFreelistType is the backend boltdb's freelist type.
+	BackendFreelistType bolt.FreelistType
+	// MmapSize is the number of bytes to mmap for the backend.
+	MmapSize uint64
+	// Logger logs backend-side operations.
+	Logger *zap.Logger
+}
+
+func DefaultBackendConfig() BackendConfig {
+	return BackendConfig{
+		BatchInterval: defaultBatchInterval,
+		BatchLimit:    defaultBatchLimit,
+		MmapSize:      initialMmapSize,
+	}
+}
+
+func New(bcfg BackendConfig) Backend {
+	return newBackend(bcfg)
+}
+
+func NewDefaultBackend(path string) Backend {
+	bcfg := DefaultBackendConfig()
+	bcfg.Path = path
+	return newBackend(bcfg)
+}
+
+func newBackend(bcfg BackendConfig) *backend {
+	bopts := &bolt.Options{}
+	if boltOpenOptions != nil {
+		*bopts = *boltOpenOptions
+	}
+	bopts.InitialMmapSize = bcfg.mmapSize()
+	bopts.FreelistType = bcfg.BackendFreelistType
+
+	db, err := bolt.Open(bcfg.Path, 0600, bopts)
+	if err != nil {
+		if bcfg.Logger != nil {
+			bcfg.Logger.Panic("failed to open database", zap.String("path", bcfg.Path), zap.Error(err))
+		} else {
+			plog.Panicf("cannot open database at %s (%v)", bcfg.Path, err)
+		}
+	}
+
+	// In future, may want to make buffering optional for low-concurrency systems
+	// or dynamically swap between buffered/non-buffered depending on workload.
+	b := &backend{
+		db: db,
+
+		batchInterval: bcfg.BatchInterval,
+		batchLimit:    bcfg.BatchLimit,
+
+		readTx: &readTx{
+			buf: txReadBuffer{
+				txBuffer: txBuffer{make(map[string]*bucketBuffer)},
+			},
+			buckets: make(map[string]*bolt.Bucket),
+			txWg:    new(sync.WaitGroup),
+		},
+
+		stopc: make(chan struct{}),
+		donec: make(chan struct{}),
+
+		lg: bcfg.Logger,
+	}
+	b.batchTx = newBatchTxBuffered(b)
+	go b.run()
+	return b
+}
+
+// BatchTx returns the current batch tx in coalescer. The tx can be used for read and
+// write operations. The write result can be retrieved within the same tx immediately.
+// The write result is isolated with other txs until the current one get committed.
+func (b *backend) BatchTx() BatchTx {
+	return b.batchTx
+}
+
+func (b *backend) ReadTx() ReadTx { return b.readTx }
+
+// ConcurrentReadTx creates and returns a new ReadTx, which:
+// A) creates and keeps a copy of backend.readTx.txReadBuffer,
+// B) references the boltdb read Tx (and its bucket cache) of current batch interval.
+func (b *backend) ConcurrentReadTx() ReadTx {
+	b.readTx.RLock()
+	defer b.readTx.RUnlock()
+	// prevent boltdb read Tx from been rolled back until store read Tx is done. Needs to be called when holding readTx.RLock().
+	b.readTx.txWg.Add(1)
+	// TODO: might want to copy the read buffer lazily - create copy when A) end of a write transaction B) end of a batch interval.
+	return &concurrentReadTx{
+		buf:     b.readTx.buf.unsafeCopy(),
+		tx:      b.readTx.tx,
+		txMu:    &b.readTx.txMu,
+		buckets: b.readTx.buckets,
+		txWg:    b.readTx.txWg,
+	}
+}
+
+// ForceCommit forces the current batching tx to commit.
+func (b *backend) ForceCommit() {
+	b.batchTx.Commit()
+}
+
+func (b *backend) Snapshot() Snapshot {
+	b.batchTx.Commit()
+
+	b.mu.RLock()
+	defer b.mu.RUnlock()
+	tx, err := b.db.Begin(false)
+	if err != nil {
+		if b.lg != nil {
+			b.lg.Fatal("failed to begin tx", zap.Error(err))
+		} else {
+			plog.Fatalf("cannot begin tx (%s)", err)
+		}
+	}
+
+	stopc, donec := make(chan struct{}), make(chan struct{})
+	dbBytes := tx.Size()
+	go func() {
+		defer close(donec)
+		// sendRateBytes is based on transferring snapshot data over a 1 gigabit/s connection
+		// assuming a min tcp throughput of 100MB/s.
+		var sendRateBytes int64 = 100 * 1024 * 1014
+		warningTimeout := time.Duration(int64((float64(dbBytes) / float64(sendRateBytes)) * float64(time.Second)))
+		if warningTimeout < minSnapshotWarningTimeout {
+			warningTimeout = minSnapshotWarningTimeout
+		}
+		start := time.Now()
+		ticker := time.NewTicker(warningTimeout)
+		defer ticker.Stop()
+		for {
+			select {
+			case <-ticker.C:
+				if b.lg != nil {
+					b.lg.Warn(
+						"snapshotting taking too long to transfer",
+						zap.Duration("taking", time.Since(start)),
+						zap.Int64("bytes", dbBytes),
+						zap.String("size", humanize.Bytes(uint64(dbBytes))),
+					)
+				} else {
+					plog.Warningf("snapshotting is taking more than %v seconds to finish transferring %v MB [started at %v]", time.Since(start).Seconds(), float64(dbBytes)/float64(1024*1014), start)
+				}
+
+			case <-stopc:
+				snapshotTransferSec.Observe(time.Since(start).Seconds())
+				return
+			}
+		}
+	}()
+
+	return &snapshot{tx, stopc, donec}
+}
+
+type IgnoreKey struct {
+	Bucket string
+	Key    string
+}
+
+func (b *backend) Hash(ignores map[IgnoreKey]struct{}) (uint32, error) {
+	h := crc32.New(crc32.MakeTable(crc32.Castagnoli))
+
+	b.mu.RLock()
+	defer b.mu.RUnlock()
+	err := b.db.View(func(tx *bolt.Tx) error {
+		c := tx.Cursor()
+		for next, _ := c.First(); next != nil; next, _ = c.Next() {
+			b := tx.Bucket(next)
+			if b == nil {
+				return fmt.Errorf("cannot get hash of bucket %s", string(next))
+			}
+			h.Write(next)
+			b.ForEach(func(k, v []byte) error {
+				bk := IgnoreKey{Bucket: string(next), Key: string(k)}
+				if _, ok := ignores[bk]; !ok {
+					h.Write(k)
+					h.Write(v)
+				}
+				return nil
+			})
+		}
+		return nil
+	})
+
+	if err != nil {
+		return 0, err
+	}
+
+	return h.Sum32(), nil
+}
+
+func (b *backend) Size() int64 {
+	return atomic.LoadInt64(&b.size)
+}
+
+func (b *backend) SizeInUse() int64 {
+	return atomic.LoadInt64(&b.sizeInUse)
+}
+
+func (b *backend) run() {
+	defer close(b.donec)
+	t := time.NewTimer(b.batchInterval)
+	defer t.Stop()
+	for {
+		select {
+		case <-t.C:
+		case <-b.stopc:
+			b.batchTx.CommitAndStop()
+			return
+		}
+		if b.batchTx.safePending() != 0 {
+			b.batchTx.Commit()
+		}
+		t.Reset(b.batchInterval)
+	}
+}
+
+func (b *backend) Close() error {
+	close(b.stopc)
+	<-b.donec
+	return b.db.Close()
+}
+
+// Commits returns total number of commits since start
+func (b *backend) Commits() int64 {
+	return atomic.LoadInt64(&b.commits)
+}
+
+func (b *backend) Defrag() error {
+	return b.defrag()
+}
+
+func (b *backend) defrag() error {
+	now := time.Now()
+
+	// TODO: make this non-blocking?
+	// lock batchTx to ensure nobody is using previous tx, and then
+	// close previous ongoing tx.
+	b.batchTx.Lock()
+	defer b.batchTx.Unlock()
+
+	// lock database after lock tx to avoid deadlock.
+	b.mu.Lock()
+	defer b.mu.Unlock()
+
+	// block concurrent read requests while resetting tx
+	b.readTx.Lock()
+	defer b.readTx.Unlock()
+
+	b.batchTx.unsafeCommit(true)
+
+	b.batchTx.tx = nil
+
+	tmpdb, err := bolt.Open(b.db.Path()+".tmp", 0600, boltOpenOptions)
+	if err != nil {
+		return err
+	}
+
+	dbp := b.db.Path()
+	tdbp := tmpdb.Path()
+	size1, sizeInUse1 := b.Size(), b.SizeInUse()
+	if b.lg != nil {
+		b.lg.Info(
+			"defragmenting",
+			zap.String("path", dbp),
+			zap.Int64("current-db-size-bytes", size1),
+			zap.String("current-db-size", humanize.Bytes(uint64(size1))),
+			zap.Int64("current-db-size-in-use-bytes", sizeInUse1),
+			zap.String("current-db-size-in-use", humanize.Bytes(uint64(sizeInUse1))),
+		)
+	}
+
+	err = defragdb(b.db, tmpdb, defragLimit)
+	if err != nil {
+		tmpdb.Close()
+		os.RemoveAll(tmpdb.Path())
+		return err
+	}
+
+	err = b.db.Close()
+	if err != nil {
+		if b.lg != nil {
+			b.lg.Fatal("failed to close database", zap.Error(err))
+		} else {
+			plog.Fatalf("cannot close database (%s)", err)
+		}
+	}
+	err = tmpdb.Close()
+	if err != nil {
+		if b.lg != nil {
+			b.lg.Fatal("failed to close tmp database", zap.Error(err))
+		} else {
+			plog.Fatalf("cannot close database (%s)", err)
+		}
+	}
+	err = os.Rename(tdbp, dbp)
+	if err != nil {
+		if b.lg != nil {
+			b.lg.Fatal("failed to rename tmp database", zap.Error(err))
+		} else {
+			plog.Fatalf("cannot rename database (%s)", err)
+		}
+	}
+
+	b.db, err = bolt.Open(dbp, 0600, boltOpenOptions)
+	if err != nil {
+		if b.lg != nil {
+			b.lg.Fatal("failed to open database", zap.String("path", dbp), zap.Error(err))
+		} else {
+			plog.Panicf("cannot open database at %s (%v)", dbp, err)
+		}
+	}
+	b.batchTx.tx = b.unsafeBegin(true)
+
+	b.readTx.reset()
+	b.readTx.tx = b.unsafeBegin(false)
+
+	size := b.readTx.tx.Size()
+	db := b.readTx.tx.DB()
+	atomic.StoreInt64(&b.size, size)
+	atomic.StoreInt64(&b.sizeInUse, size-(int64(db.Stats().FreePageN)*int64(db.Info().PageSize)))
+
+	took := time.Since(now)
+	defragSec.Observe(took.Seconds())
+
+	size2, sizeInUse2 := b.Size(), b.SizeInUse()
+	if b.lg != nil {
+		b.lg.Info(
+			"defragmented",
+			zap.String("path", dbp),
+			zap.Int64("current-db-size-bytes-diff", size2-size1),
+			zap.Int64("current-db-size-bytes", size2),
+			zap.String("current-db-size", humanize.Bytes(uint64(size2))),
+			zap.Int64("current-db-size-in-use-bytes-diff", sizeInUse2-sizeInUse1),
+			zap.Int64("current-db-size-in-use-bytes", sizeInUse2),
+			zap.String("current-db-size-in-use", humanize.Bytes(uint64(sizeInUse2))),
+			zap.Duration("took", took),
+		)
+	}
+	return nil
+}
+
+func defragdb(odb, tmpdb *bolt.DB, limit int) error {
+	// open a tx on tmpdb for writes
+	tmptx, err := tmpdb.Begin(true)
+	if err != nil {
+		return err
+	}
+
+	// open a tx on old db for read
+	tx, err := odb.Begin(false)
+	if err != nil {
+		return err
+	}
+	defer tx.Rollback()
+
+	c := tx.Cursor()
+
+	count := 0
+	for next, _ := c.First(); next != nil; next, _ = c.Next() {
+		b := tx.Bucket(next)
+		if b == nil {
+			return fmt.Errorf("backend: cannot defrag bucket %s", string(next))
+		}
+
+		tmpb, berr := tmptx.CreateBucketIfNotExists(next)
+		if berr != nil {
+			return berr
+		}
+		tmpb.FillPercent = 0.9 // for seq write in for each
+
+		b.ForEach(func(k, v []byte) error {
+			count++
+			if count > limit {
+				err = tmptx.Commit()
+				if err != nil {
+					return err
+				}
+				tmptx, err = tmpdb.Begin(true)
+				if err != nil {
+					return err
+				}
+				tmpb = tmptx.Bucket(next)
+				tmpb.FillPercent = 0.9 // for seq write in for each
+
+				count = 0
+			}
+			return tmpb.Put(k, v)
+		})
+	}
+
+	return tmptx.Commit()
+}
+
+func (b *backend) begin(write bool) *bolt.Tx {
+	b.mu.RLock()
+	tx := b.unsafeBegin(write)
+	b.mu.RUnlock()
+
+	size := tx.Size()
+	db := tx.DB()
+	stats := db.Stats()
+	atomic.StoreInt64(&b.size, size)
+	atomic.StoreInt64(&b.sizeInUse, size-(int64(stats.FreePageN)*int64(db.Info().PageSize)))
+	atomic.StoreInt64(&b.openReadTxN, int64(stats.OpenTxN))
+
+	return tx
+}
+
+func (b *backend) unsafeBegin(write bool) *bolt.Tx {
+	tx, err := b.db.Begin(write)
+	if err != nil {
+		if b.lg != nil {
+			b.lg.Fatal("failed to begin tx", zap.Error(err))
+		} else {
+			plog.Fatalf("cannot begin tx (%s)", err)
+		}
+	}
+	return tx
+}
+
+func (b *backend) OpenReadTxN() int64 {
+	return atomic.LoadInt64(&b.openReadTxN)
+}
+
+// NewTmpBackend creates a backend implementation for testing.
+func NewTmpBackend(batchInterval time.Duration, batchLimit int) (*backend, string) {
+	dir, err := ioutil.TempDir(os.TempDir(), "etcd_backend_test")
+	if err != nil {
+		panic(err)
+	}
+	tmpPath := filepath.Join(dir, "database")
+	bcfg := DefaultBackendConfig()
+	bcfg.Path, bcfg.BatchInterval, bcfg.BatchLimit = tmpPath, batchInterval, batchLimit
+	return newBackend(bcfg), tmpPath
+}
+
+func NewDefaultTmpBackend() (*backend, string) {
+	return NewTmpBackend(defaultBatchInterval, defaultBatchLimit)
+}
+
+type snapshot struct {
+	*bolt.Tx
+	stopc chan struct{}
+	donec chan struct{}
+}
+
+func (s *snapshot) Close() error {
+	close(s.stopc)
+	<-s.donec
+	return s.Tx.Rollback()
+}
diff --git a/vendor/go.etcd.io/etcd/mvcc/backend/batch_tx.go b/vendor/go.etcd.io/etcd/mvcc/backend/batch_tx.go
new file mode 100644
index 0000000..d5c8a88
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/backend/batch_tx.go
@@ -0,0 +1,339 @@
+// Copyright 2015 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 backend
+
+import (
+	"bytes"
+	"math"
+	"sync"
+	"sync/atomic"
+	"time"
+
+	bolt "go.etcd.io/bbolt"
+	"go.uber.org/zap"
+)
+
+type BatchTx interface {
+	ReadTx
+	UnsafeCreateBucket(name []byte)
+	UnsafePut(bucketName []byte, key []byte, value []byte)
+	UnsafeSeqPut(bucketName []byte, key []byte, value []byte)
+	UnsafeDelete(bucketName []byte, key []byte)
+	// Commit commits a previous tx and begins a new writable one.
+	Commit()
+	// CommitAndStop commits the previous tx and does not create a new one.
+	CommitAndStop()
+}
+
+type batchTx struct {
+	sync.Mutex
+	tx      *bolt.Tx
+	backend *backend
+
+	pending int
+}
+
+func (t *batchTx) Lock() {
+	t.Mutex.Lock()
+}
+
+func (t *batchTx) Unlock() {
+	if t.pending >= t.backend.batchLimit {
+		t.commit(false)
+	}
+	t.Mutex.Unlock()
+}
+
+// BatchTx interface embeds ReadTx interface. But RLock() and RUnlock() do not
+// have appropriate semantics in BatchTx interface. Therefore should not be called.
+// TODO: might want to decouple ReadTx and BatchTx
+
+func (t *batchTx) RLock() {
+	panic("unexpected RLock")
+}
+
+func (t *batchTx) RUnlock() {
+	panic("unexpected RUnlock")
+}
+
+func (t *batchTx) UnsafeCreateBucket(name []byte) {
+	_, err := t.tx.CreateBucket(name)
+	if err != nil && err != bolt.ErrBucketExists {
+		if t.backend.lg != nil {
+			t.backend.lg.Fatal(
+				"failed to create a bucket",
+				zap.String("bucket-name", string(name)),
+				zap.Error(err),
+			)
+		} else {
+			plog.Fatalf("cannot create bucket %s (%v)", name, err)
+		}
+	}
+	t.pending++
+}
+
+// UnsafePut must be called holding the lock on the tx.
+func (t *batchTx) UnsafePut(bucketName []byte, key []byte, value []byte) {
+	t.unsafePut(bucketName, key, value, false)
+}
+
+// UnsafeSeqPut must be called holding the lock on the tx.
+func (t *batchTx) UnsafeSeqPut(bucketName []byte, key []byte, value []byte) {
+	t.unsafePut(bucketName, key, value, true)
+}
+
+func (t *batchTx) unsafePut(bucketName []byte, key []byte, value []byte, seq bool) {
+	bucket := t.tx.Bucket(bucketName)
+	if bucket == nil {
+		if t.backend.lg != nil {
+			t.backend.lg.Fatal(
+				"failed to find a bucket",
+				zap.String("bucket-name", string(bucketName)),
+			)
+		} else {
+			plog.Fatalf("bucket %s does not exist", bucketName)
+		}
+	}
+	if seq {
+		// it is useful to increase fill percent when the workloads are mostly append-only.
+		// this can delay the page split and reduce space usage.
+		bucket.FillPercent = 0.9
+	}
+	if err := bucket.Put(key, value); err != nil {
+		if t.backend.lg != nil {
+			t.backend.lg.Fatal(
+				"failed to write to a bucket",
+				zap.String("bucket-name", string(bucketName)),
+				zap.Error(err),
+			)
+		} else {
+			plog.Fatalf("cannot put key into bucket (%v)", err)
+		}
+	}
+	t.pending++
+}
+
+// UnsafeRange must be called holding the lock on the tx.
+func (t *batchTx) UnsafeRange(bucketName, key, endKey []byte, limit int64) ([][]byte, [][]byte) {
+	bucket := t.tx.Bucket(bucketName)
+	if bucket == nil {
+		if t.backend.lg != nil {
+			t.backend.lg.Fatal(
+				"failed to find a bucket",
+				zap.String("bucket-name", string(bucketName)),
+			)
+		} else {
+			plog.Fatalf("bucket %s does not exist", bucketName)
+		}
+	}
+	return unsafeRange(bucket.Cursor(), key, endKey, limit)
+}
+
+func unsafeRange(c *bolt.Cursor, key, endKey []byte, limit int64) (keys [][]byte, vs [][]byte) {
+	if limit <= 0 {
+		limit = math.MaxInt64
+	}
+	var isMatch func(b []byte) bool
+	if len(endKey) > 0 {
+		isMatch = func(b []byte) bool { return bytes.Compare(b, endKey) < 0 }
+	} else {
+		isMatch = func(b []byte) bool { return bytes.Equal(b, key) }
+		limit = 1
+	}
+
+	for ck, cv := c.Seek(key); ck != nil && isMatch(ck); ck, cv = c.Next() {
+		vs = append(vs, cv)
+		keys = append(keys, ck)
+		if limit == int64(len(keys)) {
+			break
+		}
+	}
+	return keys, vs
+}
+
+// UnsafeDelete must be called holding the lock on the tx.
+func (t *batchTx) UnsafeDelete(bucketName []byte, key []byte) {
+	bucket := t.tx.Bucket(bucketName)
+	if bucket == nil {
+		if t.backend.lg != nil {
+			t.backend.lg.Fatal(
+				"failed to find a bucket",
+				zap.String("bucket-name", string(bucketName)),
+			)
+		} else {
+			plog.Fatalf("bucket %s does not exist", bucketName)
+		}
+	}
+	err := bucket.Delete(key)
+	if err != nil {
+		if t.backend.lg != nil {
+			t.backend.lg.Fatal(
+				"failed to delete a key",
+				zap.String("bucket-name", string(bucketName)),
+				zap.Error(err),
+			)
+		} else {
+			plog.Fatalf("cannot delete key from bucket (%v)", err)
+		}
+	}
+	t.pending++
+}
+
+// UnsafeForEach must be called holding the lock on the tx.
+func (t *batchTx) UnsafeForEach(bucketName []byte, visitor func(k, v []byte) error) error {
+	return unsafeForEach(t.tx, bucketName, visitor)
+}
+
+func unsafeForEach(tx *bolt.Tx, bucket []byte, visitor func(k, v []byte) error) error {
+	if b := tx.Bucket(bucket); b != nil {
+		return b.ForEach(visitor)
+	}
+	return nil
+}
+
+// Commit commits a previous tx and begins a new writable one.
+func (t *batchTx) Commit() {
+	t.Lock()
+	t.commit(false)
+	t.Unlock()
+}
+
+// CommitAndStop commits the previous tx and does not create a new one.
+func (t *batchTx) CommitAndStop() {
+	t.Lock()
+	t.commit(true)
+	t.Unlock()
+}
+
+func (t *batchTx) safePending() int {
+	t.Mutex.Lock()
+	defer t.Mutex.Unlock()
+	return t.pending
+}
+
+func (t *batchTx) commit(stop bool) {
+	// commit the last tx
+	if t.tx != nil {
+		if t.pending == 0 && !stop {
+			return
+		}
+
+		start := time.Now()
+
+		// gofail: var beforeCommit struct{}
+		err := t.tx.Commit()
+		// gofail: var afterCommit struct{}
+
+		rebalanceSec.Observe(t.tx.Stats().RebalanceTime.Seconds())
+		spillSec.Observe(t.tx.Stats().SpillTime.Seconds())
+		writeSec.Observe(t.tx.Stats().WriteTime.Seconds())
+		commitSec.Observe(time.Since(start).Seconds())
+		atomic.AddInt64(&t.backend.commits, 1)
+
+		t.pending = 0
+		if err != nil {
+			if t.backend.lg != nil {
+				t.backend.lg.Fatal("failed to commit tx", zap.Error(err))
+			} else {
+				plog.Fatalf("cannot commit tx (%s)", err)
+			}
+		}
+	}
+	if !stop {
+		t.tx = t.backend.begin(true)
+	}
+}
+
+type batchTxBuffered struct {
+	batchTx
+	buf txWriteBuffer
+}
+
+func newBatchTxBuffered(backend *backend) *batchTxBuffered {
+	tx := &batchTxBuffered{
+		batchTx: batchTx{backend: backend},
+		buf: txWriteBuffer{
+			txBuffer: txBuffer{make(map[string]*bucketBuffer)},
+			seq:      true,
+		},
+	}
+	tx.Commit()
+	return tx
+}
+
+func (t *batchTxBuffered) Unlock() {
+	if t.pending != 0 {
+		t.backend.readTx.Lock() // blocks txReadBuffer for writing.
+		t.buf.writeback(&t.backend.readTx.buf)
+		t.backend.readTx.Unlock()
+		if t.pending >= t.backend.batchLimit {
+			t.commit(false)
+		}
+	}
+	t.batchTx.Unlock()
+}
+
+func (t *batchTxBuffered) Commit() {
+	t.Lock()
+	t.commit(false)
+	t.Unlock()
+}
+
+func (t *batchTxBuffered) CommitAndStop() {
+	t.Lock()
+	t.commit(true)
+	t.Unlock()
+}
+
+func (t *batchTxBuffered) commit(stop bool) {
+	// all read txs must be closed to acquire boltdb commit rwlock
+	t.backend.readTx.Lock()
+	t.unsafeCommit(stop)
+	t.backend.readTx.Unlock()
+}
+
+func (t *batchTxBuffered) unsafeCommit(stop bool) {
+	if t.backend.readTx.tx != nil {
+		// wait all store read transactions using the current boltdb tx to finish,
+		// then close the boltdb tx
+		go func(tx *bolt.Tx, wg *sync.WaitGroup) {
+			wg.Wait()
+			if err := tx.Rollback(); err != nil {
+				if t.backend.lg != nil {
+					t.backend.lg.Fatal("failed to rollback tx", zap.Error(err))
+				} else {
+					plog.Fatalf("cannot rollback tx (%s)", err)
+				}
+			}
+		}(t.backend.readTx.tx, t.backend.readTx.txWg)
+		t.backend.readTx.reset()
+	}
+
+	t.batchTx.commit(stop)
+
+	if !stop {
+		t.backend.readTx.tx = t.backend.begin(false)
+	}
+}
+
+func (t *batchTxBuffered) UnsafePut(bucketName []byte, key []byte, value []byte) {
+	t.batchTx.UnsafePut(bucketName, key, value)
+	t.buf.put(bucketName, key, value)
+}
+
+func (t *batchTxBuffered) UnsafeSeqPut(bucketName []byte, key []byte, value []byte) {
+	t.batchTx.UnsafeSeqPut(bucketName, key, value)
+	t.buf.putSeq(bucketName, key, value)
+}
diff --git a/vendor/go.etcd.io/etcd/mvcc/backend/config_default.go b/vendor/go.etcd.io/etcd/mvcc/backend/config_default.go
new file mode 100644
index 0000000..f15f030
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/backend/config_default.go
@@ -0,0 +1,23 @@
+// Copyright 2016 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.
+
+// +build !linux,!windows
+
+package backend
+
+import bolt "go.etcd.io/bbolt"
+
+var boltOpenOptions *bolt.Options
+
+func (bcfg *BackendConfig) mmapSize() int { return int(bcfg.MmapSize) }
diff --git a/vendor/go.etcd.io/etcd/mvcc/backend/config_linux.go b/vendor/go.etcd.io/etcd/mvcc/backend/config_linux.go
new file mode 100644
index 0000000..f712671
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/backend/config_linux.go
@@ -0,0 +1,34 @@
+// Copyright 2015 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 backend
+
+import (
+	"syscall"
+
+	bolt "go.etcd.io/bbolt"
+)
+
+// syscall.MAP_POPULATE on linux 2.6.23+ does sequential read-ahead
+// which can speed up entire-database read with boltdb. We want to
+// enable MAP_POPULATE for faster key-value store recovery in storage
+// package. If your kernel version is lower than 2.6.23
+// (https://github.com/torvalds/linux/releases/tag/v2.6.23), mmap might
+// silently ignore this flag. Please update your kernel to prevent this.
+var boltOpenOptions = &bolt.Options{
+	MmapFlags:      syscall.MAP_POPULATE,
+	NoFreelistSync: true,
+}
+
+func (bcfg *BackendConfig) mmapSize() int { return int(bcfg.MmapSize) }
diff --git a/vendor/go.etcd.io/etcd/mvcc/backend/config_windows.go b/vendor/go.etcd.io/etcd/mvcc/backend/config_windows.go
new file mode 100644
index 0000000..c650059
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/backend/config_windows.go
@@ -0,0 +1,26 @@
+// 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.
+
+// +build windows
+
+package backend
+
+import bolt "go.etcd.io/bbolt"
+
+var boltOpenOptions *bolt.Options = nil
+
+// setting mmap size != 0 on windows will allocate the entire
+// mmap size for the file, instead of growing it. So, force 0.
+
+func (bcfg *BackendConfig) mmapSize() int { return 0 }
diff --git a/vendor/go.etcd.io/etcd/mvcc/backend/doc.go b/vendor/go.etcd.io/etcd/mvcc/backend/doc.go
new file mode 100644
index 0000000..9cc42fa
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/backend/doc.go
@@ -0,0 +1,16 @@
+// Copyright 2015 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 backend defines a standard interface for etcd's backend MVCC storage.
+package backend
diff --git a/vendor/go.etcd.io/etcd/mvcc/backend/metrics.go b/vendor/go.etcd.io/etcd/mvcc/backend/metrics.go
new file mode 100644
index 0000000..d9641af
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/backend/metrics.go
@@ -0,0 +1,95 @@
+// Copyright 2016 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 backend
+
+import "github.com/prometheus/client_golang/prometheus"
+
+var (
+	commitSec = prometheus.NewHistogram(prometheus.HistogramOpts{
+		Namespace: "etcd",
+		Subsystem: "disk",
+		Name:      "backend_commit_duration_seconds",
+		Help:      "The latency distributions of commit called by backend.",
+
+		// lowest bucket start of upper bound 0.001 sec (1 ms) with factor 2
+		// highest bucket start of 0.001 sec * 2^13 == 8.192 sec
+		Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
+	})
+
+	rebalanceSec = prometheus.NewHistogram(prometheus.HistogramOpts{
+		Namespace: "etcd_debugging",
+		Subsystem: "disk",
+		Name:      "backend_commit_rebalance_duration_seconds",
+		Help:      "The latency distributions of commit.rebalance called by bboltdb backend.",
+
+		// lowest bucket start of upper bound 0.001 sec (1 ms) with factor 2
+		// highest bucket start of 0.001 sec * 2^13 == 8.192 sec
+		Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
+	})
+
+	spillSec = prometheus.NewHistogram(prometheus.HistogramOpts{
+		Namespace: "etcd_debugging",
+		Subsystem: "disk",
+		Name:      "backend_commit_spill_duration_seconds",
+		Help:      "The latency distributions of commit.spill called by bboltdb backend.",
+
+		// lowest bucket start of upper bound 0.001 sec (1 ms) with factor 2
+		// highest bucket start of 0.001 sec * 2^13 == 8.192 sec
+		Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
+	})
+
+	writeSec = prometheus.NewHistogram(prometheus.HistogramOpts{
+		Namespace: "etcd_debugging",
+		Subsystem: "disk",
+		Name:      "backend_commit_write_duration_seconds",
+		Help:      "The latency distributions of commit.write called by bboltdb backend.",
+
+		// lowest bucket start of upper bound 0.001 sec (1 ms) with factor 2
+		// highest bucket start of 0.001 sec * 2^13 == 8.192 sec
+		Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
+	})
+
+	defragSec = prometheus.NewHistogram(prometheus.HistogramOpts{
+		Namespace: "etcd",
+		Subsystem: "disk",
+		Name:      "backend_defrag_duration_seconds",
+		Help:      "The latency distribution of backend defragmentation.",
+
+		// 100 MB usually takes 1 sec, so start with 10 MB of 100 ms
+		// lowest bucket start of upper bound 0.1 sec (100 ms) with factor 2
+		// highest bucket start of 0.1 sec * 2^12 == 409.6 sec
+		Buckets: prometheus.ExponentialBuckets(.1, 2, 13),
+	})
+
+	snapshotTransferSec = prometheus.NewHistogram(prometheus.HistogramOpts{
+		Namespace: "etcd",
+		Subsystem: "disk",
+		Name:      "backend_snapshot_duration_seconds",
+		Help:      "The latency distribution of backend snapshots.",
+
+		// lowest bucket start of upper bound 0.01 sec (10 ms) with factor 2
+		// highest bucket start of 0.01 sec * 2^16 == 655.36 sec
+		Buckets: prometheus.ExponentialBuckets(.01, 2, 17),
+	})
+)
+
+func init() {
+	prometheus.MustRegister(commitSec)
+	prometheus.MustRegister(rebalanceSec)
+	prometheus.MustRegister(spillSec)
+	prometheus.MustRegister(writeSec)
+	prometheus.MustRegister(defragSec)
+	prometheus.MustRegister(snapshotTransferSec)
+}
diff --git a/vendor/go.etcd.io/etcd/mvcc/backend/read_tx.go b/vendor/go.etcd.io/etcd/mvcc/backend/read_tx.go
new file mode 100644
index 0000000..91fe72e
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/backend/read_tx.go
@@ -0,0 +1,210 @@
+// 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 backend
+
+import (
+	"bytes"
+	"math"
+	"sync"
+
+	bolt "go.etcd.io/bbolt"
+)
+
+// safeRangeBucket is a hack to avoid inadvertently reading duplicate keys;
+// overwrites on a bucket should only fetch with limit=1, but safeRangeBucket
+// is known to never overwrite any key so range is safe.
+var safeRangeBucket = []byte("key")
+
+type ReadTx interface {
+	Lock()
+	Unlock()
+	RLock()
+	RUnlock()
+
+	UnsafeRange(bucketName []byte, key, endKey []byte, limit int64) (keys [][]byte, vals [][]byte)
+	UnsafeForEach(bucketName []byte, visitor func(k, v []byte) error) error
+}
+
+type readTx struct {
+	// mu protects accesses to the txReadBuffer
+	mu  sync.RWMutex
+	buf txReadBuffer
+
+	// TODO: group and encapsulate {txMu, tx, buckets, txWg}, as they share the same lifecycle.
+	// txMu protects accesses to buckets and tx on Range requests.
+	txMu    sync.RWMutex
+	tx      *bolt.Tx
+	buckets map[string]*bolt.Bucket
+	// txWg protects tx from being rolled back at the end of a batch interval until all reads using this tx are done.
+	txWg *sync.WaitGroup
+}
+
+func (rt *readTx) Lock()    { rt.mu.Lock() }
+func (rt *readTx) Unlock()  { rt.mu.Unlock() }
+func (rt *readTx) RLock()   { rt.mu.RLock() }
+func (rt *readTx) RUnlock() { rt.mu.RUnlock() }
+
+func (rt *readTx) UnsafeRange(bucketName, key, endKey []byte, limit int64) ([][]byte, [][]byte) {
+	if endKey == nil {
+		// forbid duplicates for single keys
+		limit = 1
+	}
+	if limit <= 0 {
+		limit = math.MaxInt64
+	}
+	if limit > 1 && !bytes.Equal(bucketName, safeRangeBucket) {
+		panic("do not use unsafeRange on non-keys bucket")
+	}
+	keys, vals := rt.buf.Range(bucketName, key, endKey, limit)
+	if int64(len(keys)) == limit {
+		return keys, vals
+	}
+
+	// find/cache bucket
+	bn := string(bucketName)
+	rt.txMu.RLock()
+	bucket, ok := rt.buckets[bn]
+	rt.txMu.RUnlock()
+	if !ok {
+		rt.txMu.Lock()
+		bucket = rt.tx.Bucket(bucketName)
+		rt.buckets[bn] = bucket
+		rt.txMu.Unlock()
+	}
+
+	// ignore missing bucket since may have been created in this batch
+	if bucket == nil {
+		return keys, vals
+	}
+	rt.txMu.Lock()
+	c := bucket.Cursor()
+	rt.txMu.Unlock()
+
+	k2, v2 := unsafeRange(c, key, endKey, limit-int64(len(keys)))
+	return append(k2, keys...), append(v2, vals...)
+}
+
+func (rt *readTx) UnsafeForEach(bucketName []byte, visitor func(k, v []byte) error) error {
+	dups := make(map[string]struct{})
+	getDups := func(k, v []byte) error {
+		dups[string(k)] = struct{}{}
+		return nil
+	}
+	visitNoDup := func(k, v []byte) error {
+		if _, ok := dups[string(k)]; ok {
+			return nil
+		}
+		return visitor(k, v)
+	}
+	if err := rt.buf.ForEach(bucketName, getDups); err != nil {
+		return err
+	}
+	rt.txMu.Lock()
+	err := unsafeForEach(rt.tx, bucketName, visitNoDup)
+	rt.txMu.Unlock()
+	if err != nil {
+		return err
+	}
+	return rt.buf.ForEach(bucketName, visitor)
+}
+
+func (rt *readTx) reset() {
+	rt.buf.reset()
+	rt.buckets = make(map[string]*bolt.Bucket)
+	rt.tx = nil
+	rt.txWg = new(sync.WaitGroup)
+}
+
+// TODO: create a base type for readTx and concurrentReadTx to avoid duplicated function implementation?
+type concurrentReadTx struct {
+	buf     txReadBuffer
+	txMu    *sync.RWMutex
+	tx      *bolt.Tx
+	buckets map[string]*bolt.Bucket
+	txWg    *sync.WaitGroup
+}
+
+func (rt *concurrentReadTx) Lock()   {}
+func (rt *concurrentReadTx) Unlock() {}
+
+// RLock is no-op. concurrentReadTx does not need to be locked after it is created.
+func (rt *concurrentReadTx) RLock() {}
+
+// RUnlock signals the end of concurrentReadTx.
+func (rt *concurrentReadTx) RUnlock() { rt.txWg.Done() }
+
+func (rt *concurrentReadTx) UnsafeForEach(bucketName []byte, visitor func(k, v []byte) error) error {
+	dups := make(map[string]struct{})
+	getDups := func(k, v []byte) error {
+		dups[string(k)] = struct{}{}
+		return nil
+	}
+	visitNoDup := func(k, v []byte) error {
+		if _, ok := dups[string(k)]; ok {
+			return nil
+		}
+		return visitor(k, v)
+	}
+	if err := rt.buf.ForEach(bucketName, getDups); err != nil {
+		return err
+	}
+	rt.txMu.Lock()
+	err := unsafeForEach(rt.tx, bucketName, visitNoDup)
+	rt.txMu.Unlock()
+	if err != nil {
+		return err
+	}
+	return rt.buf.ForEach(bucketName, visitor)
+}
+
+func (rt *concurrentReadTx) UnsafeRange(bucketName, key, endKey []byte, limit int64) ([][]byte, [][]byte) {
+	if endKey == nil {
+		// forbid duplicates for single keys
+		limit = 1
+	}
+	if limit <= 0 {
+		limit = math.MaxInt64
+	}
+	if limit > 1 && !bytes.Equal(bucketName, safeRangeBucket) {
+		panic("do not use unsafeRange on non-keys bucket")
+	}
+	keys, vals := rt.buf.Range(bucketName, key, endKey, limit)
+	if int64(len(keys)) == limit {
+		return keys, vals
+	}
+
+	// find/cache bucket
+	bn := string(bucketName)
+	rt.txMu.RLock()
+	bucket, ok := rt.buckets[bn]
+	rt.txMu.RUnlock()
+	if !ok {
+		rt.txMu.Lock()
+		bucket = rt.tx.Bucket(bucketName)
+		rt.buckets[bn] = bucket
+		rt.txMu.Unlock()
+	}
+
+	// ignore missing bucket since may have been created in this batch
+	if bucket == nil {
+		return keys, vals
+	}
+	rt.txMu.Lock()
+	c := bucket.Cursor()
+	rt.txMu.Unlock()
+
+	k2, v2 := unsafeRange(c, key, endKey, limit-int64(len(keys)))
+	return append(k2, keys...), append(v2, vals...)
+}
diff --git a/vendor/go.etcd.io/etcd/mvcc/backend/tx_buffer.go b/vendor/go.etcd.io/etcd/mvcc/backend/tx_buffer.go
new file mode 100644
index 0000000..d734638
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/backend/tx_buffer.go
@@ -0,0 +1,203 @@
+// 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 backend
+
+import (
+	"bytes"
+	"sort"
+)
+
+// txBuffer handles functionality shared between txWriteBuffer and txReadBuffer.
+type txBuffer struct {
+	buckets map[string]*bucketBuffer
+}
+
+func (txb *txBuffer) reset() {
+	for k, v := range txb.buckets {
+		if v.used == 0 {
+			// demote
+			delete(txb.buckets, k)
+		}
+		v.used = 0
+	}
+}
+
+// txWriteBuffer buffers writes of pending updates that have not yet committed.
+type txWriteBuffer struct {
+	txBuffer
+	seq bool
+}
+
+func (txw *txWriteBuffer) put(bucket, k, v []byte) {
+	txw.seq = false
+	txw.putSeq(bucket, k, v)
+}
+
+func (txw *txWriteBuffer) putSeq(bucket, k, v []byte) {
+	b, ok := txw.buckets[string(bucket)]
+	if !ok {
+		b = newBucketBuffer()
+		txw.buckets[string(bucket)] = b
+	}
+	b.add(k, v)
+}
+
+func (txw *txWriteBuffer) writeback(txr *txReadBuffer) {
+	for k, wb := range txw.buckets {
+		rb, ok := txr.buckets[k]
+		if !ok {
+			delete(txw.buckets, k)
+			txr.buckets[k] = wb
+			continue
+		}
+		if !txw.seq && wb.used > 1 {
+			// assume no duplicate keys
+			sort.Sort(wb)
+		}
+		rb.merge(wb)
+	}
+	txw.reset()
+}
+
+// txReadBuffer accesses buffered updates.
+type txReadBuffer struct{ txBuffer }
+
+func (txr *txReadBuffer) Range(bucketName, key, endKey []byte, limit int64) ([][]byte, [][]byte) {
+	if b := txr.buckets[string(bucketName)]; b != nil {
+		return b.Range(key, endKey, limit)
+	}
+	return nil, nil
+}
+
+func (txr *txReadBuffer) ForEach(bucketName []byte, visitor func(k, v []byte) error) error {
+	if b := txr.buckets[string(bucketName)]; b != nil {
+		return b.ForEach(visitor)
+	}
+	return nil
+}
+
+// unsafeCopy returns a copy of txReadBuffer, caller should acquire backend.readTx.RLock()
+func (txr *txReadBuffer) unsafeCopy() txReadBuffer {
+	txrCopy := txReadBuffer{
+		txBuffer: txBuffer{
+			buckets: make(map[string]*bucketBuffer, len(txr.txBuffer.buckets)),
+		},
+	}
+	for bucketName, bucket := range txr.txBuffer.buckets {
+		txrCopy.txBuffer.buckets[bucketName] = bucket.Copy()
+	}
+	return txrCopy
+}
+
+type kv struct {
+	key []byte
+	val []byte
+}
+
+// bucketBuffer buffers key-value pairs that are pending commit.
+type bucketBuffer struct {
+	buf []kv
+	// used tracks number of elements in use so buf can be reused without reallocation.
+	used int
+}
+
+func newBucketBuffer() *bucketBuffer {
+	return &bucketBuffer{buf: make([]kv, 512), used: 0}
+}
+
+func (bb *bucketBuffer) Range(key, endKey []byte, limit int64) (keys [][]byte, vals [][]byte) {
+	f := func(i int) bool { return bytes.Compare(bb.buf[i].key, key) >= 0 }
+	idx := sort.Search(bb.used, f)
+	if idx < 0 {
+		return nil, nil
+	}
+	if len(endKey) == 0 {
+		if bytes.Equal(key, bb.buf[idx].key) {
+			keys = append(keys, bb.buf[idx].key)
+			vals = append(vals, bb.buf[idx].val)
+		}
+		return keys, vals
+	}
+	if bytes.Compare(endKey, bb.buf[idx].key) <= 0 {
+		return nil, nil
+	}
+	for i := idx; i < bb.used && int64(len(keys)) < limit; i++ {
+		if bytes.Compare(endKey, bb.buf[i].key) <= 0 {
+			break
+		}
+		keys = append(keys, bb.buf[i].key)
+		vals = append(vals, bb.buf[i].val)
+	}
+	return keys, vals
+}
+
+func (bb *bucketBuffer) ForEach(visitor func(k, v []byte) error) error {
+	for i := 0; i < bb.used; i++ {
+		if err := visitor(bb.buf[i].key, bb.buf[i].val); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+func (bb *bucketBuffer) add(k, v []byte) {
+	bb.buf[bb.used].key, bb.buf[bb.used].val = k, v
+	bb.used++
+	if bb.used == len(bb.buf) {
+		buf := make([]kv, (3*len(bb.buf))/2)
+		copy(buf, bb.buf)
+		bb.buf = buf
+	}
+}
+
+// merge merges data from bb into bbsrc.
+func (bb *bucketBuffer) merge(bbsrc *bucketBuffer) {
+	for i := 0; i < bbsrc.used; i++ {
+		bb.add(bbsrc.buf[i].key, bbsrc.buf[i].val)
+	}
+	if bb.used == bbsrc.used {
+		return
+	}
+	if bytes.Compare(bb.buf[(bb.used-bbsrc.used)-1].key, bbsrc.buf[0].key) < 0 {
+		return
+	}
+
+	sort.Stable(bb)
+
+	// remove duplicates, using only newest update
+	widx := 0
+	for ridx := 1; ridx < bb.used; ridx++ {
+		if !bytes.Equal(bb.buf[ridx].key, bb.buf[widx].key) {
+			widx++
+		}
+		bb.buf[widx] = bb.buf[ridx]
+	}
+	bb.used = widx + 1
+}
+
+func (bb *bucketBuffer) Len() int { return bb.used }
+func (bb *bucketBuffer) Less(i, j int) bool {
+	return bytes.Compare(bb.buf[i].key, bb.buf[j].key) < 0
+}
+func (bb *bucketBuffer) Swap(i, j int) { bb.buf[i], bb.buf[j] = bb.buf[j], bb.buf[i] }
+
+func (bb *bucketBuffer) Copy() *bucketBuffer {
+	bbCopy := bucketBuffer{
+		buf:  make([]kv, len(bb.buf)),
+		used: bb.used,
+	}
+	copy(bbCopy.buf, bb.buf)
+	return &bbCopy
+}
diff --git a/vendor/go.etcd.io/etcd/mvcc/doc.go b/vendor/go.etcd.io/etcd/mvcc/doc.go
new file mode 100644
index 0000000..ad5be03
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/doc.go
@@ -0,0 +1,16 @@
+// Copyright 2015 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 mvcc defines etcd's stable MVCC storage.
+package mvcc
diff --git a/vendor/go.etcd.io/etcd/mvcc/index.go b/vendor/go.etcd.io/etcd/mvcc/index.go
new file mode 100644
index 0000000..f8cc6df
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/index.go
@@ -0,0 +1,258 @@
+// Copyright 2015 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 mvcc
+
+import (
+	"sort"
+	"sync"
+
+	"github.com/google/btree"
+	"go.uber.org/zap"
+)
+
+type index interface {
+	Get(key []byte, atRev int64) (rev, created revision, ver int64, err error)
+	Range(key, end []byte, atRev int64) ([][]byte, []revision)
+	Revisions(key, end []byte, atRev int64) []revision
+	Put(key []byte, rev revision)
+	Tombstone(key []byte, rev revision) error
+	RangeSince(key, end []byte, rev int64) []revision
+	Compact(rev int64) map[revision]struct{}
+	Keep(rev int64) map[revision]struct{}
+	Equal(b index) bool
+
+	Insert(ki *keyIndex)
+	KeyIndex(ki *keyIndex) *keyIndex
+}
+
+type treeIndex struct {
+	sync.RWMutex
+	tree *btree.BTree
+	lg   *zap.Logger
+}
+
+func newTreeIndex(lg *zap.Logger) index {
+	return &treeIndex{
+		tree: btree.New(32),
+		lg:   lg,
+	}
+}
+
+func (ti *treeIndex) Put(key []byte, rev revision) {
+	keyi := &keyIndex{key: key}
+
+	ti.Lock()
+	defer ti.Unlock()
+	item := ti.tree.Get(keyi)
+	if item == nil {
+		keyi.put(ti.lg, rev.main, rev.sub)
+		ti.tree.ReplaceOrInsert(keyi)
+		return
+	}
+	okeyi := item.(*keyIndex)
+	okeyi.put(ti.lg, rev.main, rev.sub)
+}
+
+func (ti *treeIndex) Get(key []byte, atRev int64) (modified, created revision, ver int64, err error) {
+	keyi := &keyIndex{key: key}
+	ti.RLock()
+	defer ti.RUnlock()
+	if keyi = ti.keyIndex(keyi); keyi == nil {
+		return revision{}, revision{}, 0, ErrRevisionNotFound
+	}
+	return keyi.get(ti.lg, atRev)
+}
+
+func (ti *treeIndex) KeyIndex(keyi *keyIndex) *keyIndex {
+	ti.RLock()
+	defer ti.RUnlock()
+	return ti.keyIndex(keyi)
+}
+
+func (ti *treeIndex) keyIndex(keyi *keyIndex) *keyIndex {
+	if item := ti.tree.Get(keyi); item != nil {
+		return item.(*keyIndex)
+	}
+	return nil
+}
+
+func (ti *treeIndex) visit(key, end []byte, f func(ki *keyIndex)) {
+	keyi, endi := &keyIndex{key: key}, &keyIndex{key: end}
+
+	ti.RLock()
+	defer ti.RUnlock()
+
+	ti.tree.AscendGreaterOrEqual(keyi, func(item btree.Item) bool {
+		if len(endi.key) > 0 && !item.Less(endi) {
+			return false
+		}
+		f(item.(*keyIndex))
+		return true
+	})
+}
+
+func (ti *treeIndex) Revisions(key, end []byte, atRev int64) (revs []revision) {
+	if end == nil {
+		rev, _, _, err := ti.Get(key, atRev)
+		if err != nil {
+			return nil
+		}
+		return []revision{rev}
+	}
+	ti.visit(key, end, func(ki *keyIndex) {
+		if rev, _, _, err := ki.get(ti.lg, atRev); err == nil {
+			revs = append(revs, rev)
+		}
+	})
+	return revs
+}
+
+func (ti *treeIndex) Range(key, end []byte, atRev int64) (keys [][]byte, revs []revision) {
+	if end == nil {
+		rev, _, _, err := ti.Get(key, atRev)
+		if err != nil {
+			return nil, nil
+		}
+		return [][]byte{key}, []revision{rev}
+	}
+	ti.visit(key, end, func(ki *keyIndex) {
+		if rev, _, _, err := ki.get(ti.lg, atRev); err == nil {
+			revs = append(revs, rev)
+			keys = append(keys, ki.key)
+		}
+	})
+	return keys, revs
+}
+
+func (ti *treeIndex) Tombstone(key []byte, rev revision) error {
+	keyi := &keyIndex{key: key}
+
+	ti.Lock()
+	defer ti.Unlock()
+	item := ti.tree.Get(keyi)
+	if item == nil {
+		return ErrRevisionNotFound
+	}
+
+	ki := item.(*keyIndex)
+	return ki.tombstone(ti.lg, rev.main, rev.sub)
+}
+
+// RangeSince returns all revisions from key(including) to end(excluding)
+// at or after the given rev. The returned slice is sorted in the order
+// of revision.
+func (ti *treeIndex) RangeSince(key, end []byte, rev int64) []revision {
+	keyi := &keyIndex{key: key}
+
+	ti.RLock()
+	defer ti.RUnlock()
+
+	if end == nil {
+		item := ti.tree.Get(keyi)
+		if item == nil {
+			return nil
+		}
+		keyi = item.(*keyIndex)
+		return keyi.since(ti.lg, rev)
+	}
+
+	endi := &keyIndex{key: end}
+	var revs []revision
+	ti.tree.AscendGreaterOrEqual(keyi, func(item btree.Item) bool {
+		if len(endi.key) > 0 && !item.Less(endi) {
+			return false
+		}
+		curKeyi := item.(*keyIndex)
+		revs = append(revs, curKeyi.since(ti.lg, rev)...)
+		return true
+	})
+	sort.Sort(revisions(revs))
+
+	return revs
+}
+
+func (ti *treeIndex) Compact(rev int64) map[revision]struct{} {
+	available := make(map[revision]struct{})
+	if ti.lg != nil {
+		ti.lg.Info("compact tree index", zap.Int64("revision", rev))
+	} else {
+		plog.Printf("store.index: compact %d", rev)
+	}
+	ti.Lock()
+	clone := ti.tree.Clone()
+	ti.Unlock()
+
+	clone.Ascend(func(item btree.Item) bool {
+		keyi := item.(*keyIndex)
+		//Lock is needed here to prevent modification to the keyIndex while
+		//compaction is going on or revision added to empty before deletion
+		ti.Lock()
+		keyi.compact(ti.lg, rev, available)
+		if keyi.isEmpty() {
+			item := ti.tree.Delete(keyi)
+			if item == nil {
+				if ti.lg != nil {
+					ti.lg.Panic("failed to delete during compaction")
+				} else {
+					plog.Panic("store.index: unexpected delete failure during compaction")
+				}
+			}
+		}
+		ti.Unlock()
+		return true
+	})
+	return available
+}
+
+// Keep finds all revisions to be kept for a Compaction at the given rev.
+func (ti *treeIndex) Keep(rev int64) map[revision]struct{} {
+	available := make(map[revision]struct{})
+	ti.RLock()
+	defer ti.RUnlock()
+	ti.tree.Ascend(func(i btree.Item) bool {
+		keyi := i.(*keyIndex)
+		keyi.keep(rev, available)
+		return true
+	})
+	return available
+}
+
+func (ti *treeIndex) Equal(bi index) bool {
+	b := bi.(*treeIndex)
+
+	if ti.tree.Len() != b.tree.Len() {
+		return false
+	}
+
+	equal := true
+
+	ti.tree.Ascend(func(item btree.Item) bool {
+		aki := item.(*keyIndex)
+		bki := b.tree.Get(item).(*keyIndex)
+		if !aki.equal(bki) {
+			equal = false
+			return false
+		}
+		return true
+	})
+
+	return equal
+}
+
+func (ti *treeIndex) Insert(ki *keyIndex) {
+	ti.Lock()
+	defer ti.Unlock()
+	ti.tree.ReplaceOrInsert(ki)
+}
diff --git a/vendor/go.etcd.io/etcd/mvcc/key_index.go b/vendor/go.etcd.io/etcd/mvcc/key_index.go
new file mode 100644
index 0000000..cf77cb4
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/key_index.go
@@ -0,0 +1,402 @@
+// Copyright 2015 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 mvcc
+
+import (
+	"bytes"
+	"errors"
+	"fmt"
+
+	"github.com/google/btree"
+	"go.uber.org/zap"
+)
+
+var (
+	ErrRevisionNotFound = errors.New("mvcc: revision not found")
+)
+
+// keyIndex stores the revisions of a key in the backend.
+// Each keyIndex has at least one key generation.
+// Each generation might have several key versions.
+// Tombstone on a key appends an tombstone version at the end
+// of the current generation and creates a new empty generation.
+// Each version of a key has an index pointing to the backend.
+//
+// For example: put(1.0);put(2.0);tombstone(3.0);put(4.0);tombstone(5.0) on key "foo"
+// generate a keyIndex:
+// key:     "foo"
+// rev: 5
+// generations:
+//    {empty}
+//    {4.0, 5.0(t)}
+//    {1.0, 2.0, 3.0(t)}
+//
+// Compact a keyIndex removes the versions with smaller or equal to
+// rev except the largest one. If the generation becomes empty
+// during compaction, it will be removed. if all the generations get
+// removed, the keyIndex should be removed.
+//
+// For example:
+// compact(2) on the previous example
+// generations:
+//    {empty}
+//    {4.0, 5.0(t)}
+//    {2.0, 3.0(t)}
+//
+// compact(4)
+// generations:
+//    {empty}
+//    {4.0, 5.0(t)}
+//
+// compact(5):
+// generations:
+//    {empty} -> key SHOULD be removed.
+//
+// compact(6):
+// generations:
+//    {empty} -> key SHOULD be removed.
+type keyIndex struct {
+	key         []byte
+	modified    revision // the main rev of the last modification
+	generations []generation
+}
+
+// put puts a revision to the keyIndex.
+func (ki *keyIndex) put(lg *zap.Logger, main int64, sub int64) {
+	rev := revision{main: main, sub: sub}
+
+	if !rev.GreaterThan(ki.modified) {
+		if lg != nil {
+			lg.Panic(
+				"'put' with an unexpected smaller revision",
+				zap.Int64("given-revision-main", rev.main),
+				zap.Int64("given-revision-sub", rev.sub),
+				zap.Int64("modified-revision-main", ki.modified.main),
+				zap.Int64("modified-revision-sub", ki.modified.sub),
+			)
+		} else {
+			plog.Panicf("store.keyindex: put with unexpected smaller revision [%v / %v]", rev, ki.modified)
+		}
+	}
+	if len(ki.generations) == 0 {
+		ki.generations = append(ki.generations, generation{})
+	}
+	g := &ki.generations[len(ki.generations)-1]
+	if len(g.revs) == 0 { // create a new key
+		keysGauge.Inc()
+		g.created = rev
+	}
+	g.revs = append(g.revs, rev)
+	g.ver++
+	ki.modified = rev
+}
+
+func (ki *keyIndex) restore(lg *zap.Logger, created, modified revision, ver int64) {
+	if len(ki.generations) != 0 {
+		if lg != nil {
+			lg.Panic(
+				"'restore' got an unexpected non-empty generations",
+				zap.Int("generations-size", len(ki.generations)),
+			)
+		} else {
+			plog.Panicf("store.keyindex: cannot restore non-empty keyIndex")
+		}
+	}
+
+	ki.modified = modified
+	g := generation{created: created, ver: ver, revs: []revision{modified}}
+	ki.generations = append(ki.generations, g)
+	keysGauge.Inc()
+}
+
+// tombstone puts a revision, pointing to a tombstone, to the keyIndex.
+// It also creates a new empty generation in the keyIndex.
+// It returns ErrRevisionNotFound when tombstone on an empty generation.
+func (ki *keyIndex) tombstone(lg *zap.Logger, main int64, sub int64) error {
+	if ki.isEmpty() {
+		if lg != nil {
+			lg.Panic(
+				"'tombstone' got an unexpected empty keyIndex",
+				zap.String("key", string(ki.key)),
+			)
+		} else {
+			plog.Panicf("store.keyindex: unexpected tombstone on empty keyIndex %s", string(ki.key))
+		}
+	}
+	if ki.generations[len(ki.generations)-1].isEmpty() {
+		return ErrRevisionNotFound
+	}
+	ki.put(lg, main, sub)
+	ki.generations = append(ki.generations, generation{})
+	keysGauge.Dec()
+	return nil
+}
+
+// get gets the modified, created revision and version of the key that satisfies the given atRev.
+// Rev must be higher than or equal to the given atRev.
+func (ki *keyIndex) get(lg *zap.Logger, atRev int64) (modified, created revision, ver int64, err error) {
+	if ki.isEmpty() {
+		if lg != nil {
+			lg.Panic(
+				"'get' got an unexpected empty keyIndex",
+				zap.String("key", string(ki.key)),
+			)
+		} else {
+			plog.Panicf("store.keyindex: unexpected get on empty keyIndex %s", string(ki.key))
+		}
+	}
+	g := ki.findGeneration(atRev)
+	if g.isEmpty() {
+		return revision{}, revision{}, 0, ErrRevisionNotFound
+	}
+
+	n := g.walk(func(rev revision) bool { return rev.main > atRev })
+	if n != -1 {
+		return g.revs[n], g.created, g.ver - int64(len(g.revs)-n-1), nil
+	}
+
+	return revision{}, revision{}, 0, ErrRevisionNotFound
+}
+
+// since returns revisions since the given rev. Only the revision with the
+// largest sub revision will be returned if multiple revisions have the same
+// main revision.
+func (ki *keyIndex) since(lg *zap.Logger, rev int64) []revision {
+	if ki.isEmpty() {
+		if lg != nil {
+			lg.Panic(
+				"'since' got an unexpected empty keyIndex",
+				zap.String("key", string(ki.key)),
+			)
+		} else {
+			plog.Panicf("store.keyindex: unexpected get on empty keyIndex %s", string(ki.key))
+		}
+	}
+	since := revision{rev, 0}
+	var gi int
+	// find the generations to start checking
+	for gi = len(ki.generations) - 1; gi > 0; gi-- {
+		g := ki.generations[gi]
+		if g.isEmpty() {
+			continue
+		}
+		if since.GreaterThan(g.created) {
+			break
+		}
+	}
+
+	var revs []revision
+	var last int64
+	for ; gi < len(ki.generations); gi++ {
+		for _, r := range ki.generations[gi].revs {
+			if since.GreaterThan(r) {
+				continue
+			}
+			if r.main == last {
+				// replace the revision with a new one that has higher sub value,
+				// because the original one should not be seen by external
+				revs[len(revs)-1] = r
+				continue
+			}
+			revs = append(revs, r)
+			last = r.main
+		}
+	}
+	return revs
+}
+
+// compact compacts a keyIndex by removing the versions with smaller or equal
+// revision than the given atRev except the largest one (If the largest one is
+// a tombstone, it will not be kept).
+// If a generation becomes empty during compaction, it will be removed.
+func (ki *keyIndex) compact(lg *zap.Logger, atRev int64, available map[revision]struct{}) {
+	if ki.isEmpty() {
+		if lg != nil {
+			lg.Panic(
+				"'compact' got an unexpected empty keyIndex",
+				zap.String("key", string(ki.key)),
+			)
+		} else {
+			plog.Panicf("store.keyindex: unexpected compact on empty keyIndex %s", string(ki.key))
+		}
+	}
+
+	genIdx, revIndex := ki.doCompact(atRev, available)
+
+	g := &ki.generations[genIdx]
+	if !g.isEmpty() {
+		// remove the previous contents.
+		if revIndex != -1 {
+			g.revs = g.revs[revIndex:]
+		}
+		// remove any tombstone
+		if len(g.revs) == 1 && genIdx != len(ki.generations)-1 {
+			delete(available, g.revs[0])
+			genIdx++
+		}
+	}
+
+	// remove the previous generations.
+	ki.generations = ki.generations[genIdx:]
+}
+
+// keep finds the revision to be kept if compact is called at given atRev.
+func (ki *keyIndex) keep(atRev int64, available map[revision]struct{}) {
+	if ki.isEmpty() {
+		return
+	}
+
+	genIdx, revIndex := ki.doCompact(atRev, available)
+	g := &ki.generations[genIdx]
+	if !g.isEmpty() {
+		// remove any tombstone
+		if revIndex == len(g.revs)-1 && genIdx != len(ki.generations)-1 {
+			delete(available, g.revs[revIndex])
+		}
+	}
+}
+
+func (ki *keyIndex) doCompact(atRev int64, available map[revision]struct{}) (genIdx int, revIndex int) {
+	// walk until reaching the first revision smaller or equal to "atRev",
+	// and add the revision to the available map
+	f := func(rev revision) bool {
+		if rev.main <= atRev {
+			available[rev] = struct{}{}
+			return false
+		}
+		return true
+	}
+
+	genIdx, g := 0, &ki.generations[0]
+	// find first generation includes atRev or created after atRev
+	for genIdx < len(ki.generations)-1 {
+		if tomb := g.revs[len(g.revs)-1].main; tomb > atRev {
+			break
+		}
+		genIdx++
+		g = &ki.generations[genIdx]
+	}
+
+	revIndex = g.walk(f)
+
+	return genIdx, revIndex
+}
+
+func (ki *keyIndex) isEmpty() bool {
+	return len(ki.generations) == 1 && ki.generations[0].isEmpty()
+}
+
+// findGeneration finds out the generation of the keyIndex that the
+// given rev belongs to. If the given rev is at the gap of two generations,
+// which means that the key does not exist at the given rev, it returns nil.
+func (ki *keyIndex) findGeneration(rev int64) *generation {
+	lastg := len(ki.generations) - 1
+	cg := lastg
+
+	for cg >= 0 {
+		if len(ki.generations[cg].revs) == 0 {
+			cg--
+			continue
+		}
+		g := ki.generations[cg]
+		if cg != lastg {
+			if tomb := g.revs[len(g.revs)-1].main; tomb <= rev {
+				return nil
+			}
+		}
+		if g.revs[0].main <= rev {
+			return &ki.generations[cg]
+		}
+		cg--
+	}
+	return nil
+}
+
+func (ki *keyIndex) Less(b btree.Item) bool {
+	return bytes.Compare(ki.key, b.(*keyIndex).key) == -1
+}
+
+func (ki *keyIndex) equal(b *keyIndex) bool {
+	if !bytes.Equal(ki.key, b.key) {
+		return false
+	}
+	if ki.modified != b.modified {
+		return false
+	}
+	if len(ki.generations) != len(b.generations) {
+		return false
+	}
+	for i := range ki.generations {
+		ag, bg := ki.generations[i], b.generations[i]
+		if !ag.equal(bg) {
+			return false
+		}
+	}
+	return true
+}
+
+func (ki *keyIndex) String() string {
+	var s string
+	for _, g := range ki.generations {
+		s += g.String()
+	}
+	return s
+}
+
+// generation contains multiple revisions of a key.
+type generation struct {
+	ver     int64
+	created revision // when the generation is created (put in first revision).
+	revs    []revision
+}
+
+func (g *generation) isEmpty() bool { return g == nil || len(g.revs) == 0 }
+
+// walk walks through the revisions in the generation in descending order.
+// It passes the revision to the given function.
+// walk returns until: 1. it finishes walking all pairs 2. the function returns false.
+// walk returns the position at where it stopped. If it stopped after
+// finishing walking, -1 will be returned.
+func (g *generation) walk(f func(rev revision) bool) int {
+	l := len(g.revs)
+	for i := range g.revs {
+		ok := f(g.revs[l-i-1])
+		if !ok {
+			return l - i - 1
+		}
+	}
+	return -1
+}
+
+func (g *generation) String() string {
+	return fmt.Sprintf("g: created[%d] ver[%d], revs %#v\n", g.created, g.ver, g.revs)
+}
+
+func (g generation) equal(b generation) bool {
+	if g.ver != b.ver {
+		return false
+	}
+	if len(g.revs) != len(b.revs) {
+		return false
+	}
+
+	for i := range g.revs {
+		ar, br := g.revs[i], b.revs[i]
+		if ar != br {
+			return false
+		}
+	}
+	return true
+}
diff --git a/vendor/go.etcd.io/etcd/mvcc/kv.go b/vendor/go.etcd.io/etcd/mvcc/kv.go
new file mode 100644
index 0000000..8e898a5
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/kv.go
@@ -0,0 +1,149 @@
+// Copyright 2015 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 mvcc
+
+import (
+	"go.etcd.io/etcd/lease"
+	"go.etcd.io/etcd/mvcc/backend"
+	"go.etcd.io/etcd/mvcc/mvccpb"
+)
+
+type RangeOptions struct {
+	Limit int64
+	Rev   int64
+	Count bool
+}
+
+type RangeResult struct {
+	KVs   []mvccpb.KeyValue
+	Rev   int64
+	Count int
+}
+
+type ReadView interface {
+	// FirstRev returns the first KV revision at the time of opening the txn.
+	// After a compaction, the first revision increases to the compaction
+	// revision.
+	FirstRev() int64
+
+	// Rev returns the revision of the KV at the time of opening the txn.
+	Rev() int64
+
+	// Range gets the keys in the range at rangeRev.
+	// The returned rev is the current revision of the KV when the operation is executed.
+	// If rangeRev <=0, range gets the keys at currentRev.
+	// If `end` is nil, the request returns the key.
+	// If `end` is not nil and not empty, it gets the keys in range [key, range_end).
+	// If `end` is not nil and empty, it gets the keys greater than or equal to key.
+	// Limit limits the number of keys returned.
+	// If the required rev is compacted, ErrCompacted will be returned.
+	Range(key, end []byte, ro RangeOptions) (r *RangeResult, err error)
+}
+
+// TxnRead represents a read-only transaction with operations that will not
+// block other read transactions.
+type TxnRead interface {
+	ReadView
+	// End marks the transaction is complete and ready to commit.
+	End()
+}
+
+type WriteView interface {
+	// DeleteRange deletes the given range from the store.
+	// A deleteRange increases the rev of the store if any key in the range exists.
+	// The number of key deleted will be returned.
+	// The returned rev is the current revision of the KV when the operation is executed.
+	// It also generates one event for each key delete in the event history.
+	// if the `end` is nil, deleteRange deletes the key.
+	// if the `end` is not nil, deleteRange deletes the keys in range [key, range_end).
+	DeleteRange(key, end []byte) (n, rev int64)
+
+	// Put puts the given key, value into the store. Put also takes additional argument lease to
+	// attach a lease to a key-value pair as meta-data. KV implementation does not validate the lease
+	// id.
+	// A put also increases the rev of the store, and generates one event in the event history.
+	// The returned rev is the current revision of the KV when the operation is executed.
+	Put(key, value []byte, lease lease.LeaseID) (rev int64)
+}
+
+// TxnWrite represents a transaction that can modify the store.
+type TxnWrite interface {
+	TxnRead
+	WriteView
+	// Changes gets the changes made since opening the write txn.
+	Changes() []mvccpb.KeyValue
+}
+
+// txnReadWrite coerces a read txn to a write, panicking on any write operation.
+type txnReadWrite struct{ TxnRead }
+
+func (trw *txnReadWrite) DeleteRange(key, end []byte) (n, rev int64) { panic("unexpected DeleteRange") }
+func (trw *txnReadWrite) Put(key, value []byte, lease lease.LeaseID) (rev int64) {
+	panic("unexpected Put")
+}
+func (trw *txnReadWrite) Changes() []mvccpb.KeyValue { return nil }
+
+func NewReadOnlyTxnWrite(txn TxnRead) TxnWrite { return &txnReadWrite{txn} }
+
+type KV interface {
+	ReadView
+	WriteView
+
+	// Read creates a read transaction.
+	Read() TxnRead
+
+	// Write creates a write transaction.
+	Write() TxnWrite
+
+	// Hash computes the hash of the KV's backend.
+	Hash() (hash uint32, revision int64, err error)
+
+	// HashByRev computes the hash of all MVCC revisions up to a given revision.
+	HashByRev(rev int64) (hash uint32, revision int64, compactRev int64, err error)
+
+	// Compact frees all superseded keys with revisions less than rev.
+	Compact(rev int64) (<-chan struct{}, error)
+
+	// Commit commits outstanding txns into the underlying backend.
+	Commit()
+
+	// Restore restores the KV store from a backend.
+	Restore(b backend.Backend) error
+	Close() error
+}
+
+// WatchableKV is a KV that can be watched.
+type WatchableKV interface {
+	KV
+	Watchable
+}
+
+// Watchable is the interface that wraps the NewWatchStream function.
+type Watchable interface {
+	// NewWatchStream returns a WatchStream that can be used to
+	// watch events happened or happening on the KV.
+	NewWatchStream() WatchStream
+}
+
+// ConsistentWatchableKV is a WatchableKV that understands the consistency
+// algorithm and consistent index.
+// If the consistent index of executing entry is not larger than the
+// consistent index of ConsistentWatchableKV, all operations in
+// this entry are skipped and return empty response.
+type ConsistentWatchableKV interface {
+	WatchableKV
+	// ConsistentIndex returns the current consistent index of the KV.
+	ConsistentIndex() uint64
+}
diff --git a/vendor/go.etcd.io/etcd/mvcc/kv_view.go b/vendor/go.etcd.io/etcd/mvcc/kv_view.go
new file mode 100644
index 0000000..bd2e777
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/kv_view.go
@@ -0,0 +1,51 @@
+// 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 mvcc
+
+import "go.etcd.io/etcd/lease"
+
+type readView struct{ kv KV }
+
+func (rv *readView) FirstRev() int64 {
+	tr := rv.kv.Read()
+	defer tr.End()
+	return tr.FirstRev()
+}
+
+func (rv *readView) Rev() int64 {
+	tr := rv.kv.Read()
+	defer tr.End()
+	return tr.Rev()
+}
+
+func (rv *readView) Range(key, end []byte, ro RangeOptions) (r *RangeResult, err error) {
+	tr := rv.kv.Read()
+	defer tr.End()
+	return tr.Range(key, end, ro)
+}
+
+type writeView struct{ kv KV }
+
+func (wv *writeView) DeleteRange(key, end []byte) (n, rev int64) {
+	tw := wv.kv.Write()
+	defer tw.End()
+	return tw.DeleteRange(key, end)
+}
+
+func (wv *writeView) Put(key, value []byte, lease lease.LeaseID) (rev int64) {
+	tw := wv.kv.Write()
+	defer tw.End()
+	return tw.Put(key, value, lease)
+}
diff --git a/vendor/go.etcd.io/etcd/mvcc/kvstore.go b/vendor/go.etcd.io/etcd/mvcc/kvstore.go
new file mode 100644
index 0000000..e367ebb
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/kvstore.go
@@ -0,0 +1,620 @@
+// Copyright 2015 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 mvcc
+
+import (
+	"context"
+	"encoding/binary"
+	"errors"
+	"fmt"
+	"hash/crc32"
+	"math"
+	"sync"
+	"sync/atomic"
+	"time"
+
+	"go.etcd.io/etcd/lease"
+	"go.etcd.io/etcd/mvcc/backend"
+	"go.etcd.io/etcd/mvcc/mvccpb"
+	"go.etcd.io/etcd/pkg/schedule"
+
+	"github.com/coreos/pkg/capnslog"
+	"go.uber.org/zap"
+)
+
+var (
+	keyBucketName  = []byte("key")
+	metaBucketName = []byte("meta")
+
+	consistentIndexKeyName  = []byte("consistent_index")
+	scheduledCompactKeyName = []byte("scheduledCompactRev")
+	finishedCompactKeyName  = []byte("finishedCompactRev")
+
+	ErrCompacted = errors.New("mvcc: required revision has been compacted")
+	ErrFutureRev = errors.New("mvcc: required revision is a future revision")
+	ErrCanceled  = errors.New("mvcc: watcher is canceled")
+	ErrClosed    = errors.New("mvcc: closed")
+
+	plog = capnslog.NewPackageLogger("go.etcd.io/etcd", "mvcc")
+)
+
+const (
+	// markedRevBytesLen is the byte length of marked revision.
+	// The first `revBytesLen` bytes represents a normal revision. The last
+	// one byte is the mark.
+	markedRevBytesLen      = revBytesLen + 1
+	markBytePosition       = markedRevBytesLen - 1
+	markTombstone     byte = 't'
+)
+
+var restoreChunkKeys = 10000 // non-const for testing
+var defaultCompactBatchLimit = 1000
+
+// ConsistentIndexGetter is an interface that wraps the Get method.
+// Consistent index is the offset of an entry in a consistent replicated log.
+type ConsistentIndexGetter interface {
+	// ConsistentIndex returns the consistent index of current executing entry.
+	ConsistentIndex() uint64
+}
+
+type StoreConfig struct {
+	CompactionBatchLimit int
+}
+
+type store struct {
+	ReadView
+	WriteView
+
+	// consistentIndex caches the "consistent_index" key's value. Accessed
+	// through atomics so must be 64-bit aligned.
+	consistentIndex uint64
+
+	cfg StoreConfig
+
+	// mu read locks for txns and write locks for non-txn store changes.
+	mu sync.RWMutex
+
+	ig ConsistentIndexGetter
+
+	b       backend.Backend
+	kvindex index
+
+	le lease.Lessor
+
+	// revMuLock protects currentRev and compactMainRev.
+	// Locked at end of write txn and released after write txn unlock lock.
+	// Locked before locking read txn and released after locking.
+	revMu sync.RWMutex
+	// currentRev is the revision of the last completed transaction.
+	currentRev int64
+	// compactMainRev is the main revision of the last compaction.
+	compactMainRev int64
+
+	// bytesBuf8 is a byte slice of length 8
+	// to avoid a repetitive allocation in saveIndex.
+	bytesBuf8 []byte
+
+	fifoSched schedule.Scheduler
+
+	stopc chan struct{}
+
+	lg *zap.Logger
+}
+
+// NewStore returns a new store. It is useful to create a store inside
+// mvcc pkg. It should only be used for testing externally.
+func NewStore(lg *zap.Logger, b backend.Backend, le lease.Lessor, ig ConsistentIndexGetter, cfg StoreConfig) *store {
+	if cfg.CompactionBatchLimit == 0 {
+		cfg.CompactionBatchLimit = defaultCompactBatchLimit
+	}
+	s := &store{
+		cfg:     cfg,
+		b:       b,
+		ig:      ig,
+		kvindex: newTreeIndex(lg),
+
+		le: le,
+
+		currentRev:     1,
+		compactMainRev: -1,
+
+		bytesBuf8: make([]byte, 8),
+		fifoSched: schedule.NewFIFOScheduler(),
+
+		stopc: make(chan struct{}),
+
+		lg: lg,
+	}
+	s.ReadView = &readView{s}
+	s.WriteView = &writeView{s}
+	if s.le != nil {
+		s.le.SetRangeDeleter(func() lease.TxnDelete { return s.Write() })
+	}
+
+	tx := s.b.BatchTx()
+	tx.Lock()
+	tx.UnsafeCreateBucket(keyBucketName)
+	tx.UnsafeCreateBucket(metaBucketName)
+	tx.Unlock()
+	s.b.ForceCommit()
+
+	s.mu.Lock()
+	defer s.mu.Unlock()
+	if err := s.restore(); err != nil {
+		// TODO: return the error instead of panic here?
+		panic("failed to recover store from backend")
+	}
+
+	return s
+}
+
+func (s *store) compactBarrier(ctx context.Context, ch chan struct{}) {
+	if ctx == nil || ctx.Err() != nil {
+		s.mu.Lock()
+		select {
+		case <-s.stopc:
+		default:
+			f := func(ctx context.Context) { s.compactBarrier(ctx, ch) }
+			s.fifoSched.Schedule(f)
+		}
+		s.mu.Unlock()
+		return
+	}
+	close(ch)
+}
+
+func (s *store) Hash() (hash uint32, revision int64, err error) {
+	// TODO: hash and revision could be inconsistent, one possible fix is to add s.revMu.RLock() at the beginning of function, which is costly
+	start := time.Now()
+
+	s.b.ForceCommit()
+	h, err := s.b.Hash(DefaultIgnores)
+
+	hashSec.Observe(time.Since(start).Seconds())
+	return h, s.currentRev, err
+}
+
+func (s *store) HashByRev(rev int64) (hash uint32, currentRev int64, compactRev int64, err error) {
+	start := time.Now()
+
+	s.mu.RLock()
+	s.revMu.RLock()
+	compactRev, currentRev = s.compactMainRev, s.currentRev
+	s.revMu.RUnlock()
+
+	if rev > 0 && rev <= compactRev {
+		s.mu.RUnlock()
+		return 0, 0, compactRev, ErrCompacted
+	} else if rev > 0 && rev > currentRev {
+		s.mu.RUnlock()
+		return 0, currentRev, 0, ErrFutureRev
+	}
+
+	if rev == 0 {
+		rev = currentRev
+	}
+	keep := s.kvindex.Keep(rev)
+
+	tx := s.b.ReadTx()
+	tx.RLock()
+	defer tx.RUnlock()
+	s.mu.RUnlock()
+
+	upper := revision{main: rev + 1}
+	lower := revision{main: compactRev + 1}
+	h := crc32.New(crc32.MakeTable(crc32.Castagnoli))
+
+	h.Write(keyBucketName)
+	err = tx.UnsafeForEach(keyBucketName, func(k, v []byte) error {
+		kr := bytesToRev(k)
+		if !upper.GreaterThan(kr) {
+			return nil
+		}
+		// skip revisions that are scheduled for deletion
+		// due to compacting; don't skip if there isn't one.
+		if lower.GreaterThan(kr) && len(keep) > 0 {
+			if _, ok := keep[kr]; !ok {
+				return nil
+			}
+		}
+		h.Write(k)
+		h.Write(v)
+		return nil
+	})
+	hash = h.Sum32()
+
+	hashRevSec.Observe(time.Since(start).Seconds())
+	return hash, currentRev, compactRev, err
+}
+
+func (s *store) updateCompactRev(rev int64) (<-chan struct{}, error) {
+	s.revMu.Lock()
+	if rev <= s.compactMainRev {
+		ch := make(chan struct{})
+		f := func(ctx context.Context) { s.compactBarrier(ctx, ch) }
+		s.fifoSched.Schedule(f)
+		s.revMu.Unlock()
+		return ch, ErrCompacted
+	}
+	if rev > s.currentRev {
+		s.revMu.Unlock()
+		return nil, ErrFutureRev
+	}
+
+	s.compactMainRev = rev
+
+	rbytes := newRevBytes()
+	revToBytes(revision{main: rev}, rbytes)
+
+	tx := s.b.BatchTx()
+	tx.Lock()
+	tx.UnsafePut(metaBucketName, scheduledCompactKeyName, rbytes)
+	tx.Unlock()
+	// ensure that desired compaction is persisted
+	s.b.ForceCommit()
+
+	s.revMu.Unlock()
+
+	return nil, nil
+}
+
+func (s *store) compact(rev int64) (<-chan struct{}, error) {
+	start := time.Now()
+	keep := s.kvindex.Compact(rev)
+	ch := make(chan struct{})
+	var j = func(ctx context.Context) {
+		if ctx.Err() != nil {
+			s.compactBarrier(ctx, ch)
+			return
+		}
+		if !s.scheduleCompaction(rev, keep) {
+			s.compactBarrier(nil, ch)
+			return
+		}
+		close(ch)
+	}
+
+	s.fifoSched.Schedule(j)
+
+	indexCompactionPauseMs.Observe(float64(time.Since(start) / time.Millisecond))
+	return ch, nil
+}
+
+func (s *store) compactLockfree(rev int64) (<-chan struct{}, error) {
+	ch, err := s.updateCompactRev(rev)
+	if nil != err {
+		return ch, err
+	}
+
+	return s.compact(rev)
+}
+
+func (s *store) Compact(rev int64) (<-chan struct{}, error) {
+	s.mu.Lock()
+
+	ch, err := s.updateCompactRev(rev)
+
+	if err != nil {
+		s.mu.Unlock()
+		return ch, err
+	}
+	s.mu.Unlock()
+
+	return s.compact(rev)
+}
+
+// DefaultIgnores is a map of keys to ignore in hash checking.
+var DefaultIgnores map[backend.IgnoreKey]struct{}
+
+func init() {
+	DefaultIgnores = map[backend.IgnoreKey]struct{}{
+		// consistent index might be changed due to v2 internal sync, which
+		// is not controllable by the user.
+		{Bucket: string(metaBucketName), Key: string(consistentIndexKeyName)}: {},
+	}
+}
+
+func (s *store) Commit() {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+
+	tx := s.b.BatchTx()
+	tx.Lock()
+	s.saveIndex(tx)
+	tx.Unlock()
+	s.b.ForceCommit()
+}
+
+func (s *store) Restore(b backend.Backend) error {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+
+	close(s.stopc)
+	s.fifoSched.Stop()
+
+	atomic.StoreUint64(&s.consistentIndex, 0)
+	s.b = b
+	s.kvindex = newTreeIndex(s.lg)
+	s.currentRev = 1
+	s.compactMainRev = -1
+	s.fifoSched = schedule.NewFIFOScheduler()
+	s.stopc = make(chan struct{})
+
+	return s.restore()
+}
+
+func (s *store) restore() error {
+	s.setupMetricsReporter()
+
+	min, max := newRevBytes(), newRevBytes()
+	revToBytes(revision{main: 1}, min)
+	revToBytes(revision{main: math.MaxInt64, sub: math.MaxInt64}, max)
+
+	keyToLease := make(map[string]lease.LeaseID)
+
+	// restore index
+	tx := s.b.BatchTx()
+	tx.Lock()
+
+	_, finishedCompactBytes := tx.UnsafeRange(metaBucketName, finishedCompactKeyName, nil, 0)
+	if len(finishedCompactBytes) != 0 {
+		s.compactMainRev = bytesToRev(finishedCompactBytes[0]).main
+
+		if s.lg != nil {
+			s.lg.Info(
+				"restored last compact revision",
+				zap.String("meta-bucket-name", string(metaBucketName)),
+				zap.String("meta-bucket-name-key", string(finishedCompactKeyName)),
+				zap.Int64("restored-compact-revision", s.compactMainRev),
+			)
+		} else {
+			plog.Printf("restore compact to %d", s.compactMainRev)
+		}
+	}
+	_, scheduledCompactBytes := tx.UnsafeRange(metaBucketName, scheduledCompactKeyName, nil, 0)
+	scheduledCompact := int64(0)
+	if len(scheduledCompactBytes) != 0 {
+		scheduledCompact = bytesToRev(scheduledCompactBytes[0]).main
+	}
+
+	// index keys concurrently as they're loaded in from tx
+	keysGauge.Set(0)
+	rkvc, revc := restoreIntoIndex(s.lg, s.kvindex)
+	for {
+		keys, vals := tx.UnsafeRange(keyBucketName, min, max, int64(restoreChunkKeys))
+		if len(keys) == 0 {
+			break
+		}
+		// rkvc blocks if the total pending keys exceeds the restore
+		// chunk size to keep keys from consuming too much memory.
+		restoreChunk(s.lg, rkvc, keys, vals, keyToLease)
+		if len(keys) < restoreChunkKeys {
+			// partial set implies final set
+			break
+		}
+		// next set begins after where this one ended
+		newMin := bytesToRev(keys[len(keys)-1][:revBytesLen])
+		newMin.sub++
+		revToBytes(newMin, min)
+	}
+	close(rkvc)
+	s.currentRev = <-revc
+
+	// keys in the range [compacted revision -N, compaction] might all be deleted due to compaction.
+	// the correct revision should be set to compaction revision in the case, not the largest revision
+	// we have seen.
+	if s.currentRev < s.compactMainRev {
+		s.currentRev = s.compactMainRev
+	}
+	if scheduledCompact <= s.compactMainRev {
+		scheduledCompact = 0
+	}
+
+	for key, lid := range keyToLease {
+		if s.le == nil {
+			panic("no lessor to attach lease")
+		}
+		err := s.le.Attach(lid, []lease.LeaseItem{{Key: key}})
+		if err != nil {
+			if s.lg != nil {
+				s.lg.Warn(
+					"failed to attach a lease",
+					zap.String("lease-id", fmt.Sprintf("%016x", lid)),
+					zap.Error(err),
+				)
+			} else {
+				plog.Errorf("unexpected Attach error: %v", err)
+			}
+		}
+	}
+
+	tx.Unlock()
+
+	if scheduledCompact != 0 {
+		s.compactLockfree(scheduledCompact)
+
+		if s.lg != nil {
+			s.lg.Info(
+				"resume scheduled compaction",
+				zap.String("meta-bucket-name", string(metaBucketName)),
+				zap.String("meta-bucket-name-key", string(scheduledCompactKeyName)),
+				zap.Int64("scheduled-compact-revision", scheduledCompact),
+			)
+		} else {
+			plog.Printf("resume scheduled compaction at %d", scheduledCompact)
+		}
+	}
+
+	return nil
+}
+
+type revKeyValue struct {
+	key  []byte
+	kv   mvccpb.KeyValue
+	kstr string
+}
+
+func restoreIntoIndex(lg *zap.Logger, idx index) (chan<- revKeyValue, <-chan int64) {
+	rkvc, revc := make(chan revKeyValue, restoreChunkKeys), make(chan int64, 1)
+	go func() {
+		currentRev := int64(1)
+		defer func() { revc <- currentRev }()
+		// restore the tree index from streaming the unordered index.
+		kiCache := make(map[string]*keyIndex, restoreChunkKeys)
+		for rkv := range rkvc {
+			ki, ok := kiCache[rkv.kstr]
+			// purge kiCache if many keys but still missing in the cache
+			if !ok && len(kiCache) >= restoreChunkKeys {
+				i := 10
+				for k := range kiCache {
+					delete(kiCache, k)
+					if i--; i == 0 {
+						break
+					}
+				}
+			}
+			// cache miss, fetch from tree index if there
+			if !ok {
+				ki = &keyIndex{key: rkv.kv.Key}
+				if idxKey := idx.KeyIndex(ki); idxKey != nil {
+					kiCache[rkv.kstr], ki = idxKey, idxKey
+					ok = true
+				}
+			}
+			rev := bytesToRev(rkv.key)
+			currentRev = rev.main
+			if ok {
+				if isTombstone(rkv.key) {
+					ki.tombstone(lg, rev.main, rev.sub)
+					continue
+				}
+				ki.put(lg, rev.main, rev.sub)
+			} else if !isTombstone(rkv.key) {
+				ki.restore(lg, revision{rkv.kv.CreateRevision, 0}, rev, rkv.kv.Version)
+				idx.Insert(ki)
+				kiCache[rkv.kstr] = ki
+			}
+		}
+	}()
+	return rkvc, revc
+}
+
+func restoreChunk(lg *zap.Logger, kvc chan<- revKeyValue, keys, vals [][]byte, keyToLease map[string]lease.LeaseID) {
+	for i, key := range keys {
+		rkv := revKeyValue{key: key}
+		if err := rkv.kv.Unmarshal(vals[i]); err != nil {
+			if lg != nil {
+				lg.Fatal("failed to unmarshal mvccpb.KeyValue", zap.Error(err))
+			} else {
+				plog.Fatalf("cannot unmarshal event: %v", err)
+			}
+		}
+		rkv.kstr = string(rkv.kv.Key)
+		if isTombstone(key) {
+			delete(keyToLease, rkv.kstr)
+		} else if lid := lease.LeaseID(rkv.kv.Lease); lid != lease.NoLease {
+			keyToLease[rkv.kstr] = lid
+		} else {
+			delete(keyToLease, rkv.kstr)
+		}
+		kvc <- rkv
+	}
+}
+
+func (s *store) Close() error {
+	close(s.stopc)
+	s.fifoSched.Stop()
+	return nil
+}
+
+func (s *store) saveIndex(tx backend.BatchTx) {
+	if s.ig == nil {
+		return
+	}
+	bs := s.bytesBuf8
+	ci := s.ig.ConsistentIndex()
+	binary.BigEndian.PutUint64(bs, ci)
+	// put the index into the underlying backend
+	// tx has been locked in TxnBegin, so there is no need to lock it again
+	tx.UnsafePut(metaBucketName, consistentIndexKeyName, bs)
+	atomic.StoreUint64(&s.consistentIndex, ci)
+}
+
+func (s *store) ConsistentIndex() uint64 {
+	if ci := atomic.LoadUint64(&s.consistentIndex); ci > 0 {
+		return ci
+	}
+	tx := s.b.BatchTx()
+	tx.Lock()
+	defer tx.Unlock()
+	_, vs := tx.UnsafeRange(metaBucketName, consistentIndexKeyName, nil, 0)
+	if len(vs) == 0 {
+		return 0
+	}
+	v := binary.BigEndian.Uint64(vs[0])
+	atomic.StoreUint64(&s.consistentIndex, v)
+	return v
+}
+
+func (s *store) setupMetricsReporter() {
+	b := s.b
+	reportDbTotalSizeInBytesMu.Lock()
+	reportDbTotalSizeInBytes = func() float64 { return float64(b.Size()) }
+	reportDbTotalSizeInBytesMu.Unlock()
+	reportDbTotalSizeInBytesDebugMu.Lock()
+	reportDbTotalSizeInBytesDebug = func() float64 { return float64(b.Size()) }
+	reportDbTotalSizeInBytesDebugMu.Unlock()
+	reportDbTotalSizeInUseInBytesMu.Lock()
+	reportDbTotalSizeInUseInBytes = func() float64 { return float64(b.SizeInUse()) }
+	reportDbTotalSizeInUseInBytesMu.Unlock()
+	reportDbOpenReadTxNMu.Lock()
+	reportDbOpenReadTxN = func() float64 { return float64(b.OpenReadTxN()) }
+	reportDbOpenReadTxNMu.Unlock()
+	reportCurrentRevMu.Lock()
+	reportCurrentRev = func() float64 {
+		s.revMu.RLock()
+		defer s.revMu.RUnlock()
+		return float64(s.currentRev)
+	}
+	reportCurrentRevMu.Unlock()
+	reportCompactRevMu.Lock()
+	reportCompactRev = func() float64 {
+		s.revMu.RLock()
+		defer s.revMu.RUnlock()
+		return float64(s.compactMainRev)
+	}
+	reportCompactRevMu.Unlock()
+}
+
+// appendMarkTombstone appends tombstone mark to normal revision bytes.
+func appendMarkTombstone(lg *zap.Logger, b []byte) []byte {
+	if len(b) != revBytesLen {
+		if lg != nil {
+			lg.Panic(
+				"cannot append tombstone mark to non-normal revision bytes",
+				zap.Int("expected-revision-bytes-size", revBytesLen),
+				zap.Int("given-revision-bytes-size", len(b)),
+			)
+		} else {
+			plog.Panicf("cannot append mark to non normal revision bytes")
+		}
+	}
+	return append(b, markTombstone)
+}
+
+// isTombstone checks whether the revision bytes is a tombstone.
+func isTombstone(b []byte) bool {
+	return len(b) == markedRevBytesLen && b[markBytePosition] == markTombstone
+}
diff --git a/vendor/go.etcd.io/etcd/mvcc/kvstore_compaction.go b/vendor/go.etcd.io/etcd/mvcc/kvstore_compaction.go
new file mode 100644
index 0000000..2adb498
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/kvstore_compaction.go
@@ -0,0 +1,79 @@
+// Copyright 2015 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 mvcc
+
+import (
+	"encoding/binary"
+	"time"
+
+	"go.uber.org/zap"
+)
+
+func (s *store) scheduleCompaction(compactMainRev int64, keep map[revision]struct{}) bool {
+	totalStart := time.Now()
+	defer func() { dbCompactionTotalMs.Observe(float64(time.Since(totalStart) / time.Millisecond)) }()
+	keyCompactions := 0
+	defer func() { dbCompactionKeysCounter.Add(float64(keyCompactions)) }()
+
+	end := make([]byte, 8)
+	binary.BigEndian.PutUint64(end, uint64(compactMainRev+1))
+
+	last := make([]byte, 8+1+8)
+	for {
+		var rev revision
+
+		start := time.Now()
+
+		tx := s.b.BatchTx()
+		tx.Lock()
+		keys, _ := tx.UnsafeRange(keyBucketName, last, end, int64(s.cfg.CompactionBatchLimit))
+		for _, key := range keys {
+			rev = bytesToRev(key)
+			if _, ok := keep[rev]; !ok {
+				tx.UnsafeDelete(keyBucketName, key)
+			}
+		}
+
+		if len(keys) < s.cfg.CompactionBatchLimit {
+			rbytes := make([]byte, 8+1+8)
+			revToBytes(revision{main: compactMainRev}, rbytes)
+			tx.UnsafePut(metaBucketName, finishedCompactKeyName, rbytes)
+			tx.Unlock()
+			if s.lg != nil {
+				s.lg.Info(
+					"finished scheduled compaction",
+					zap.Int64("compact-revision", compactMainRev),
+					zap.Duration("took", time.Since(totalStart)),
+				)
+			} else {
+				plog.Infof("finished scheduled compaction at %d (took %v)", compactMainRev, time.Since(totalStart))
+			}
+			return true
+		}
+
+		// update last
+		revToBytes(revision{main: rev.main, sub: rev.sub + 1}, last)
+		tx.Unlock()
+		// Immediately commit the compaction deletes instead of letting them accumulate in the write buffer
+		s.b.ForceCommit()
+		dbCompactionPauseMs.Observe(float64(time.Since(start) / time.Millisecond))
+
+		select {
+		case <-time.After(10 * time.Millisecond):
+		case <-s.stopc:
+			return false
+		}
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/mvcc/kvstore_txn.go b/vendor/go.etcd.io/etcd/mvcc/kvstore_txn.go
new file mode 100644
index 0000000..9698254
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/kvstore_txn.go
@@ -0,0 +1,313 @@
+// 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 mvcc
+
+import (
+	"go.etcd.io/etcd/lease"
+	"go.etcd.io/etcd/mvcc/backend"
+	"go.etcd.io/etcd/mvcc/mvccpb"
+	"go.uber.org/zap"
+)
+
+type storeTxnRead struct {
+	s  *store
+	tx backend.ReadTx
+
+	firstRev int64
+	rev      int64
+}
+
+func (s *store) Read() TxnRead {
+	s.mu.RLock()
+	s.revMu.RLock()
+	// backend holds b.readTx.RLock() only when creating the concurrentReadTx. After
+	// ConcurrentReadTx is created, it will not block write transaction.
+	tx := s.b.ConcurrentReadTx()
+	tx.RLock() // RLock is no-op. concurrentReadTx does not need to be locked after it is created.
+	firstRev, rev := s.compactMainRev, s.currentRev
+	s.revMu.RUnlock()
+	return newMetricsTxnRead(&storeTxnRead{s, tx, firstRev, rev})
+}
+
+func (tr *storeTxnRead) FirstRev() int64 { return tr.firstRev }
+func (tr *storeTxnRead) Rev() int64      { return tr.rev }
+
+func (tr *storeTxnRead) Range(key, end []byte, ro RangeOptions) (r *RangeResult, err error) {
+	return tr.rangeKeys(key, end, tr.Rev(), ro)
+}
+
+func (tr *storeTxnRead) End() {
+	tr.tx.RUnlock() // RUnlock signals the end of concurrentReadTx.
+	tr.s.mu.RUnlock()
+}
+
+type storeTxnWrite struct {
+	storeTxnRead
+	tx backend.BatchTx
+	// beginRev is the revision where the txn begins; it will write to the next revision.
+	beginRev int64
+	changes  []mvccpb.KeyValue
+}
+
+func (s *store) Write() TxnWrite {
+	s.mu.RLock()
+	tx := s.b.BatchTx()
+	tx.Lock()
+	tw := &storeTxnWrite{
+		storeTxnRead: storeTxnRead{s, tx, 0, 0},
+		tx:           tx,
+		beginRev:     s.currentRev,
+		changes:      make([]mvccpb.KeyValue, 0, 4),
+	}
+	return newMetricsTxnWrite(tw)
+}
+
+func (tw *storeTxnWrite) Rev() int64 { return tw.beginRev }
+
+func (tw *storeTxnWrite) Range(key, end []byte, ro RangeOptions) (r *RangeResult, err error) {
+	rev := tw.beginRev
+	if len(tw.changes) > 0 {
+		rev++
+	}
+	return tw.rangeKeys(key, end, rev, ro)
+}
+
+func (tw *storeTxnWrite) DeleteRange(key, end []byte) (int64, int64) {
+	if n := tw.deleteRange(key, end); n != 0 || len(tw.changes) > 0 {
+		return n, tw.beginRev + 1
+	}
+	return 0, tw.beginRev
+}
+
+func (tw *storeTxnWrite) Put(key, value []byte, lease lease.LeaseID) int64 {
+	tw.put(key, value, lease)
+	return tw.beginRev + 1
+}
+
+func (tw *storeTxnWrite) End() {
+	// only update index if the txn modifies the mvcc state.
+	if len(tw.changes) != 0 {
+		tw.s.saveIndex(tw.tx)
+		// hold revMu lock to prevent new read txns from opening until writeback.
+		tw.s.revMu.Lock()
+		tw.s.currentRev++
+	}
+	tw.tx.Unlock()
+	if len(tw.changes) != 0 {
+		tw.s.revMu.Unlock()
+	}
+	tw.s.mu.RUnlock()
+}
+
+func (tr *storeTxnRead) rangeKeys(key, end []byte, curRev int64, ro RangeOptions) (*RangeResult, error) {
+	rev := ro.Rev
+	if rev > curRev {
+		return &RangeResult{KVs: nil, Count: -1, Rev: curRev}, ErrFutureRev
+	}
+	if rev <= 0 {
+		rev = curRev
+	}
+	if rev < tr.s.compactMainRev {
+		return &RangeResult{KVs: nil, Count: -1, Rev: 0}, ErrCompacted
+	}
+
+	revpairs := tr.s.kvindex.Revisions(key, end, rev)
+	if len(revpairs) == 0 {
+		return &RangeResult{KVs: nil, Count: 0, Rev: curRev}, nil
+	}
+	if ro.Count {
+		return &RangeResult{KVs: nil, Count: len(revpairs), Rev: curRev}, nil
+	}
+
+	limit := int(ro.Limit)
+	if limit <= 0 || limit > len(revpairs) {
+		limit = len(revpairs)
+	}
+
+	kvs := make([]mvccpb.KeyValue, limit)
+	revBytes := newRevBytes()
+	for i, revpair := range revpairs[:len(kvs)] {
+		revToBytes(revpair, revBytes)
+		_, vs := tr.tx.UnsafeRange(keyBucketName, revBytes, nil, 0)
+		if len(vs) != 1 {
+			if tr.s.lg != nil {
+				tr.s.lg.Fatal(
+					"range failed to find revision pair",
+					zap.Int64("revision-main", revpair.main),
+					zap.Int64("revision-sub", revpair.sub),
+				)
+			} else {
+				plog.Fatalf("range cannot find rev (%d,%d)", revpair.main, revpair.sub)
+			}
+		}
+		if err := kvs[i].Unmarshal(vs[0]); err != nil {
+			if tr.s.lg != nil {
+				tr.s.lg.Fatal(
+					"failed to unmarshal mvccpb.KeyValue",
+					zap.Error(err),
+				)
+			} else {
+				plog.Fatalf("cannot unmarshal event: %v", err)
+			}
+		}
+	}
+	return &RangeResult{KVs: kvs, Count: len(revpairs), Rev: curRev}, nil
+}
+
+func (tw *storeTxnWrite) put(key, value []byte, leaseID lease.LeaseID) {
+	rev := tw.beginRev + 1
+	c := rev
+	oldLease := lease.NoLease
+
+	// if the key exists before, use its previous created and
+	// get its previous leaseID
+	_, created, ver, err := tw.s.kvindex.Get(key, rev)
+	if err == nil {
+		c = created.main
+		oldLease = tw.s.le.GetLease(lease.LeaseItem{Key: string(key)})
+	}
+
+	ibytes := newRevBytes()
+	idxRev := revision{main: rev, sub: int64(len(tw.changes))}
+	revToBytes(idxRev, ibytes)
+
+	ver = ver + 1
+	kv := mvccpb.KeyValue{
+		Key:            key,
+		Value:          value,
+		CreateRevision: c,
+		ModRevision:    rev,
+		Version:        ver,
+		Lease:          int64(leaseID),
+	}
+
+	d, err := kv.Marshal()
+	if err != nil {
+		if tw.storeTxnRead.s.lg != nil {
+			tw.storeTxnRead.s.lg.Fatal(
+				"failed to marshal mvccpb.KeyValue",
+				zap.Error(err),
+			)
+		} else {
+			plog.Fatalf("cannot marshal event: %v", err)
+		}
+	}
+
+	tw.tx.UnsafeSeqPut(keyBucketName, ibytes, d)
+	tw.s.kvindex.Put(key, idxRev)
+	tw.changes = append(tw.changes, kv)
+
+	if oldLease != lease.NoLease {
+		if tw.s.le == nil {
+			panic("no lessor to detach lease")
+		}
+		err = tw.s.le.Detach(oldLease, []lease.LeaseItem{{Key: string(key)}})
+		if err != nil {
+			if tw.storeTxnRead.s.lg != nil {
+				tw.storeTxnRead.s.lg.Fatal(
+					"failed to detach old lease from a key",
+					zap.Error(err),
+				)
+			} else {
+				plog.Errorf("unexpected error from lease detach: %v", err)
+			}
+		}
+	}
+	if leaseID != lease.NoLease {
+		if tw.s.le == nil {
+			panic("no lessor to attach lease")
+		}
+		err = tw.s.le.Attach(leaseID, []lease.LeaseItem{{Key: string(key)}})
+		if err != nil {
+			panic("unexpected error from lease Attach")
+		}
+	}
+}
+
+func (tw *storeTxnWrite) deleteRange(key, end []byte) int64 {
+	rrev := tw.beginRev
+	if len(tw.changes) > 0 {
+		rrev++
+	}
+	keys, _ := tw.s.kvindex.Range(key, end, rrev)
+	if len(keys) == 0 {
+		return 0
+	}
+	for _, key := range keys {
+		tw.delete(key)
+	}
+	return int64(len(keys))
+}
+
+func (tw *storeTxnWrite) delete(key []byte) {
+	ibytes := newRevBytes()
+	idxRev := revision{main: tw.beginRev + 1, sub: int64(len(tw.changes))}
+	revToBytes(idxRev, ibytes)
+
+	if tw.storeTxnRead.s != nil && tw.storeTxnRead.s.lg != nil {
+		ibytes = appendMarkTombstone(tw.storeTxnRead.s.lg, ibytes)
+	} else {
+		// TODO: remove this in v3.5
+		ibytes = appendMarkTombstone(nil, ibytes)
+	}
+
+	kv := mvccpb.KeyValue{Key: key}
+
+	d, err := kv.Marshal()
+	if err != nil {
+		if tw.storeTxnRead.s.lg != nil {
+			tw.storeTxnRead.s.lg.Fatal(
+				"failed to marshal mvccpb.KeyValue",
+				zap.Error(err),
+			)
+		} else {
+			plog.Fatalf("cannot marshal event: %v", err)
+		}
+	}
+
+	tw.tx.UnsafeSeqPut(keyBucketName, ibytes, d)
+	err = tw.s.kvindex.Tombstone(key, idxRev)
+	if err != nil {
+		if tw.storeTxnRead.s.lg != nil {
+			tw.storeTxnRead.s.lg.Fatal(
+				"failed to tombstone an existing key",
+				zap.String("key", string(key)),
+				zap.Error(err),
+			)
+		} else {
+			plog.Fatalf("cannot tombstone an existing key (%s): %v", string(key), err)
+		}
+	}
+	tw.changes = append(tw.changes, kv)
+
+	item := lease.LeaseItem{Key: string(key)}
+	leaseID := tw.s.le.GetLease(item)
+
+	if leaseID != lease.NoLease {
+		err = tw.s.le.Detach(leaseID, []lease.LeaseItem{item})
+		if err != nil {
+			if tw.storeTxnRead.s.lg != nil {
+				tw.storeTxnRead.s.lg.Fatal(
+					"failed to detach old lease from a key",
+					zap.Error(err),
+				)
+			} else {
+				plog.Errorf("cannot detach %v", err)
+			}
+		}
+	}
+}
+
+func (tw *storeTxnWrite) Changes() []mvccpb.KeyValue { return tw.changes }
diff --git a/vendor/go.etcd.io/etcd/mvcc/metrics.go b/vendor/go.etcd.io/etcd/mvcc/metrics.go
new file mode 100644
index 0000000..7526ee4
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/metrics.go
@@ -0,0 +1,336 @@
+// Copyright 2015 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 mvcc
+
+import (
+	"sync"
+
+	"github.com/prometheus/client_golang/prometheus"
+)
+
+var (
+	rangeCounter = prometheus.NewCounter(
+		prometheus.CounterOpts{
+			Namespace: "etcd",
+			Subsystem: "mvcc",
+			Name:      "range_total",
+			Help:      "Total number of ranges seen by this member.",
+		})
+	rangeCounterDebug = prometheus.NewCounter(
+		prometheus.CounterOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "mvcc",
+			Name:      "range_total",
+			Help:      "Total number of ranges seen by this member.",
+		})
+
+	putCounter = prometheus.NewCounter(
+		prometheus.CounterOpts{
+			Namespace: "etcd",
+			Subsystem: "mvcc",
+			Name:      "put_total",
+			Help:      "Total number of puts seen by this member.",
+		})
+	// TODO: remove in 3.5 release
+	putCounterDebug = prometheus.NewCounter(
+		prometheus.CounterOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "mvcc",
+			Name:      "put_total",
+			Help:      "Total number of puts seen by this member.",
+		})
+
+	deleteCounter = prometheus.NewCounter(
+		prometheus.CounterOpts{
+			Namespace: "etcd",
+			Subsystem: "mvcc",
+			Name:      "delete_total",
+			Help:      "Total number of deletes seen by this member.",
+		})
+	// TODO: remove in 3.5 release
+	deleteCounterDebug = prometheus.NewCounter(
+		prometheus.CounterOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "mvcc",
+			Name:      "delete_total",
+			Help:      "Total number of deletes seen by this member.",
+		})
+
+	txnCounter = prometheus.NewCounter(
+		prometheus.CounterOpts{
+			Namespace: "etcd",
+			Subsystem: "mvcc",
+			Name:      "txn_total",
+			Help:      "Total number of txns seen by this member.",
+		})
+	txnCounterDebug = prometheus.NewCounter(
+		prometheus.CounterOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "mvcc",
+			Name:      "txn_total",
+			Help:      "Total number of txns seen by this member.",
+		})
+
+	keysGauge = prometheus.NewGauge(
+		prometheus.GaugeOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "mvcc",
+			Name:      "keys_total",
+			Help:      "Total number of keys.",
+		})
+
+	watchStreamGauge = prometheus.NewGauge(
+		prometheus.GaugeOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "mvcc",
+			Name:      "watch_stream_total",
+			Help:      "Total number of watch streams.",
+		})
+
+	watcherGauge = prometheus.NewGauge(
+		prometheus.GaugeOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "mvcc",
+			Name:      "watcher_total",
+			Help:      "Total number of watchers.",
+		})
+
+	slowWatcherGauge = prometheus.NewGauge(
+		prometheus.GaugeOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "mvcc",
+			Name:      "slow_watcher_total",
+			Help:      "Total number of unsynced slow watchers.",
+		})
+
+	totalEventsCounter = prometheus.NewCounter(
+		prometheus.CounterOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "mvcc",
+			Name:      "events_total",
+			Help:      "Total number of events sent by this member.",
+		})
+
+	pendingEventsGauge = prometheus.NewGauge(
+		prometheus.GaugeOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "mvcc",
+			Name:      "pending_events_total",
+			Help:      "Total number of pending events to be sent.",
+		})
+
+	indexCompactionPauseMs = prometheus.NewHistogram(
+		prometheus.HistogramOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "mvcc",
+			Name:      "index_compaction_pause_duration_milliseconds",
+			Help:      "Bucketed histogram of index compaction pause duration.",
+
+			// lowest bucket start of upper bound 0.5 ms with factor 2
+			// highest bucket start of 0.5 ms * 2^13 == 4.096 sec
+			Buckets: prometheus.ExponentialBuckets(0.5, 2, 14),
+		})
+
+	dbCompactionPauseMs = prometheus.NewHistogram(
+		prometheus.HistogramOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "mvcc",
+			Name:      "db_compaction_pause_duration_milliseconds",
+			Help:      "Bucketed histogram of db compaction pause duration.",
+
+			// lowest bucket start of upper bound 1 ms with factor 2
+			// highest bucket start of 1 ms * 2^12 == 4.096 sec
+			Buckets: prometheus.ExponentialBuckets(1, 2, 13),
+		})
+
+	dbCompactionTotalMs = prometheus.NewHistogram(
+		prometheus.HistogramOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "mvcc",
+			Name:      "db_compaction_total_duration_milliseconds",
+			Help:      "Bucketed histogram of db compaction total duration.",
+
+			// lowest bucket start of upper bound 100 ms with factor 2
+			// highest bucket start of 100 ms * 2^13 == 8.192 sec
+			Buckets: prometheus.ExponentialBuckets(100, 2, 14),
+		})
+
+	dbCompactionKeysCounter = prometheus.NewCounter(
+		prometheus.CounterOpts{
+			Namespace: "etcd_debugging",
+			Subsystem: "mvcc",
+			Name:      "db_compaction_keys_total",
+			Help:      "Total number of db keys compacted.",
+		})
+
+	dbTotalSize = prometheus.NewGaugeFunc(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "mvcc",
+		Name:      "db_total_size_in_bytes",
+		Help:      "Total size of the underlying database physically allocated in bytes.",
+	},
+		func() float64 {
+			reportDbTotalSizeInBytesMu.RLock()
+			defer reportDbTotalSizeInBytesMu.RUnlock()
+			return reportDbTotalSizeInBytes()
+		},
+	)
+	// overridden by mvcc initialization
+	reportDbTotalSizeInBytesMu sync.RWMutex
+	reportDbTotalSizeInBytes   = func() float64 { return 0 }
+
+	// TODO: remove this in v3.5
+	dbTotalSizeDebug = prometheus.NewGaugeFunc(prometheus.GaugeOpts{
+		Namespace: "etcd_debugging",
+		Subsystem: "mvcc",
+		Name:      "db_total_size_in_bytes",
+		Help:      "Total size of the underlying database physically allocated in bytes.",
+	},
+		func() float64 {
+			reportDbTotalSizeInBytesDebugMu.RLock()
+			defer reportDbTotalSizeInBytesDebugMu.RUnlock()
+			return reportDbTotalSizeInBytesDebug()
+		},
+	)
+	// overridden by mvcc initialization
+	reportDbTotalSizeInBytesDebugMu sync.RWMutex
+	reportDbTotalSizeInBytesDebug   = func() float64 { return 0 }
+
+	dbTotalSizeInUse = prometheus.NewGaugeFunc(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "mvcc",
+		Name:      "db_total_size_in_use_in_bytes",
+		Help:      "Total size of the underlying database logically in use in bytes.",
+	},
+		func() float64 {
+			reportDbTotalSizeInUseInBytesMu.RLock()
+			defer reportDbTotalSizeInUseInBytesMu.RUnlock()
+			return reportDbTotalSizeInUseInBytes()
+		},
+	)
+	// overridden by mvcc initialization
+	reportDbTotalSizeInUseInBytesMu sync.RWMutex
+	reportDbTotalSizeInUseInBytes   = func() float64 { return 0 }
+
+	dbOpenReadTxN = prometheus.NewGaugeFunc(prometheus.GaugeOpts{
+		Namespace: "etcd",
+		Subsystem: "mvcc",
+		Name:      "db_open_read_transactions",
+		Help:      "The number of currently open read transactions",
+	},
+
+		func() float64 {
+			reportDbOpenReadTxNMu.RLock()
+			defer reportDbOpenReadTxNMu.RUnlock()
+			return reportDbOpenReadTxN()
+		},
+	)
+	// overridden by mvcc initialization
+	reportDbOpenReadTxNMu sync.RWMutex
+	reportDbOpenReadTxN   = func() float64 { return 0 }
+
+	hashSec = prometheus.NewHistogram(prometheus.HistogramOpts{
+		Namespace: "etcd",
+		Subsystem: "mvcc",
+		Name:      "hash_duration_seconds",
+		Help:      "The latency distribution of storage hash operation.",
+
+		// 100 MB usually takes 100 ms, so start with 10 MB of 10 ms
+		// lowest bucket start of upper bound 0.01 sec (10 ms) with factor 2
+		// highest bucket start of 0.01 sec * 2^14 == 163.84 sec
+		Buckets: prometheus.ExponentialBuckets(.01, 2, 15),
+	})
+
+	hashRevSec = prometheus.NewHistogram(prometheus.HistogramOpts{
+		Namespace: "etcd",
+		Subsystem: "mvcc",
+		Name:      "hash_rev_duration_seconds",
+		Help:      "The latency distribution of storage hash by revision operation.",
+
+		// 100 MB usually takes 100 ms, so start with 10 MB of 10 ms
+		// lowest bucket start of upper bound 0.01 sec (10 ms) with factor 2
+		// highest bucket start of 0.01 sec * 2^14 == 163.84 sec
+		Buckets: prometheus.ExponentialBuckets(.01, 2, 15),
+	})
+
+	currentRev = prometheus.NewGaugeFunc(prometheus.GaugeOpts{
+		Namespace: "etcd_debugging",
+		Subsystem: "mvcc",
+		Name:      "current_revision",
+		Help:      "The current revision of store.",
+	},
+		func() float64 {
+			reportCurrentRevMu.RLock()
+			defer reportCurrentRevMu.RUnlock()
+			return reportCurrentRev()
+		},
+	)
+	// overridden by mvcc initialization
+	reportCurrentRevMu sync.RWMutex
+	reportCurrentRev   = func() float64 { return 0 }
+
+	compactRev = prometheus.NewGaugeFunc(prometheus.GaugeOpts{
+		Namespace: "etcd_debugging",
+		Subsystem: "mvcc",
+		Name:      "compact_revision",
+		Help:      "The revision of the last compaction in store.",
+	},
+		func() float64 {
+			reportCompactRevMu.RLock()
+			defer reportCompactRevMu.RUnlock()
+			return reportCompactRev()
+		},
+	)
+	// overridden by mvcc initialization
+	reportCompactRevMu sync.RWMutex
+	reportCompactRev   = func() float64 { return 0 }
+)
+
+func init() {
+	prometheus.MustRegister(rangeCounter)
+	prometheus.MustRegister(rangeCounterDebug)
+	prometheus.MustRegister(putCounter)
+	prometheus.MustRegister(putCounterDebug)
+	prometheus.MustRegister(deleteCounter)
+	prometheus.MustRegister(deleteCounterDebug)
+	prometheus.MustRegister(txnCounter)
+	prometheus.MustRegister(txnCounterDebug)
+	prometheus.MustRegister(keysGauge)
+	prometheus.MustRegister(watchStreamGauge)
+	prometheus.MustRegister(watcherGauge)
+	prometheus.MustRegister(slowWatcherGauge)
+	prometheus.MustRegister(totalEventsCounter)
+	prometheus.MustRegister(pendingEventsGauge)
+	prometheus.MustRegister(indexCompactionPauseMs)
+	prometheus.MustRegister(dbCompactionPauseMs)
+	prometheus.MustRegister(dbCompactionTotalMs)
+	prometheus.MustRegister(dbCompactionKeysCounter)
+	prometheus.MustRegister(dbTotalSize)
+	prometheus.MustRegister(dbTotalSizeDebug)
+	prometheus.MustRegister(dbTotalSizeInUse)
+	prometheus.MustRegister(dbOpenReadTxN)
+	prometheus.MustRegister(hashSec)
+	prometheus.MustRegister(hashRevSec)
+	prometheus.MustRegister(currentRev)
+	prometheus.MustRegister(compactRev)
+}
+
+// ReportEventReceived reports that an event is received.
+// This function should be called when the external systems received an
+// event from mvcc.Watcher.
+func ReportEventReceived(n int) {
+	pendingEventsGauge.Sub(float64(n))
+	totalEventsCounter.Add(float64(n))
+}
diff --git a/vendor/go.etcd.io/etcd/mvcc/metrics_txn.go b/vendor/go.etcd.io/etcd/mvcc/metrics_txn.go
new file mode 100644
index 0000000..64b629c
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/metrics_txn.go
@@ -0,0 +1,67 @@
+// 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 mvcc
+
+import "go.etcd.io/etcd/lease"
+
+type metricsTxnWrite struct {
+	TxnWrite
+	ranges  uint
+	puts    uint
+	deletes uint
+}
+
+func newMetricsTxnRead(tr TxnRead) TxnRead {
+	return &metricsTxnWrite{&txnReadWrite{tr}, 0, 0, 0}
+}
+
+func newMetricsTxnWrite(tw TxnWrite) TxnWrite {
+	return &metricsTxnWrite{tw, 0, 0, 0}
+}
+
+func (tw *metricsTxnWrite) Range(key, end []byte, ro RangeOptions) (*RangeResult, error) {
+	tw.ranges++
+	return tw.TxnWrite.Range(key, end, ro)
+}
+
+func (tw *metricsTxnWrite) DeleteRange(key, end []byte) (n, rev int64) {
+	tw.deletes++
+	return tw.TxnWrite.DeleteRange(key, end)
+}
+
+func (tw *metricsTxnWrite) Put(key, value []byte, lease lease.LeaseID) (rev int64) {
+	tw.puts++
+	return tw.TxnWrite.Put(key, value, lease)
+}
+
+func (tw *metricsTxnWrite) End() {
+	defer tw.TxnWrite.End()
+	if sum := tw.ranges + tw.puts + tw.deletes; sum > 1 {
+		txnCounter.Inc()
+		txnCounterDebug.Inc() // TODO: remove in 3.5 release
+	}
+
+	ranges := float64(tw.ranges)
+	rangeCounter.Add(ranges)
+	rangeCounterDebug.Add(ranges) // TODO: remove in 3.5 release
+
+	puts := float64(tw.puts)
+	putCounter.Add(puts)
+	putCounterDebug.Add(puts) // TODO: remove in 3.5 release
+
+	deletes := float64(tw.deletes)
+	deleteCounter.Add(deletes)
+	deleteCounterDebug.Add(deletes) // TODO: remove in 3.5 release
+}
diff --git a/vendor/go.etcd.io/etcd/mvcc/revision.go b/vendor/go.etcd.io/etcd/mvcc/revision.go
new file mode 100644
index 0000000..d621386
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/revision.go
@@ -0,0 +1,67 @@
+// Copyright 2015 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 mvcc
+
+import "encoding/binary"
+
+// revBytesLen is the byte length of a normal revision.
+// First 8 bytes is the revision.main in big-endian format. The 9th byte
+// is a '_'. The last 8 bytes is the revision.sub in big-endian format.
+const revBytesLen = 8 + 1 + 8
+
+// A revision indicates modification of the key-value space.
+// The set of changes that share same main revision changes the key-value space atomically.
+type revision struct {
+	// main is the main revision of a set of changes that happen atomically.
+	main int64
+
+	// sub is the sub revision of a change in a set of changes that happen
+	// atomically. Each change has different increasing sub revision in that
+	// set.
+	sub int64
+}
+
+func (a revision) GreaterThan(b revision) bool {
+	if a.main > b.main {
+		return true
+	}
+	if a.main < b.main {
+		return false
+	}
+	return a.sub > b.sub
+}
+
+func newRevBytes() []byte {
+	return make([]byte, revBytesLen, markedRevBytesLen)
+}
+
+func revToBytes(rev revision, bytes []byte) {
+	binary.BigEndian.PutUint64(bytes, uint64(rev.main))
+	bytes[8] = '_'
+	binary.BigEndian.PutUint64(bytes[9:], uint64(rev.sub))
+}
+
+func bytesToRev(bytes []byte) revision {
+	return revision{
+		main: int64(binary.BigEndian.Uint64(bytes[0:8])),
+		sub:  int64(binary.BigEndian.Uint64(bytes[9:])),
+	}
+}
+
+type revisions []revision
+
+func (a revisions) Len() int           { return len(a) }
+func (a revisions) Less(i, j int) bool { return a[j].GreaterThan(a[i]) }
+func (a revisions) Swap(i, j int)      { a[i], a[j] = a[j], a[i] }
diff --git a/vendor/go.etcd.io/etcd/mvcc/util.go b/vendor/go.etcd.io/etcd/mvcc/util.go
new file mode 100644
index 0000000..032621a
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/util.go
@@ -0,0 +1,57 @@
+// Copyright 2016 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 mvcc
+
+import (
+	"encoding/binary"
+	"fmt"
+
+	"go.etcd.io/etcd/mvcc/backend"
+	"go.etcd.io/etcd/mvcc/mvccpb"
+)
+
+func UpdateConsistentIndex(be backend.Backend, index uint64) {
+	tx := be.BatchTx()
+	tx.Lock()
+	defer tx.Unlock()
+
+	var oldi uint64
+	_, vs := tx.UnsafeRange(metaBucketName, consistentIndexKeyName, nil, 0)
+	if len(vs) != 0 {
+		oldi = binary.BigEndian.Uint64(vs[0])
+	}
+
+	if index <= oldi {
+		return
+	}
+
+	bs := make([]byte, 8)
+	binary.BigEndian.PutUint64(bs, index)
+	tx.UnsafePut(metaBucketName, consistentIndexKeyName, bs)
+}
+
+func WriteKV(be backend.Backend, kv mvccpb.KeyValue) {
+	ibytes := newRevBytes()
+	revToBytes(revision{main: kv.ModRevision}, ibytes)
+
+	d, err := kv.Marshal()
+	if err != nil {
+		panic(fmt.Errorf("cannot marshal event: %v", err))
+	}
+
+	be.BatchTx().Lock()
+	be.BatchTx().UnsafePut(keyBucketName, ibytes, d)
+	be.BatchTx().Unlock()
+}
diff --git a/vendor/go.etcd.io/etcd/mvcc/watchable_store.go b/vendor/go.etcd.io/etcd/mvcc/watchable_store.go
new file mode 100644
index 0000000..3cf491d
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/watchable_store.go
@@ -0,0 +1,552 @@
+// Copyright 2015 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 mvcc
+
+import (
+	"sync"
+	"time"
+
+	"go.etcd.io/etcd/lease"
+	"go.etcd.io/etcd/mvcc/backend"
+	"go.etcd.io/etcd/mvcc/mvccpb"
+	"go.uber.org/zap"
+)
+
+// non-const so modifiable by tests
+var (
+	// chanBufLen is the length of the buffered chan
+	// for sending out watched events.
+	// TODO: find a good buf value. 1024 is just a random one that
+	// seems to be reasonable.
+	chanBufLen = 1024
+
+	// maxWatchersPerSync is the number of watchers to sync in a single batch
+	maxWatchersPerSync = 512
+)
+
+type watchable interface {
+	watch(key, end []byte, startRev int64, id WatchID, ch chan<- WatchResponse, fcs ...FilterFunc) (*watcher, cancelFunc)
+	progress(w *watcher)
+	rev() int64
+}
+
+type watchableStore struct {
+	*store
+
+	// mu protects watcher groups and batches. It should never be locked
+	// before locking store.mu to avoid deadlock.
+	mu sync.RWMutex
+
+	// victims are watcher batches that were blocked on the watch channel
+	victims []watcherBatch
+	victimc chan struct{}
+
+	// contains all unsynced watchers that needs to sync with events that have happened
+	unsynced watcherGroup
+
+	// contains all synced watchers that are in sync with the progress of the store.
+	// The key of the map is the key that the watcher watches on.
+	synced watcherGroup
+
+	stopc chan struct{}
+	wg    sync.WaitGroup
+}
+
+// cancelFunc updates unsynced and synced maps when running
+// cancel operations.
+type cancelFunc func()
+
+func New(lg *zap.Logger, b backend.Backend, le lease.Lessor, ig ConsistentIndexGetter, cfg StoreConfig) ConsistentWatchableKV {
+	return newWatchableStore(lg, b, le, ig, cfg)
+}
+
+func newWatchableStore(lg *zap.Logger, b backend.Backend, le lease.Lessor, ig ConsistentIndexGetter, cfg StoreConfig) *watchableStore {
+	s := &watchableStore{
+		store:    NewStore(lg, b, le, ig, cfg),
+		victimc:  make(chan struct{}, 1),
+		unsynced: newWatcherGroup(),
+		synced:   newWatcherGroup(),
+		stopc:    make(chan struct{}),
+	}
+	s.store.ReadView = &readView{s}
+	s.store.WriteView = &writeView{s}
+	if s.le != nil {
+		// use this store as the deleter so revokes trigger watch events
+		s.le.SetRangeDeleter(func() lease.TxnDelete { return s.Write() })
+	}
+	s.wg.Add(2)
+	go s.syncWatchersLoop()
+	go s.syncVictimsLoop()
+	return s
+}
+
+func (s *watchableStore) Close() error {
+	close(s.stopc)
+	s.wg.Wait()
+	return s.store.Close()
+}
+
+func (s *watchableStore) NewWatchStream() WatchStream {
+	watchStreamGauge.Inc()
+	return &watchStream{
+		watchable: s,
+		ch:        make(chan WatchResponse, chanBufLen),
+		cancels:   make(map[WatchID]cancelFunc),
+		watchers:  make(map[WatchID]*watcher),
+	}
+}
+
+func (s *watchableStore) watch(key, end []byte, startRev int64, id WatchID, ch chan<- WatchResponse, fcs ...FilterFunc) (*watcher, cancelFunc) {
+	wa := &watcher{
+		key:    key,
+		end:    end,
+		minRev: startRev,
+		id:     id,
+		ch:     ch,
+		fcs:    fcs,
+	}
+
+	s.mu.Lock()
+	s.revMu.RLock()
+	synced := startRev > s.store.currentRev || startRev == 0
+	if synced {
+		wa.minRev = s.store.currentRev + 1
+		if startRev > wa.minRev {
+			wa.minRev = startRev
+		}
+	}
+	if synced {
+		s.synced.add(wa)
+	} else {
+		slowWatcherGauge.Inc()
+		s.unsynced.add(wa)
+	}
+	s.revMu.RUnlock()
+	s.mu.Unlock()
+
+	watcherGauge.Inc()
+
+	return wa, func() { s.cancelWatcher(wa) }
+}
+
+// cancelWatcher removes references of the watcher from the watchableStore
+func (s *watchableStore) cancelWatcher(wa *watcher) {
+	for {
+		s.mu.Lock()
+		if s.unsynced.delete(wa) {
+			slowWatcherGauge.Dec()
+			break
+		} else if s.synced.delete(wa) {
+			break
+		} else if wa.compacted {
+			break
+		} else if wa.ch == nil {
+			// already canceled (e.g., cancel/close race)
+			break
+		}
+
+		if !wa.victim {
+			panic("watcher not victim but not in watch groups")
+		}
+
+		var victimBatch watcherBatch
+		for _, wb := range s.victims {
+			if wb[wa] != nil {
+				victimBatch = wb
+				break
+			}
+		}
+		if victimBatch != nil {
+			slowWatcherGauge.Dec()
+			delete(victimBatch, wa)
+			break
+		}
+
+		// victim being processed so not accessible; retry
+		s.mu.Unlock()
+		time.Sleep(time.Millisecond)
+	}
+
+	watcherGauge.Dec()
+	wa.ch = nil
+	s.mu.Unlock()
+}
+
+func (s *watchableStore) Restore(b backend.Backend) error {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+	err := s.store.Restore(b)
+	if err != nil {
+		return err
+	}
+
+	for wa := range s.synced.watchers {
+		wa.restore = true
+		s.unsynced.add(wa)
+	}
+	s.synced = newWatcherGroup()
+	return nil
+}
+
+// syncWatchersLoop syncs the watcher in the unsynced map every 100ms.
+func (s *watchableStore) syncWatchersLoop() {
+	defer s.wg.Done()
+
+	for {
+		s.mu.RLock()
+		st := time.Now()
+		lastUnsyncedWatchers := s.unsynced.size()
+		s.mu.RUnlock()
+
+		unsyncedWatchers := 0
+		if lastUnsyncedWatchers > 0 {
+			unsyncedWatchers = s.syncWatchers()
+		}
+		syncDuration := time.Since(st)
+
+		waitDuration := 100 * time.Millisecond
+		// more work pending?
+		if unsyncedWatchers != 0 && lastUnsyncedWatchers > unsyncedWatchers {
+			// be fair to other store operations by yielding time taken
+			waitDuration = syncDuration
+		}
+
+		select {
+		case <-time.After(waitDuration):
+		case <-s.stopc:
+			return
+		}
+	}
+}
+
+// syncVictimsLoop tries to write precomputed watcher responses to
+// watchers that had a blocked watcher channel
+func (s *watchableStore) syncVictimsLoop() {
+	defer s.wg.Done()
+
+	for {
+		for s.moveVictims() != 0 {
+			// try to update all victim watchers
+		}
+		s.mu.RLock()
+		isEmpty := len(s.victims) == 0
+		s.mu.RUnlock()
+
+		var tickc <-chan time.Time
+		if !isEmpty {
+			tickc = time.After(10 * time.Millisecond)
+		}
+
+		select {
+		case <-tickc:
+		case <-s.victimc:
+		case <-s.stopc:
+			return
+		}
+	}
+}
+
+// moveVictims tries to update watches with already pending event data
+func (s *watchableStore) moveVictims() (moved int) {
+	s.mu.Lock()
+	victims := s.victims
+	s.victims = nil
+	s.mu.Unlock()
+
+	var newVictim watcherBatch
+	for _, wb := range victims {
+		// try to send responses again
+		for w, eb := range wb {
+			// watcher has observed the store up to, but not including, w.minRev
+			rev := w.minRev - 1
+			if w.send(WatchResponse{WatchID: w.id, Events: eb.evs, Revision: rev}) {
+				pendingEventsGauge.Add(float64(len(eb.evs)))
+			} else {
+				if newVictim == nil {
+					newVictim = make(watcherBatch)
+				}
+				newVictim[w] = eb
+				continue
+			}
+			moved++
+		}
+
+		// assign completed victim watchers to unsync/sync
+		s.mu.Lock()
+		s.store.revMu.RLock()
+		curRev := s.store.currentRev
+		for w, eb := range wb {
+			if newVictim != nil && newVictim[w] != nil {
+				// couldn't send watch response; stays victim
+				continue
+			}
+			w.victim = false
+			if eb.moreRev != 0 {
+				w.minRev = eb.moreRev
+			}
+			if w.minRev <= curRev {
+				s.unsynced.add(w)
+			} else {
+				slowWatcherGauge.Dec()
+				s.synced.add(w)
+			}
+		}
+		s.store.revMu.RUnlock()
+		s.mu.Unlock()
+	}
+
+	if len(newVictim) > 0 {
+		s.mu.Lock()
+		s.victims = append(s.victims, newVictim)
+		s.mu.Unlock()
+	}
+
+	return moved
+}
+
+// syncWatchers syncs unsynced watchers by:
+//	1. choose a set of watchers from the unsynced watcher group
+//	2. iterate over the set to get the minimum revision and remove compacted watchers
+//	3. use minimum revision to get all key-value pairs and send those events to watchers
+//	4. remove synced watchers in set from unsynced group and move to synced group
+func (s *watchableStore) syncWatchers() int {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+
+	if s.unsynced.size() == 0 {
+		return 0
+	}
+
+	s.store.revMu.RLock()
+	defer s.store.revMu.RUnlock()
+
+	// in order to find key-value pairs from unsynced watchers, we need to
+	// find min revision index, and these revisions can be used to
+	// query the backend store of key-value pairs
+	curRev := s.store.currentRev
+	compactionRev := s.store.compactMainRev
+
+	wg, minRev := s.unsynced.choose(maxWatchersPerSync, curRev, compactionRev)
+	minBytes, maxBytes := newRevBytes(), newRevBytes()
+	revToBytes(revision{main: minRev}, minBytes)
+	revToBytes(revision{main: curRev + 1}, maxBytes)
+
+	// UnsafeRange returns keys and values. And in boltdb, keys are revisions.
+	// values are actual key-value pairs in backend.
+	tx := s.store.b.ReadTx()
+	tx.RLock()
+	revs, vs := tx.UnsafeRange(keyBucketName, minBytes, maxBytes, 0)
+	var evs []mvccpb.Event
+	if s.store != nil && s.store.lg != nil {
+		evs = kvsToEvents(s.store.lg, wg, revs, vs)
+	} else {
+		// TODO: remove this in v3.5
+		evs = kvsToEvents(nil, wg, revs, vs)
+	}
+	tx.RUnlock()
+
+	var victims watcherBatch
+	wb := newWatcherBatch(wg, evs)
+	for w := range wg.watchers {
+		w.minRev = curRev + 1
+
+		eb, ok := wb[w]
+		if !ok {
+			// bring un-notified watcher to synced
+			s.synced.add(w)
+			s.unsynced.delete(w)
+			continue
+		}
+
+		if eb.moreRev != 0 {
+			w.minRev = eb.moreRev
+		}
+
+		if w.send(WatchResponse{WatchID: w.id, Events: eb.evs, Revision: curRev}) {
+			pendingEventsGauge.Add(float64(len(eb.evs)))
+		} else {
+			if victims == nil {
+				victims = make(watcherBatch)
+			}
+			w.victim = true
+		}
+
+		if w.victim {
+			victims[w] = eb
+		} else {
+			if eb.moreRev != 0 {
+				// stay unsynced; more to read
+				continue
+			}
+			s.synced.add(w)
+		}
+		s.unsynced.delete(w)
+	}
+	s.addVictim(victims)
+
+	vsz := 0
+	for _, v := range s.victims {
+		vsz += len(v)
+	}
+	slowWatcherGauge.Set(float64(s.unsynced.size() + vsz))
+
+	return s.unsynced.size()
+}
+
+// kvsToEvents gets all events for the watchers from all key-value pairs
+func kvsToEvents(lg *zap.Logger, wg *watcherGroup, revs, vals [][]byte) (evs []mvccpb.Event) {
+	for i, v := range vals {
+		var kv mvccpb.KeyValue
+		if err := kv.Unmarshal(v); err != nil {
+			if lg != nil {
+				lg.Panic("failed to unmarshal mvccpb.KeyValue", zap.Error(err))
+			} else {
+				plog.Panicf("cannot unmarshal event: %v", err)
+			}
+		}
+
+		if !wg.contains(string(kv.Key)) {
+			continue
+		}
+
+		ty := mvccpb.PUT
+		if isTombstone(revs[i]) {
+			ty = mvccpb.DELETE
+			// patch in mod revision so watchers won't skip
+			kv.ModRevision = bytesToRev(revs[i]).main
+		}
+		evs = append(evs, mvccpb.Event{Kv: &kv, Type: ty})
+	}
+	return evs
+}
+
+// notify notifies the fact that given event at the given rev just happened to
+// watchers that watch on the key of the event.
+func (s *watchableStore) notify(rev int64, evs []mvccpb.Event) {
+	var victim watcherBatch
+	for w, eb := range newWatcherBatch(&s.synced, evs) {
+		if eb.revs != 1 {
+			if s.store != nil && s.store.lg != nil {
+				s.store.lg.Panic(
+					"unexpected multiple revisions in watch notification",
+					zap.Int("number-of-revisions", eb.revs),
+				)
+			} else {
+				plog.Panicf("unexpected multiple revisions in notification")
+			}
+		}
+		if w.send(WatchResponse{WatchID: w.id, Events: eb.evs, Revision: rev}) {
+			pendingEventsGauge.Add(float64(len(eb.evs)))
+		} else {
+			// move slow watcher to victims
+			w.minRev = rev + 1
+			if victim == nil {
+				victim = make(watcherBatch)
+			}
+			w.victim = true
+			victim[w] = eb
+			s.synced.delete(w)
+			slowWatcherGauge.Inc()
+		}
+	}
+	s.addVictim(victim)
+}
+
+func (s *watchableStore) addVictim(victim watcherBatch) {
+	if victim == nil {
+		return
+	}
+	s.victims = append(s.victims, victim)
+	select {
+	case s.victimc <- struct{}{}:
+	default:
+	}
+}
+
+func (s *watchableStore) rev() int64 { return s.store.Rev() }
+
+func (s *watchableStore) progress(w *watcher) {
+	s.mu.RLock()
+	defer s.mu.RUnlock()
+
+	if _, ok := s.synced.watchers[w]; ok {
+		w.send(WatchResponse{WatchID: w.id, Revision: s.rev()})
+		// If the ch is full, this watcher is receiving events.
+		// We do not need to send progress at all.
+	}
+}
+
+type watcher struct {
+	// the watcher key
+	key []byte
+	// end indicates the end of the range to watch.
+	// If end is set, the watcher is on a range.
+	end []byte
+
+	// victim is set when ch is blocked and undergoing victim processing
+	victim bool
+
+	// compacted is set when the watcher is removed because of compaction
+	compacted bool
+
+	// restore is true when the watcher is being restored from leader snapshot
+	// which means that this watcher has just been moved from "synced" to "unsynced"
+	// watcher group, possibly with a future revision when it was first added
+	// to the synced watcher
+	// "unsynced" watcher revision must always be <= current revision,
+	// except when the watcher were to be moved from "synced" watcher group
+	restore bool
+
+	// minRev is the minimum revision update the watcher will accept
+	minRev int64
+	id     WatchID
+
+	fcs []FilterFunc
+	// a chan to send out the watch response.
+	// The chan might be shared with other watchers.
+	ch chan<- WatchResponse
+}
+
+func (w *watcher) send(wr WatchResponse) bool {
+	progressEvent := len(wr.Events) == 0
+
+	if len(w.fcs) != 0 {
+		ne := make([]mvccpb.Event, 0, len(wr.Events))
+		for i := range wr.Events {
+			filtered := false
+			for _, filter := range w.fcs {
+				if filter(wr.Events[i]) {
+					filtered = true
+					break
+				}
+			}
+			if !filtered {
+				ne = append(ne, wr.Events[i])
+			}
+		}
+		wr.Events = ne
+	}
+
+	// if all events are filtered out, we should send nothing.
+	if !progressEvent && len(wr.Events) == 0 {
+		return true
+	}
+	select {
+	case w.ch <- wr:
+		return true
+	default:
+		return false
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/mvcc/watchable_store_txn.go b/vendor/go.etcd.io/etcd/mvcc/watchable_store_txn.go
new file mode 100644
index 0000000..3bcfa4d
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/watchable_store_txn.go
@@ -0,0 +1,51 @@
+// 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 mvcc
+
+import "go.etcd.io/etcd/mvcc/mvccpb"
+
+func (tw *watchableStoreTxnWrite) End() {
+	changes := tw.Changes()
+	if len(changes) == 0 {
+		tw.TxnWrite.End()
+		return
+	}
+
+	rev := tw.Rev() + 1
+	evs := make([]mvccpb.Event, len(changes))
+	for i, change := range changes {
+		evs[i].Kv = &changes[i]
+		if change.CreateRevision == 0 {
+			evs[i].Type = mvccpb.DELETE
+			evs[i].Kv.ModRevision = rev
+		} else {
+			evs[i].Type = mvccpb.PUT
+		}
+	}
+
+	// end write txn under watchable store lock so the updates are visible
+	// when asynchronous event posting checks the current store revision
+	tw.s.mu.Lock()
+	tw.s.notify(rev, evs)
+	tw.TxnWrite.End()
+	tw.s.mu.Unlock()
+}
+
+type watchableStoreTxnWrite struct {
+	TxnWrite
+	s *watchableStore
+}
+
+func (s *watchableStore) Write() TxnWrite { return &watchableStoreTxnWrite{s.store.Write(), s} }
diff --git a/vendor/go.etcd.io/etcd/mvcc/watcher.go b/vendor/go.etcd.io/etcd/mvcc/watcher.go
new file mode 100644
index 0000000..2846d62
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/watcher.go
@@ -0,0 +1,193 @@
+// Copyright 2015 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 mvcc
+
+import (
+	"bytes"
+	"errors"
+	"sync"
+
+	"go.etcd.io/etcd/mvcc/mvccpb"
+)
+
+// AutoWatchID is the watcher ID passed in WatchStream.Watch when no
+// user-provided ID is available. If pass, an ID will automatically be assigned.
+const AutoWatchID WatchID = 0
+
+var (
+	ErrWatcherNotExist    = errors.New("mvcc: watcher does not exist")
+	ErrEmptyWatcherRange  = errors.New("mvcc: watcher range is empty")
+	ErrWatcherDuplicateID = errors.New("mvcc: duplicate watch ID provided on the WatchStream")
+)
+
+type WatchID int64
+
+// FilterFunc returns true if the given event should be filtered out.
+type FilterFunc func(e mvccpb.Event) bool
+
+type WatchStream interface {
+	// Watch creates a watcher. The watcher watches the events happening or
+	// happened on the given key or range [key, end) from the given startRev.
+	//
+	// The whole event history can be watched unless compacted.
+	// If "startRev" <=0, watch observes events after currentRev.
+	//
+	// The returned "id" is the ID of this watcher. It appears as WatchID
+	// in events that are sent to the created watcher through stream channel.
+	// The watch ID is used when it's not equal to AutoWatchID. Otherwise,
+	// an auto-generated watch ID is returned.
+	Watch(id WatchID, key, end []byte, startRev int64, fcs ...FilterFunc) (WatchID, error)
+
+	// Chan returns a chan. All watch response will be sent to the returned chan.
+	Chan() <-chan WatchResponse
+
+	// RequestProgress requests the progress of the watcher with given ID. The response
+	// will only be sent if the watcher is currently synced.
+	// The responses will be sent through the WatchRespone Chan attached
+	// with this stream to ensure correct ordering.
+	// The responses contains no events. The revision in the response is the progress
+	// of the watchers since the watcher is currently synced.
+	RequestProgress(id WatchID)
+
+	// Cancel cancels a watcher by giving its ID. If watcher does not exist, an error will be
+	// returned.
+	Cancel(id WatchID) error
+
+	// Close closes Chan and release all related resources.
+	Close()
+
+	// Rev returns the current revision of the KV the stream watches on.
+	Rev() int64
+}
+
+type WatchResponse struct {
+	// WatchID is the WatchID of the watcher this response sent to.
+	WatchID WatchID
+
+	// Events contains all the events that needs to send.
+	Events []mvccpb.Event
+
+	// Revision is the revision of the KV when the watchResponse is created.
+	// For a normal response, the revision should be the same as the last
+	// modified revision inside Events. For a delayed response to a unsynced
+	// watcher, the revision is greater than the last modified revision
+	// inside Events.
+	Revision int64
+
+	// CompactRevision is set when the watcher is cancelled due to compaction.
+	CompactRevision int64
+}
+
+// watchStream contains a collection of watchers that share
+// one streaming chan to send out watched events and other control events.
+type watchStream struct {
+	watchable watchable
+	ch        chan WatchResponse
+
+	mu sync.Mutex // guards fields below it
+	// nextID is the ID pre-allocated for next new watcher in this stream
+	nextID   WatchID
+	closed   bool
+	cancels  map[WatchID]cancelFunc
+	watchers map[WatchID]*watcher
+}
+
+// Watch creates a new watcher in the stream and returns its WatchID.
+func (ws *watchStream) Watch(id WatchID, key, end []byte, startRev int64, fcs ...FilterFunc) (WatchID, error) {
+	// prevent wrong range where key >= end lexicographically
+	// watch request with 'WithFromKey' has empty-byte range end
+	if len(end) != 0 && bytes.Compare(key, end) != -1 {
+		return -1, ErrEmptyWatcherRange
+	}
+
+	ws.mu.Lock()
+	defer ws.mu.Unlock()
+	if ws.closed {
+		return -1, ErrEmptyWatcherRange
+	}
+
+	if id == AutoWatchID {
+		for ws.watchers[ws.nextID] != nil {
+			ws.nextID++
+		}
+		id = ws.nextID
+		ws.nextID++
+	} else if _, ok := ws.watchers[id]; ok {
+		return -1, ErrWatcherDuplicateID
+	}
+
+	w, c := ws.watchable.watch(key, end, startRev, id, ws.ch, fcs...)
+
+	ws.cancels[id] = c
+	ws.watchers[id] = w
+	return id, nil
+}
+
+func (ws *watchStream) Chan() <-chan WatchResponse {
+	return ws.ch
+}
+
+func (ws *watchStream) Cancel(id WatchID) error {
+	ws.mu.Lock()
+	cancel, ok := ws.cancels[id]
+	w := ws.watchers[id]
+	ok = ok && !ws.closed
+	ws.mu.Unlock()
+
+	if !ok {
+		return ErrWatcherNotExist
+	}
+	cancel()
+
+	ws.mu.Lock()
+	// The watch isn't removed until cancel so that if Close() is called,
+	// it will wait for the cancel. Otherwise, Close() could close the
+	// watch channel while the store is still posting events.
+	if ww := ws.watchers[id]; ww == w {
+		delete(ws.cancels, id)
+		delete(ws.watchers, id)
+	}
+	ws.mu.Unlock()
+
+	return nil
+}
+
+func (ws *watchStream) Close() {
+	ws.mu.Lock()
+	defer ws.mu.Unlock()
+
+	for _, cancel := range ws.cancels {
+		cancel()
+	}
+	ws.closed = true
+	close(ws.ch)
+	watchStreamGauge.Dec()
+}
+
+func (ws *watchStream) Rev() int64 {
+	ws.mu.Lock()
+	defer ws.mu.Unlock()
+	return ws.watchable.rev()
+}
+
+func (ws *watchStream) RequestProgress(id WatchID) {
+	ws.mu.Lock()
+	w, ok := ws.watchers[id]
+	ws.mu.Unlock()
+	if !ok {
+		return
+	}
+	ws.watchable.progress(w)
+}
diff --git a/vendor/go.etcd.io/etcd/mvcc/watcher_group.go b/vendor/go.etcd.io/etcd/mvcc/watcher_group.go
new file mode 100644
index 0000000..151f0de
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/mvcc/watcher_group.go
@@ -0,0 +1,293 @@
+// Copyright 2016 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 mvcc
+
+import (
+	"fmt"
+	"math"
+
+	"go.etcd.io/etcd/mvcc/mvccpb"
+	"go.etcd.io/etcd/pkg/adt"
+)
+
+var (
+	// watchBatchMaxRevs is the maximum distinct revisions that
+	// may be sent to an unsynced watcher at a time. Declared as
+	// var instead of const for testing purposes.
+	watchBatchMaxRevs = 1000
+)
+
+type eventBatch struct {
+	// evs is a batch of revision-ordered events
+	evs []mvccpb.Event
+	// revs is the minimum unique revisions observed for this batch
+	revs int
+	// moreRev is first revision with more events following this batch
+	moreRev int64
+}
+
+func (eb *eventBatch) add(ev mvccpb.Event) {
+	if eb.revs > watchBatchMaxRevs {
+		// maxed out batch size
+		return
+	}
+
+	if len(eb.evs) == 0 {
+		// base case
+		eb.revs = 1
+		eb.evs = append(eb.evs, ev)
+		return
+	}
+
+	// revision accounting
+	ebRev := eb.evs[len(eb.evs)-1].Kv.ModRevision
+	evRev := ev.Kv.ModRevision
+	if evRev > ebRev {
+		eb.revs++
+		if eb.revs > watchBatchMaxRevs {
+			eb.moreRev = evRev
+			return
+		}
+	}
+
+	eb.evs = append(eb.evs, ev)
+}
+
+type watcherBatch map[*watcher]*eventBatch
+
+func (wb watcherBatch) add(w *watcher, ev mvccpb.Event) {
+	eb := wb[w]
+	if eb == nil {
+		eb = &eventBatch{}
+		wb[w] = eb
+	}
+	eb.add(ev)
+}
+
+// newWatcherBatch maps watchers to their matched events. It enables quick
+// events look up by watcher.
+func newWatcherBatch(wg *watcherGroup, evs []mvccpb.Event) watcherBatch {
+	if len(wg.watchers) == 0 {
+		return nil
+	}
+
+	wb := make(watcherBatch)
+	for _, ev := range evs {
+		for w := range wg.watcherSetByKey(string(ev.Kv.Key)) {
+			if ev.Kv.ModRevision >= w.minRev {
+				// don't double notify
+				wb.add(w, ev)
+			}
+		}
+	}
+	return wb
+}
+
+type watcherSet map[*watcher]struct{}
+
+func (w watcherSet) add(wa *watcher) {
+	if _, ok := w[wa]; ok {
+		panic("add watcher twice!")
+	}
+	w[wa] = struct{}{}
+}
+
+func (w watcherSet) union(ws watcherSet) {
+	for wa := range ws {
+		w.add(wa)
+	}
+}
+
+func (w watcherSet) delete(wa *watcher) {
+	if _, ok := w[wa]; !ok {
+		panic("removing missing watcher!")
+	}
+	delete(w, wa)
+}
+
+type watcherSetByKey map[string]watcherSet
+
+func (w watcherSetByKey) add(wa *watcher) {
+	set := w[string(wa.key)]
+	if set == nil {
+		set = make(watcherSet)
+		w[string(wa.key)] = set
+	}
+	set.add(wa)
+}
+
+func (w watcherSetByKey) delete(wa *watcher) bool {
+	k := string(wa.key)
+	if v, ok := w[k]; ok {
+		if _, ok := v[wa]; ok {
+			delete(v, wa)
+			if len(v) == 0 {
+				// remove the set; nothing left
+				delete(w, k)
+			}
+			return true
+		}
+	}
+	return false
+}
+
+// watcherGroup is a collection of watchers organized by their ranges
+type watcherGroup struct {
+	// keyWatchers has the watchers that watch on a single key
+	keyWatchers watcherSetByKey
+	// ranges has the watchers that watch a range; it is sorted by interval
+	ranges adt.IntervalTree
+	// watchers is the set of all watchers
+	watchers watcherSet
+}
+
+func newWatcherGroup() watcherGroup {
+	return watcherGroup{
+		keyWatchers: make(watcherSetByKey),
+		ranges:      adt.NewIntervalTree(),
+		watchers:    make(watcherSet),
+	}
+}
+
+// add puts a watcher in the group.
+func (wg *watcherGroup) add(wa *watcher) {
+	wg.watchers.add(wa)
+	if wa.end == nil {
+		wg.keyWatchers.add(wa)
+		return
+	}
+
+	// interval already registered?
+	ivl := adt.NewStringAffineInterval(string(wa.key), string(wa.end))
+	if iv := wg.ranges.Find(ivl); iv != nil {
+		iv.Val.(watcherSet).add(wa)
+		return
+	}
+
+	// not registered, put in interval tree
+	ws := make(watcherSet)
+	ws.add(wa)
+	wg.ranges.Insert(ivl, ws)
+}
+
+// contains is whether the given key has a watcher in the group.
+func (wg *watcherGroup) contains(key string) bool {
+	_, ok := wg.keyWatchers[key]
+	return ok || wg.ranges.Intersects(adt.NewStringAffinePoint(key))
+}
+
+// size gives the number of unique watchers in the group.
+func (wg *watcherGroup) size() int { return len(wg.watchers) }
+
+// delete removes a watcher from the group.
+func (wg *watcherGroup) delete(wa *watcher) bool {
+	if _, ok := wg.watchers[wa]; !ok {
+		return false
+	}
+	wg.watchers.delete(wa)
+	if wa.end == nil {
+		wg.keyWatchers.delete(wa)
+		return true
+	}
+
+	ivl := adt.NewStringAffineInterval(string(wa.key), string(wa.end))
+	iv := wg.ranges.Find(ivl)
+	if iv == nil {
+		return false
+	}
+
+	ws := iv.Val.(watcherSet)
+	delete(ws, wa)
+	if len(ws) == 0 {
+		// remove interval missing watchers
+		if ok := wg.ranges.Delete(ivl); !ok {
+			panic("could not remove watcher from interval tree")
+		}
+	}
+
+	return true
+}
+
+// choose selects watchers from the watcher group to update
+func (wg *watcherGroup) choose(maxWatchers int, curRev, compactRev int64) (*watcherGroup, int64) {
+	if len(wg.watchers) < maxWatchers {
+		return wg, wg.chooseAll(curRev, compactRev)
+	}
+	ret := newWatcherGroup()
+	for w := range wg.watchers {
+		if maxWatchers <= 0 {
+			break
+		}
+		maxWatchers--
+		ret.add(w)
+	}
+	return &ret, ret.chooseAll(curRev, compactRev)
+}
+
+func (wg *watcherGroup) chooseAll(curRev, compactRev int64) int64 {
+	minRev := int64(math.MaxInt64)
+	for w := range wg.watchers {
+		if w.minRev > curRev {
+			// after network partition, possibly choosing future revision watcher from restore operation
+			// with watch key "proxy-namespace__lostleader" and revision "math.MaxInt64 - 2"
+			// do not panic when such watcher had been moved from "synced" watcher during restore operation
+			if !w.restore {
+				panic(fmt.Errorf("watcher minimum revision %d should not exceed current revision %d", w.minRev, curRev))
+			}
+
+			// mark 'restore' done, since it's chosen
+			w.restore = false
+		}
+		if w.minRev < compactRev {
+			select {
+			case w.ch <- WatchResponse{WatchID: w.id, CompactRevision: compactRev}:
+				w.compacted = true
+				wg.delete(w)
+			default:
+				// retry next time
+			}
+			continue
+		}
+		if minRev > w.minRev {
+			minRev = w.minRev
+		}
+	}
+	return minRev
+}
+
+// watcherSetByKey gets the set of watchers that receive events on the given key.
+func (wg *watcherGroup) watcherSetByKey(key string) watcherSet {
+	wkeys := wg.keyWatchers[key]
+	wranges := wg.ranges.Stab(adt.NewStringAffinePoint(key))
+
+	// zero-copy cases
+	switch {
+	case len(wranges) == 0:
+		// no need to merge ranges or copy; reuse single-key set
+		return wkeys
+	case len(wranges) == 0 && len(wkeys) == 0:
+		return nil
+	case len(wranges) == 1 && len(wkeys) == 0:
+		return wranges[0].Val.(watcherSet)
+	}
+
+	// copy case
+	ret := make(watcherSet)
+	ret.union(wg.keyWatchers[key])
+	for _, item := range wranges {
+		ret.union(item.Val.(watcherSet))
+	}
+	return ret
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/adt/README.md b/vendor/go.etcd.io/etcd/pkg/adt/README.md
new file mode 100644
index 0000000..a2089cd
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/adt/README.md
@@ -0,0 +1,48 @@
+
+## Red-Black Tree
+
+*"Introduction to Algorithms" (Cormen et al, 3rd ed.), Chapter 13*
+
+1. Every node is either red or black.
+2. The root is black.
+3. Every leaf (NIL) is black.
+4. If a node is red, then both its children are black.
+5. For each node, all simple paths from the node to descendant leaves contain the
+same number of black nodes.
+
+For example,
+
+```go
+import (
+    "fmt"
+
+    "go.etcd.io/etcd/pkg/adt"
+)
+
+func main() {
+    ivt := adt.NewIntervalTree()
+    ivt.Insert(NewInt64Interval(510, 511), 0)
+    ivt.Insert(NewInt64Interval(82, 83), 0)
+    ivt.Insert(NewInt64Interval(830, 831), 0)
+    ...
+```
+
+After inserting the values `510`, `82`, `830`, `11`, `383`, `647`, `899`, `261`, `410`, `514`, `815`, `888`, `972`, `238`, `292`, `953`.
+
+![red-black-tree-01-insertion.png](img/red-black-tree-01-insertion.png)
+
+Deleting the node `514` should not trigger any rebalancing:
+
+![red-black-tree-02-delete-514.png](img/red-black-tree-02-delete-514.png)
+
+Deleting the node `11` triggers multiple rotates for rebalancing:
+
+![red-black-tree-03-delete-11.png](img/red-black-tree-03-delete-11.png)
+![red-black-tree-04-delete-11.png](img/red-black-tree-04-delete-11.png)
+![red-black-tree-05-delete-11.png](img/red-black-tree-05-delete-11.png)
+![red-black-tree-06-delete-11.png](img/red-black-tree-06-delete-11.png)
+![red-black-tree-07-delete-11.png](img/red-black-tree-07-delete-11.png)
+![red-black-tree-08-delete-11.png](img/red-black-tree-08-delete-11.png)
+![red-black-tree-09-delete-11.png](img/red-black-tree-09-delete-11.png)
+
+Try yourself at https://www.cs.usfca.edu/~galles/visualization/RedBlack.html.
diff --git a/vendor/go.etcd.io/etcd/pkg/adt/doc.go b/vendor/go.etcd.io/etcd/pkg/adt/doc.go
new file mode 100644
index 0000000..1a95591
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/adt/doc.go
@@ -0,0 +1,16 @@
+// Copyright 2016 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 adt implements useful abstract data types.
+package adt
diff --git a/vendor/go.etcd.io/etcd/pkg/adt/interval_tree.go b/vendor/go.etcd.io/etcd/pkg/adt/interval_tree.go
new file mode 100644
index 0000000..2e5b2dd
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/adt/interval_tree.go
@@ -0,0 +1,951 @@
+// Copyright 2016 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 adt
+
+import (
+	"bytes"
+	"fmt"
+	"math"
+	"strings"
+)
+
+// Comparable is an interface for trichotomic comparisons.
+type Comparable interface {
+	// Compare gives the result of a 3-way comparison
+	// a.Compare(b) = 1 => a > b
+	// a.Compare(b) = 0 => a == b
+	// a.Compare(b) = -1 => a < b
+	Compare(c Comparable) int
+}
+
+type rbcolor int
+
+const (
+	black rbcolor = iota
+	red
+)
+
+func (c rbcolor) String() string {
+	switch c {
+	case black:
+		return "black"
+	case red:
+		return "black"
+	default:
+		panic(fmt.Errorf("unknown color %d", c))
+	}
+}
+
+// Interval implements a Comparable interval [begin, end)
+// TODO: support different sorts of intervals: (a,b), [a,b], (a, b]
+type Interval struct {
+	Begin Comparable
+	End   Comparable
+}
+
+// Compare on an interval gives == if the interval overlaps.
+func (ivl *Interval) Compare(c Comparable) int {
+	ivl2 := c.(*Interval)
+	ivbCmpBegin := ivl.Begin.Compare(ivl2.Begin)
+	ivbCmpEnd := ivl.Begin.Compare(ivl2.End)
+	iveCmpBegin := ivl.End.Compare(ivl2.Begin)
+
+	// ivl is left of ivl2
+	if ivbCmpBegin < 0 && iveCmpBegin <= 0 {
+		return -1
+	}
+
+	// iv is right of iv2
+	if ivbCmpEnd >= 0 {
+		return 1
+	}
+
+	return 0
+}
+
+type intervalNode struct {
+	// iv is the interval-value pair entry.
+	iv IntervalValue
+	// max endpoint of all descendent nodes.
+	max Comparable
+	// left and right are sorted by low endpoint of key interval
+	left, right *intervalNode
+	// parent is the direct ancestor of the node
+	parent *intervalNode
+	c      rbcolor
+}
+
+func (x *intervalNode) color(sentinel *intervalNode) rbcolor {
+	if x == sentinel {
+		return black
+	}
+	return x.c
+}
+
+func (x *intervalNode) height(sentinel *intervalNode) int {
+	if x == sentinel {
+		return 0
+	}
+	ld := x.left.height(sentinel)
+	rd := x.right.height(sentinel)
+	if ld < rd {
+		return rd + 1
+	}
+	return ld + 1
+}
+
+func (x *intervalNode) min(sentinel *intervalNode) *intervalNode {
+	for x.left != sentinel {
+		x = x.left
+	}
+	return x
+}
+
+// successor is the next in-order node in the tree
+func (x *intervalNode) successor(sentinel *intervalNode) *intervalNode {
+	if x.right != sentinel {
+		return x.right.min(sentinel)
+	}
+	y := x.parent
+	for y != sentinel && x == y.right {
+		x = y
+		y = y.parent
+	}
+	return y
+}
+
+// updateMax updates the maximum values for a node and its ancestors
+func (x *intervalNode) updateMax(sentinel *intervalNode) {
+	for x != sentinel {
+		oldmax := x.max
+		max := x.iv.Ivl.End
+		if x.left != sentinel && x.left.max.Compare(max) > 0 {
+			max = x.left.max
+		}
+		if x.right != sentinel && x.right.max.Compare(max) > 0 {
+			max = x.right.max
+		}
+		if oldmax.Compare(max) == 0 {
+			break
+		}
+		x.max = max
+		x = x.parent
+	}
+}
+
+type nodeVisitor func(n *intervalNode) bool
+
+// visit will call a node visitor on each node that overlaps the given interval
+func (x *intervalNode) visit(iv *Interval, sentinel *intervalNode, nv nodeVisitor) bool {
+	if x == sentinel {
+		return true
+	}
+	v := iv.Compare(&x.iv.Ivl)
+	switch {
+	case v < 0:
+		if !x.left.visit(iv, sentinel, nv) {
+			return false
+		}
+	case v > 0:
+		maxiv := Interval{x.iv.Ivl.Begin, x.max}
+		if maxiv.Compare(iv) == 0 {
+			if !x.left.visit(iv, sentinel, nv) || !x.right.visit(iv, sentinel, nv) {
+				return false
+			}
+		}
+	default:
+		if !x.left.visit(iv, sentinel, nv) || !nv(x) || !x.right.visit(iv, sentinel, nv) {
+			return false
+		}
+	}
+	return true
+}
+
+// IntervalValue represents a range tree node that contains a range and a value.
+type IntervalValue struct {
+	Ivl Interval
+	Val interface{}
+}
+
+// IntervalTree represents a (mostly) textbook implementation of the
+// "Introduction to Algorithms" (Cormen et al, 3rd ed.) chapter 13 red-black tree
+// and chapter 14.3 interval tree with search supporting "stabbing queries".
+type IntervalTree interface {
+	// Insert adds a node with the given interval into the tree.
+	Insert(ivl Interval, val interface{})
+	// Delete removes the node with the given interval from the tree, returning
+	// true if a node is in fact removed.
+	Delete(ivl Interval) bool
+	// Len gives the number of elements in the tree.
+	Len() int
+	// Height is the number of levels in the tree; one node has height 1.
+	Height() int
+	// MaxHeight is the expected maximum tree height given the number of nodes.
+	MaxHeight() int
+	// Visit calls a visitor function on every tree node intersecting the given interval.
+	// It will visit each interval [x, y) in ascending order sorted on x.
+	Visit(ivl Interval, ivv IntervalVisitor)
+	// Find gets the IntervalValue for the node matching the given interval
+	Find(ivl Interval) *IntervalValue
+	// Intersects returns true if there is some tree node intersecting the given interval.
+	Intersects(iv Interval) bool
+	// Contains returns true if the interval tree's keys cover the entire given interval.
+	Contains(ivl Interval) bool
+	// Stab returns a slice with all elements in the tree intersecting the interval.
+	Stab(iv Interval) []*IntervalValue
+	// Union merges a given interval tree into the receiver.
+	Union(inIvt IntervalTree, ivl Interval)
+}
+
+// NewIntervalTree returns a new interval tree.
+func NewIntervalTree() IntervalTree {
+	sentinel := &intervalNode{
+		iv:     IntervalValue{},
+		max:    nil,
+		left:   nil,
+		right:  nil,
+		parent: nil,
+		c:      black,
+	}
+	return &intervalTree{
+		root:     sentinel,
+		count:    0,
+		sentinel: sentinel,
+	}
+}
+
+type intervalTree struct {
+	root  *intervalNode
+	count int
+
+	// red-black NIL node
+	// use 'sentinel' as a dummy object to simplify boundary conditions
+	// use the sentinel to treat a nil child of a node x as an ordinary node whose parent is x
+	// use one shared sentinel to represent all nil leaves and the root's parent
+	sentinel *intervalNode
+}
+
+// TODO: make this consistent with textbook implementation
+//
+// "Introduction to Algorithms" (Cormen et al, 3rd ed.), chapter 13.4, p324
+//
+//	 0. RB-DELETE(T, z)
+//	 1.
+//	 2. y = z
+//	 3. y-original-color = y.color
+//	 4.
+//	 5. if z.left == T.nil
+//	 6. 	x = z.right
+//	 7. 	RB-TRANSPLANT(T, z, z.right)
+//	 8. else if z.right == T.nil
+//	 9. 	x = z.left
+//	10. 	RB-TRANSPLANT(T, z, z.left)
+//	11. else
+//	12. 	y = TREE-MINIMUM(z.right)
+//	13. 	y-original-color = y.color
+//	14. 	x = y.right
+//	15. 	if y.p == z
+//	16. 		x.p = y
+//	17. 	else
+//	18. 		RB-TRANSPLANT(T, y, y.right)
+//	19. 		y.right = z.right
+//	20. 		y.right.p = y
+//	21. 	RB-TRANSPLANT(T, z, y)
+//	22. 	y.left = z.left
+//	23. 	y.left.p = y
+//	24. 	y.color = z.color
+//	25.
+//	26. if y-original-color == BLACK
+//	27. 	RB-DELETE-FIXUP(T, x)
+
+// Delete removes the node with the given interval from the tree, returning
+// true if a node is in fact removed.
+func (ivt *intervalTree) Delete(ivl Interval) bool {
+	z := ivt.find(ivl)
+	if z == ivt.sentinel {
+		return false
+	}
+
+	y := z
+	if z.left != ivt.sentinel && z.right != ivt.sentinel {
+		y = z.successor(ivt.sentinel)
+	}
+
+	x := ivt.sentinel
+	if y.left != ivt.sentinel {
+		x = y.left
+	} else if y.right != ivt.sentinel {
+		x = y.right
+	}
+
+	x.parent = y.parent
+
+	if y.parent == ivt.sentinel {
+		ivt.root = x
+	} else {
+		if y == y.parent.left {
+			y.parent.left = x
+		} else {
+			y.parent.right = x
+		}
+		y.parent.updateMax(ivt.sentinel)
+	}
+	if y != z {
+		z.iv = y.iv
+		z.updateMax(ivt.sentinel)
+	}
+
+	if y.color(ivt.sentinel) == black {
+		ivt.deleteFixup(x)
+	}
+
+	ivt.count--
+	return true
+}
+
+// "Introduction to Algorithms" (Cormen et al, 3rd ed.), chapter 13.4, p326
+//
+//	 0. RB-DELETE-FIXUP(T, z)
+//	 1.
+//	 2. while x ≠ T.root and x.color == BLACK
+//	 3. 	if x == x.p.left
+//	 4. 		w = x.p.right
+//	 5. 		if w.color == RED
+//	 6. 			w.color = BLACK
+//	 7. 			x.p.color = RED
+//	 8. 			LEFT-ROTATE(T, x, p)
+//	 9. 		if w.left.color == BLACK and w.right.color == BLACK
+//	10. 			w.color = RED
+//	11. 			x = x.p
+//	12. 		else if w.right.color == BLACK
+//	13. 				w.left.color = BLACK
+//	14. 				w.color = RED
+//	15. 				RIGHT-ROTATE(T, w)
+//	16. 				w = w.p.right
+//	17. 			w.color = x.p.color
+//	18. 			x.p.color = BLACK
+//	19. 			LEFT-ROTATE(T, w.p)
+//	20. 			x = T.root
+//	21. 	else
+//	22. 		w = x.p.left
+//	23. 		if w.color == RED
+//	24. 			w.color = BLACK
+//	25. 			x.p.color = RED
+//	26. 			RIGHT-ROTATE(T, x, p)
+//	27. 		if w.right.color == BLACK and w.left.color == BLACK
+//	28. 			w.color = RED
+//	29. 			x = x.p
+//	30. 		else if w.left.color == BLACK
+//	31. 				w.right.color = BLACK
+//	32. 				w.color = RED
+//	33. 				LEFT-ROTATE(T, w)
+//	34. 				w = w.p.left
+//	35. 			w.color = x.p.color
+//	36. 			x.p.color = BLACK
+//	37. 			RIGHT-ROTATE(T, w.p)
+//	38. 			x = T.root
+//	39.
+//	40. x.color = BLACK
+//
+func (ivt *intervalTree) deleteFixup(x *intervalNode) {
+	for x != ivt.root && x.color(ivt.sentinel) == black {
+		if x == x.parent.left { // line 3-20
+			w := x.parent.right
+			if w.color(ivt.sentinel) == red {
+				w.c = black
+				x.parent.c = red
+				ivt.rotateLeft(x.parent)
+				w = x.parent.right
+			}
+			if w == nil {
+				break
+			}
+			if w.left.color(ivt.sentinel) == black && w.right.color(ivt.sentinel) == black {
+				w.c = red
+				x = x.parent
+			} else {
+				if w.right.color(ivt.sentinel) == black {
+					w.left.c = black
+					w.c = red
+					ivt.rotateRight(w)
+					w = x.parent.right
+				}
+				w.c = x.parent.color(ivt.sentinel)
+				x.parent.c = black
+				w.right.c = black
+				ivt.rotateLeft(x.parent)
+				x = ivt.root
+			}
+		} else { // line 22-38
+			// same as above but with left and right exchanged
+			w := x.parent.left
+			if w.color(ivt.sentinel) == red {
+				w.c = black
+				x.parent.c = red
+				ivt.rotateRight(x.parent)
+				w = x.parent.left
+			}
+			if w == nil {
+				break
+			}
+			if w.left.color(ivt.sentinel) == black && w.right.color(ivt.sentinel) == black {
+				w.c = red
+				x = x.parent
+			} else {
+				if w.left.color(ivt.sentinel) == black {
+					w.right.c = black
+					w.c = red
+					ivt.rotateLeft(w)
+					w = x.parent.left
+				}
+				w.c = x.parent.color(ivt.sentinel)
+				x.parent.c = black
+				w.left.c = black
+				ivt.rotateRight(x.parent)
+				x = ivt.root
+			}
+		}
+	}
+
+	if x != nil {
+		x.c = black
+	}
+}
+
+func (ivt *intervalTree) createIntervalNode(ivl Interval, val interface{}) *intervalNode {
+	return &intervalNode{
+		iv:     IntervalValue{ivl, val},
+		max:    ivl.End,
+		c:      red,
+		left:   ivt.sentinel,
+		right:  ivt.sentinel,
+		parent: ivt.sentinel,
+	}
+}
+
+// TODO: make this consistent with textbook implementation
+//
+// "Introduction to Algorithms" (Cormen et al, 3rd ed.), chapter 13.3, p315
+//
+//	 0. RB-INSERT(T, z)
+//	 1.
+//	 2. y = T.nil
+//	 3. x = T.root
+//	 4.
+//	 5. while x ≠ T.nil
+//	 6. 	y = x
+//	 7. 	if z.key < x.key
+//	 8. 		x = x.left
+//	 9. 	else
+//	10. 		x = x.right
+//	11.
+//	12. z.p = y
+//	13.
+//	14. if y == T.nil
+//	15. 	T.root = z
+//	16. else if z.key < y.key
+//	17. 	y.left = z
+//	18. else
+//	19. 	y.right = z
+//	20.
+//	21. z.left = T.nil
+//	22. z.right = T.nil
+//	23. z.color = RED
+//	24.
+//	25. RB-INSERT-FIXUP(T, z)
+
+// Insert adds a node with the given interval into the tree.
+func (ivt *intervalTree) Insert(ivl Interval, val interface{}) {
+	y := ivt.sentinel
+	z := ivt.createIntervalNode(ivl, val)
+	x := ivt.root
+	for x != ivt.sentinel {
+		y = x
+		if z.iv.Ivl.Begin.Compare(x.iv.Ivl.Begin) < 0 {
+			x = x.left
+		} else {
+			x = x.right
+		}
+	}
+
+	z.parent = y
+	if y == ivt.sentinel {
+		ivt.root = z
+	} else {
+		if z.iv.Ivl.Begin.Compare(y.iv.Ivl.Begin) < 0 {
+			y.left = z
+		} else {
+			y.right = z
+		}
+		y.updateMax(ivt.sentinel)
+	}
+	z.c = red
+
+	ivt.insertFixup(z)
+	ivt.count++
+}
+
+// "Introduction to Algorithms" (Cormen et al, 3rd ed.), chapter 13.3, p316
+//
+//	 0. RB-INSERT-FIXUP(T, z)
+//	 1.
+//	 2. while z.p.color == RED
+//	 3. 	if z.p == z.p.p.left
+//	 4. 		y = z.p.p.right
+//	 5. 		if y.color == RED
+//	 6. 			z.p.color = BLACK
+//	 7. 			y.color = BLACK
+//	 8. 			z.p.p.color = RED
+//	 9. 			z = z.p.p
+//	10. 		else if z == z.p.right
+//	11. 				z = z.p
+//	12. 				LEFT-ROTATE(T, z)
+//	13. 			z.p.color = BLACK
+//	14. 			z.p.p.color = RED
+//	15. 			RIGHT-ROTATE(T, z.p.p)
+//	16. 	else
+//	17. 		y = z.p.p.left
+//	18. 		if y.color == RED
+//	19. 			z.p.color = BLACK
+//	20. 			y.color = BLACK
+//	21. 			z.p.p.color = RED
+//	22. 			z = z.p.p
+//	23. 		else if z == z.p.right
+//	24. 				z = z.p
+//	25. 				RIGHT-ROTATE(T, z)
+//	26. 			z.p.color = BLACK
+//	27. 			z.p.p.color = RED
+//	28. 			LEFT-ROTATE(T, z.p.p)
+//	29.
+//	30. T.root.color = BLACK
+//
+func (ivt *intervalTree) insertFixup(z *intervalNode) {
+	for z.parent.color(ivt.sentinel) == red {
+		if z.parent == z.parent.parent.left { // line 3-15
+
+			y := z.parent.parent.right
+			if y.color(ivt.sentinel) == red {
+				y.c = black
+				z.parent.c = black
+				z.parent.parent.c = red
+				z = z.parent.parent
+			} else {
+				if z == z.parent.right {
+					z = z.parent
+					ivt.rotateLeft(z)
+				}
+				z.parent.c = black
+				z.parent.parent.c = red
+				ivt.rotateRight(z.parent.parent)
+			}
+		} else { // line 16-28
+			// same as then with left/right exchanged
+			y := z.parent.parent.left
+			if y.color(ivt.sentinel) == red {
+				y.c = black
+				z.parent.c = black
+				z.parent.parent.c = red
+				z = z.parent.parent
+			} else {
+				if z == z.parent.left {
+					z = z.parent
+					ivt.rotateRight(z)
+				}
+				z.parent.c = black
+				z.parent.parent.c = red
+				ivt.rotateLeft(z.parent.parent)
+			}
+		}
+	}
+
+	// line 30
+	ivt.root.c = black
+}
+
+// rotateLeft moves x so it is left of its right child
+//
+// "Introduction to Algorithms" (Cormen et al, 3rd ed.), chapter 13.2, p313
+//
+//	 0. LEFT-ROTATE(T, x)
+//	 1.
+//	 2. y = x.right
+//	 3. x.right = y.left
+//	 4.
+//	 5. if y.left ≠ T.nil
+//	 6. 	y.left.p = x
+//	 7.
+//	 8. y.p = x.p
+//	 9.
+//	10. if x.p == T.nil
+//	11. 	T.root = y
+//	12. else if x == x.p.left
+//	13. 	x.p.left = y
+//	14. else
+//	15. 	x.p.right = y
+//	16.
+//	17. y.left = x
+//	18. x.p = y
+//
+func (ivt *intervalTree) rotateLeft(x *intervalNode) {
+	// rotateLeft x must have right child
+	if x.right == ivt.sentinel {
+		return
+	}
+
+	// line 2-3
+	y := x.right
+	x.right = y.left
+
+	// line 5-6
+	if y.left != ivt.sentinel {
+		y.left.parent = x
+	}
+	x.updateMax(ivt.sentinel)
+
+	// line 10-15, 18
+	ivt.replaceParent(x, y)
+
+	// line 17
+	y.left = x
+	y.updateMax(ivt.sentinel)
+}
+
+// rotateRight moves x so it is right of its left child
+//
+//	 0. RIGHT-ROTATE(T, x)
+//	 1.
+//	 2. y = x.left
+//	 3. x.left = y.right
+//	 4.
+//	 5. if y.right ≠ T.nil
+//	 6. 	y.right.p = x
+//	 7.
+//	 8. y.p = x.p
+//	 9.
+//	10. if x.p == T.nil
+//	11. 	T.root = y
+//	12. else if x == x.p.right
+//	13. 	x.p.right = y
+//	14. else
+//	15. 	x.p.left = y
+//	16.
+//	17. y.right = x
+//	18. x.p = y
+//
+func (ivt *intervalTree) rotateRight(x *intervalNode) {
+	// rotateRight x must have left child
+	if x.left == ivt.sentinel {
+		return
+	}
+
+	// line 2-3
+	y := x.left
+	x.left = y.right
+
+	// line 5-6
+	if y.right != ivt.sentinel {
+		y.right.parent = x
+	}
+	x.updateMax(ivt.sentinel)
+
+	// line 10-15, 18
+	ivt.replaceParent(x, y)
+
+	// line 17
+	y.right = x
+	y.updateMax(ivt.sentinel)
+}
+
+// replaceParent replaces x's parent with y
+func (ivt *intervalTree) replaceParent(x *intervalNode, y *intervalNode) {
+	y.parent = x.parent
+	if x.parent == ivt.sentinel {
+		ivt.root = y
+	} else {
+		if x == x.parent.left {
+			x.parent.left = y
+		} else {
+			x.parent.right = y
+		}
+		x.parent.updateMax(ivt.sentinel)
+	}
+	x.parent = y
+}
+
+// Len gives the number of elements in the tree
+func (ivt *intervalTree) Len() int { return ivt.count }
+
+// Height is the number of levels in the tree; one node has height 1.
+func (ivt *intervalTree) Height() int { return ivt.root.height(ivt.sentinel) }
+
+// MaxHeight is the expected maximum tree height given the number of nodes
+func (ivt *intervalTree) MaxHeight() int {
+	return int((2 * math.Log2(float64(ivt.Len()+1))) + 0.5)
+}
+
+// IntervalVisitor is used on tree searches; return false to stop searching.
+type IntervalVisitor func(n *IntervalValue) bool
+
+// Visit calls a visitor function on every tree node intersecting the given interval.
+// It will visit each interval [x, y) in ascending order sorted on x.
+func (ivt *intervalTree) Visit(ivl Interval, ivv IntervalVisitor) {
+	ivt.root.visit(&ivl, ivt.sentinel, func(n *intervalNode) bool { return ivv(&n.iv) })
+}
+
+// find the exact node for a given interval
+func (ivt *intervalTree) find(ivl Interval) *intervalNode {
+	ret := ivt.sentinel
+	f := func(n *intervalNode) bool {
+		if n.iv.Ivl != ivl {
+			return true
+		}
+		ret = n
+		return false
+	}
+	ivt.root.visit(&ivl, ivt.sentinel, f)
+	return ret
+}
+
+// Find gets the IntervalValue for the node matching the given interval
+func (ivt *intervalTree) Find(ivl Interval) (ret *IntervalValue) {
+	n := ivt.find(ivl)
+	if n == ivt.sentinel {
+		return nil
+	}
+	return &n.iv
+}
+
+// Intersects returns true if there is some tree node intersecting the given interval.
+func (ivt *intervalTree) Intersects(iv Interval) bool {
+	x := ivt.root
+	for x != ivt.sentinel && iv.Compare(&x.iv.Ivl) != 0 {
+		if x.left != ivt.sentinel && x.left.max.Compare(iv.Begin) > 0 {
+			x = x.left
+		} else {
+			x = x.right
+		}
+	}
+	return x != ivt.sentinel
+}
+
+// Contains returns true if the interval tree's keys cover the entire given interval.
+func (ivt *intervalTree) Contains(ivl Interval) bool {
+	var maxEnd, minBegin Comparable
+
+	isContiguous := true
+	ivt.Visit(ivl, func(n *IntervalValue) bool {
+		if minBegin == nil {
+			minBegin = n.Ivl.Begin
+			maxEnd = n.Ivl.End
+			return true
+		}
+		if maxEnd.Compare(n.Ivl.Begin) < 0 {
+			isContiguous = false
+			return false
+		}
+		if n.Ivl.End.Compare(maxEnd) > 0 {
+			maxEnd = n.Ivl.End
+		}
+		return true
+	})
+
+	return isContiguous && minBegin != nil && maxEnd.Compare(ivl.End) >= 0 && minBegin.Compare(ivl.Begin) <= 0
+}
+
+// Stab returns a slice with all elements in the tree intersecting the interval.
+func (ivt *intervalTree) Stab(iv Interval) (ivs []*IntervalValue) {
+	if ivt.count == 0 {
+		return nil
+	}
+	f := func(n *IntervalValue) bool { ivs = append(ivs, n); return true }
+	ivt.Visit(iv, f)
+	return ivs
+}
+
+// Union merges a given interval tree into the receiver.
+func (ivt *intervalTree) Union(inIvt IntervalTree, ivl Interval) {
+	f := func(n *IntervalValue) bool {
+		ivt.Insert(n.Ivl, n.Val)
+		return true
+	}
+	inIvt.Visit(ivl, f)
+}
+
+type visitedInterval struct {
+	root  Interval
+	left  Interval
+	right Interval
+	color rbcolor
+	depth int
+}
+
+func (vi visitedInterval) String() string {
+	bd := new(strings.Builder)
+	bd.WriteString(fmt.Sprintf("root [%v,%v,%v], left [%v,%v], right [%v,%v], depth %d",
+		vi.root.Begin, vi.root.End, vi.color,
+		vi.left.Begin, vi.left.End,
+		vi.right.Begin, vi.right.End,
+		vi.depth,
+	))
+	return bd.String()
+}
+
+// visitLevel traverses tree in level order.
+// used for testing
+func (ivt *intervalTree) visitLevel() []visitedInterval {
+	if ivt.root == ivt.sentinel {
+		return nil
+	}
+
+	rs := make([]visitedInterval, 0, ivt.Len())
+
+	type pair struct {
+		node  *intervalNode
+		depth int
+	}
+	queue := []pair{{ivt.root, 0}}
+	for len(queue) > 0 {
+		f := queue[0]
+		queue = queue[1:]
+
+		vi := visitedInterval{
+			root:  f.node.iv.Ivl,
+			color: f.node.color(ivt.sentinel),
+			depth: f.depth,
+		}
+		if f.node.left != ivt.sentinel {
+			vi.left = f.node.left.iv.Ivl
+			queue = append(queue, pair{f.node.left, f.depth + 1})
+		}
+		if f.node.right != ivt.sentinel {
+			vi.right = f.node.right.iv.Ivl
+			queue = append(queue, pair{f.node.right, f.depth + 1})
+		}
+
+		rs = append(rs, vi)
+	}
+
+	return rs
+}
+
+type StringComparable string
+
+func (s StringComparable) Compare(c Comparable) int {
+	sc := c.(StringComparable)
+	if s < sc {
+		return -1
+	}
+	if s > sc {
+		return 1
+	}
+	return 0
+}
+
+func NewStringInterval(begin, end string) Interval {
+	return Interval{StringComparable(begin), StringComparable(end)}
+}
+
+func NewStringPoint(s string) Interval {
+	return Interval{StringComparable(s), StringComparable(s + "\x00")}
+}
+
+// StringAffineComparable treats "" as > all other strings
+type StringAffineComparable string
+
+func (s StringAffineComparable) Compare(c Comparable) int {
+	sc := c.(StringAffineComparable)
+
+	if len(s) == 0 {
+		if len(sc) == 0 {
+			return 0
+		}
+		return 1
+	}
+	if len(sc) == 0 {
+		return -1
+	}
+
+	if s < sc {
+		return -1
+	}
+	if s > sc {
+		return 1
+	}
+	return 0
+}
+
+func NewStringAffineInterval(begin, end string) Interval {
+	return Interval{StringAffineComparable(begin), StringAffineComparable(end)}
+}
+
+func NewStringAffinePoint(s string) Interval {
+	return NewStringAffineInterval(s, s+"\x00")
+}
+
+func NewInt64Interval(a int64, b int64) Interval {
+	return Interval{Int64Comparable(a), Int64Comparable(b)}
+}
+
+func newInt64EmptyInterval() Interval {
+	return Interval{Begin: nil, End: nil}
+}
+
+func NewInt64Point(a int64) Interval {
+	return Interval{Int64Comparable(a), Int64Comparable(a + 1)}
+}
+
+type Int64Comparable int64
+
+func (v Int64Comparable) Compare(c Comparable) int {
+	vc := c.(Int64Comparable)
+	cmp := v - vc
+	if cmp < 0 {
+		return -1
+	}
+	if cmp > 0 {
+		return 1
+	}
+	return 0
+}
+
+// BytesAffineComparable treats empty byte arrays as > all other byte arrays
+type BytesAffineComparable []byte
+
+func (b BytesAffineComparable) Compare(c Comparable) int {
+	bc := c.(BytesAffineComparable)
+
+	if len(b) == 0 {
+		if len(bc) == 0 {
+			return 0
+		}
+		return 1
+	}
+	if len(bc) == 0 {
+		return -1
+	}
+
+	return bytes.Compare(b, bc)
+}
+
+func NewBytesAffineInterval(begin, end []byte) Interval {
+	return Interval{BytesAffineComparable(begin), BytesAffineComparable(end)}
+}
+
+func NewBytesAffinePoint(b []byte) Interval {
+	be := make([]byte, len(b)+1)
+	copy(be, b)
+	be[len(b)] = 0
+	return NewBytesAffineInterval(b, be)
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/contention/contention.go b/vendor/go.etcd.io/etcd/pkg/contention/contention.go
new file mode 100644
index 0000000..26ce9a2
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/contention/contention.go
@@ -0,0 +1,69 @@
+// Copyright 2016 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 contention
+
+import (
+	"sync"
+	"time"
+)
+
+// TimeoutDetector detects routine starvations by
+// observing the actual time duration to finish an action
+// or between two events that should happen in a fixed
+// interval. If the observed duration is longer than
+// the expectation, the detector will report the result.
+type TimeoutDetector struct {
+	mu          sync.Mutex // protects all
+	maxDuration time.Duration
+	// map from event to time
+	// time is the last seen time of the event.
+	records map[uint64]time.Time
+}
+
+// NewTimeoutDetector creates the TimeoutDetector.
+func NewTimeoutDetector(maxDuration time.Duration) *TimeoutDetector {
+	return &TimeoutDetector{
+		maxDuration: maxDuration,
+		records:     make(map[uint64]time.Time),
+	}
+}
+
+// Reset resets the NewTimeoutDetector.
+func (td *TimeoutDetector) Reset() {
+	td.mu.Lock()
+	defer td.mu.Unlock()
+
+	td.records = make(map[uint64]time.Time)
+}
+
+// Observe observes an event for given id. It returns false and exceeded duration
+// if the interval is longer than the expectation.
+func (td *TimeoutDetector) Observe(which uint64) (bool, time.Duration) {
+	td.mu.Lock()
+	defer td.mu.Unlock()
+
+	ok := true
+	now := time.Now()
+	exceed := time.Duration(0)
+
+	if pt, found := td.records[which]; found {
+		exceed = now.Sub(pt) - td.maxDuration
+		if exceed > 0 {
+			ok = false
+		}
+	}
+	td.records[which] = now
+	return ok, exceed
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/contention/doc.go b/vendor/go.etcd.io/etcd/pkg/contention/doc.go
new file mode 100644
index 0000000..daf4522
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/contention/doc.go
@@ -0,0 +1,16 @@
+// Copyright 2016 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 contention provides facilities for detecting system contention.
+package contention
diff --git a/vendor/go.etcd.io/etcd/pkg/cpuutil/doc.go b/vendor/go.etcd.io/etcd/pkg/cpuutil/doc.go
new file mode 100644
index 0000000..0323b2d
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/cpuutil/doc.go
@@ -0,0 +1,16 @@
+// 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 cpuutil provides facilities for detecting cpu-specific features.
+package cpuutil
diff --git a/vendor/go.etcd.io/etcd/pkg/cpuutil/endian.go b/vendor/go.etcd.io/etcd/pkg/cpuutil/endian.go
new file mode 100644
index 0000000..06c06cd
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/cpuutil/endian.go
@@ -0,0 +1,36 @@
+// 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 cpuutil
+
+import (
+	"encoding/binary"
+	"unsafe"
+)
+
+const intWidth int = int(unsafe.Sizeof(0))
+
+var byteOrder binary.ByteOrder
+
+// ByteOrder returns the byte order for the CPU's native endianness.
+func ByteOrder() binary.ByteOrder { return byteOrder }
+
+func init() {
+	i := int(0x1)
+	if v := (*[intWidth]byte)(unsafe.Pointer(&i)); v[0] == 0 {
+		byteOrder = binary.BigEndian
+	} else {
+		byteOrder = binary.LittleEndian
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/crc/crc.go b/vendor/go.etcd.io/etcd/pkg/crc/crc.go
new file mode 100644
index 0000000..4b998a4
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/crc/crc.go
@@ -0,0 +1,43 @@
+// Copyright 2009 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// Package crc provides utility function for cyclic redundancy check
+// algorithms.
+package crc
+
+import (
+	"hash"
+	"hash/crc32"
+)
+
+// The size of a CRC-32 checksum in bytes.
+const Size = 4
+
+type digest struct {
+	crc uint32
+	tab *crc32.Table
+}
+
+// New creates a new hash.Hash32 computing the CRC-32 checksum
+// using the polynomial represented by the Table.
+// Modified by xiangli to take a prevcrc.
+func New(prev uint32, tab *crc32.Table) hash.Hash32 { return &digest{prev, tab} }
+
+func (d *digest) Size() int { return Size }
+
+func (d *digest) BlockSize() int { return 1 }
+
+func (d *digest) Reset() { d.crc = 0 }
+
+func (d *digest) Write(p []byte) (n int, err error) {
+	d.crc = crc32.Update(d.crc, d.tab, p)
+	return len(p), nil
+}
+
+func (d *digest) Sum32() uint32 { return d.crc }
+
+func (d *digest) Sum(in []byte) []byte {
+	s := d.Sum32()
+	return append(in, byte(s>>24), byte(s>>16), byte(s>>8), byte(s))
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/debugutil/doc.go b/vendor/go.etcd.io/etcd/pkg/debugutil/doc.go
new file mode 100644
index 0000000..74499eb
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/debugutil/doc.go
@@ -0,0 +1,16 @@
+// 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 debugutil includes utility functions for debugging.
+package debugutil
diff --git a/vendor/go.etcd.io/etcd/pkg/debugutil/pprof.go b/vendor/go.etcd.io/etcd/pkg/debugutil/pprof.go
new file mode 100644
index 0000000..8d5544a
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/debugutil/pprof.go
@@ -0,0 +1,47 @@
+// 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 debugutil
+
+import (
+	"net/http"
+	"net/http/pprof"
+	"runtime"
+)
+
+const HTTPPrefixPProf = "/debug/pprof"
+
+// PProfHandlers returns a map of pprof handlers keyed by the HTTP path.
+func PProfHandlers() map[string]http.Handler {
+	// set only when there's no existing setting
+	if runtime.SetMutexProfileFraction(-1) == 0 {
+		// 1 out of 5 mutex events are reported, on average
+		runtime.SetMutexProfileFraction(5)
+	}
+
+	m := make(map[string]http.Handler)
+
+	m[HTTPPrefixPProf+"/"] = http.HandlerFunc(pprof.Index)
+	m[HTTPPrefixPProf+"/profile"] = http.HandlerFunc(pprof.Profile)
+	m[HTTPPrefixPProf+"/symbol"] = http.HandlerFunc(pprof.Symbol)
+	m[HTTPPrefixPProf+"/cmdline"] = http.HandlerFunc(pprof.Cmdline)
+	m[HTTPPrefixPProf+"/trace "] = http.HandlerFunc(pprof.Trace)
+	m[HTTPPrefixPProf+"/heap"] = pprof.Handler("heap")
+	m[HTTPPrefixPProf+"/goroutine"] = pprof.Handler("goroutine")
+	m[HTTPPrefixPProf+"/threadcreate"] = pprof.Handler("threadcreate")
+	m[HTTPPrefixPProf+"/block"] = pprof.Handler("block")
+	m[HTTPPrefixPProf+"/mutex"] = pprof.Handler("mutex")
+
+	return m
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/dir_unix.go b/vendor/go.etcd.io/etcd/pkg/fileutil/dir_unix.go
new file mode 100644
index 0000000..58a77df
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/dir_unix.go
@@ -0,0 +1,22 @@
+// Copyright 2016 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.
+
+// +build !windows
+
+package fileutil
+
+import "os"
+
+// OpenDir opens a directory for syncing.
+func OpenDir(path string) (*os.File, error) { return os.Open(path) }
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/dir_windows.go b/vendor/go.etcd.io/etcd/pkg/fileutil/dir_windows.go
new file mode 100644
index 0000000..c123395
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/dir_windows.go
@@ -0,0 +1,46 @@
+// Copyright 2016 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.
+
+// +build windows
+
+package fileutil
+
+import (
+	"os"
+	"syscall"
+)
+
+// OpenDir opens a directory in windows with write access for syncing.
+func OpenDir(path string) (*os.File, error) {
+	fd, err := openDir(path)
+	if err != nil {
+		return nil, err
+	}
+	return os.NewFile(uintptr(fd), path), nil
+}
+
+func openDir(path string) (fd syscall.Handle, err error) {
+	if len(path) == 0 {
+		return syscall.InvalidHandle, syscall.ERROR_FILE_NOT_FOUND
+	}
+	pathp, err := syscall.UTF16PtrFromString(path)
+	if err != nil {
+		return syscall.InvalidHandle, err
+	}
+	access := uint32(syscall.GENERIC_READ | syscall.GENERIC_WRITE)
+	sharemode := uint32(syscall.FILE_SHARE_READ | syscall.FILE_SHARE_WRITE)
+	createmode := uint32(syscall.OPEN_EXISTING)
+	fl := uint32(syscall.FILE_FLAG_BACKUP_SEMANTICS)
+	return syscall.CreateFile(pathp, access, sharemode, nil, createmode, fl, 0)
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/doc.go b/vendor/go.etcd.io/etcd/pkg/fileutil/doc.go
new file mode 100644
index 0000000..69dde5a
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/doc.go
@@ -0,0 +1,16 @@
+// Copyright 2018 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 fileutil implements utility functions related to files and paths.
+package fileutil
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/fileutil.go b/vendor/go.etcd.io/etcd/pkg/fileutil/fileutil.go
new file mode 100644
index 0000000..5d9fb53
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/fileutil.go
@@ -0,0 +1,104 @@
+// Copyright 2015 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 fileutil
+
+import (
+	"fmt"
+	"io"
+	"io/ioutil"
+	"os"
+	"path/filepath"
+
+	"github.com/coreos/pkg/capnslog"
+)
+
+const (
+	// PrivateFileMode grants owner to read/write a file.
+	PrivateFileMode = 0600
+	// PrivateDirMode grants owner to make/remove files inside the directory.
+	PrivateDirMode = 0700
+)
+
+var plog = capnslog.NewPackageLogger("go.etcd.io/etcd", "pkg/fileutil")
+
+// IsDirWriteable checks if dir is writable by writing and removing a file
+// to dir. It returns nil if dir is writable.
+func IsDirWriteable(dir string) error {
+	f := filepath.Join(dir, ".touch")
+	if err := ioutil.WriteFile(f, []byte(""), PrivateFileMode); err != nil {
+		return err
+	}
+	return os.Remove(f)
+}
+
+// TouchDirAll is similar to os.MkdirAll. It creates directories with 0700 permission if any directory
+// does not exists. TouchDirAll also ensures the given directory is writable.
+func TouchDirAll(dir string) error {
+	// If path is already a directory, MkdirAll does nothing
+	// and returns nil.
+	err := os.MkdirAll(dir, PrivateDirMode)
+	if err != nil {
+		// if mkdirAll("a/text") and "text" is not
+		// a directory, this will return syscall.ENOTDIR
+		return err
+	}
+	return IsDirWriteable(dir)
+}
+
+// CreateDirAll is similar to TouchDirAll but returns error
+// if the deepest directory was not empty.
+func CreateDirAll(dir string) error {
+	err := TouchDirAll(dir)
+	if err == nil {
+		var ns []string
+		ns, err = ReadDir(dir)
+		if err != nil {
+			return err
+		}
+		if len(ns) != 0 {
+			err = fmt.Errorf("expected %q to be empty, got %q", dir, ns)
+		}
+	}
+	return err
+}
+
+// Exist returns true if a file or directory exists.
+func Exist(name string) bool {
+	_, err := os.Stat(name)
+	return err == nil
+}
+
+// ZeroToEnd zeros a file starting from SEEK_CUR to its SEEK_END. May temporarily
+// shorten the length of the file.
+func ZeroToEnd(f *os.File) error {
+	// TODO: support FALLOC_FL_ZERO_RANGE
+	off, err := f.Seek(0, io.SeekCurrent)
+	if err != nil {
+		return err
+	}
+	lenf, lerr := f.Seek(0, io.SeekEnd)
+	if lerr != nil {
+		return lerr
+	}
+	if err = f.Truncate(off); err != nil {
+		return err
+	}
+	// make sure blocks remain allocated
+	if err = Preallocate(f, lenf, true); err != nil {
+		return err
+	}
+	_, err = f.Seek(off, io.SeekStart)
+	return err
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/lock.go b/vendor/go.etcd.io/etcd/pkg/fileutil/lock.go
new file mode 100644
index 0000000..338627f
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/lock.go
@@ -0,0 +1,26 @@
+// Copyright 2016 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 fileutil
+
+import (
+	"errors"
+	"os"
+)
+
+var (
+	ErrLocked = errors.New("fileutil: file already locked")
+)
+
+type LockedFile struct{ *os.File }
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/lock_flock.go b/vendor/go.etcd.io/etcd/pkg/fileutil/lock_flock.go
new file mode 100644
index 0000000..542550b
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/lock_flock.go
@@ -0,0 +1,49 @@
+// Copyright 2016 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.
+
+// +build !windows,!plan9,!solaris
+
+package fileutil
+
+import (
+	"os"
+	"syscall"
+)
+
+func flockTryLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) {
+	f, err := os.OpenFile(path, flag, perm)
+	if err != nil {
+		return nil, err
+	}
+	if err = syscall.Flock(int(f.Fd()), syscall.LOCK_EX|syscall.LOCK_NB); err != nil {
+		f.Close()
+		if err == syscall.EWOULDBLOCK {
+			err = ErrLocked
+		}
+		return nil, err
+	}
+	return &LockedFile{f}, nil
+}
+
+func flockLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) {
+	f, err := os.OpenFile(path, flag, perm)
+	if err != nil {
+		return nil, err
+	}
+	if err = syscall.Flock(int(f.Fd()), syscall.LOCK_EX); err != nil {
+		f.Close()
+		return nil, err
+	}
+	return &LockedFile{f}, err
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/lock_linux.go b/vendor/go.etcd.io/etcd/pkg/fileutil/lock_linux.go
new file mode 100644
index 0000000..b0abc98
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/lock_linux.go
@@ -0,0 +1,97 @@
+// Copyright 2016 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.
+
+// +build linux
+
+package fileutil
+
+import (
+	"fmt"
+	"io"
+	"os"
+	"syscall"
+)
+
+// This used to call syscall.Flock() but that call fails with EBADF on NFS.
+// An alternative is lockf() which works on NFS but that call lets a process lock
+// the same file twice. Instead, use Linux's non-standard open file descriptor
+// locks which will block if the process already holds the file lock.
+//
+// constants from /usr/include/bits/fcntl-linux.h
+const (
+	F_OFD_GETLK  = 37
+	F_OFD_SETLK  = 37
+	F_OFD_SETLKW = 38
+)
+
+var (
+	wrlck = syscall.Flock_t{
+		Type:   syscall.F_WRLCK,
+		Whence: int16(io.SeekStart),
+		Start:  0,
+		Len:    0,
+	}
+
+	linuxTryLockFile = flockTryLockFile
+	linuxLockFile    = flockLockFile
+)
+
+func init() {
+	// use open file descriptor locks if the system supports it
+	getlk := syscall.Flock_t{Type: syscall.F_RDLCK}
+	if err := syscall.FcntlFlock(0, F_OFD_GETLK, &getlk); err == nil {
+		linuxTryLockFile = ofdTryLockFile
+		linuxLockFile = ofdLockFile
+	}
+}
+
+func TryLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) {
+	return linuxTryLockFile(path, flag, perm)
+}
+
+func ofdTryLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) {
+	f, err := os.OpenFile(path, flag, perm)
+	if err != nil {
+		return nil, fmt.Errorf("ofdTryLockFile failed to open %q (%v)", path, err)
+	}
+
+	flock := wrlck
+	if err = syscall.FcntlFlock(f.Fd(), F_OFD_SETLK, &flock); err != nil {
+		f.Close()
+		if err == syscall.EWOULDBLOCK {
+			err = ErrLocked
+		}
+		return nil, err
+	}
+	return &LockedFile{f}, nil
+}
+
+func LockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) {
+	return linuxLockFile(path, flag, perm)
+}
+
+func ofdLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) {
+	f, err := os.OpenFile(path, flag, perm)
+	if err != nil {
+		return nil, fmt.Errorf("ofdLockFile failed to open %q (%v)", path, err)
+	}
+
+	flock := wrlck
+	err = syscall.FcntlFlock(f.Fd(), F_OFD_SETLKW, &flock)
+	if err != nil {
+		f.Close()
+		return nil, err
+	}
+	return &LockedFile{f}, nil
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/lock_plan9.go b/vendor/go.etcd.io/etcd/pkg/fileutil/lock_plan9.go
new file mode 100644
index 0000000..fee6a7c
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/lock_plan9.go
@@ -0,0 +1,45 @@
+// Copyright 2015 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 fileutil
+
+import (
+	"os"
+	"syscall"
+	"time"
+)
+
+func TryLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) {
+	if err := os.Chmod(path, syscall.DMEXCL|PrivateFileMode); err != nil {
+		return nil, err
+	}
+	f, err := os.Open(path, flag, perm)
+	if err != nil {
+		return nil, ErrLocked
+	}
+	return &LockedFile{f}, nil
+}
+
+func LockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) {
+	if err := os.Chmod(path, syscall.DMEXCL|PrivateFileMode); err != nil {
+		return nil, err
+	}
+	for {
+		f, err := os.OpenFile(path, flag, perm)
+		if err == nil {
+			return &LockedFile{f}, nil
+		}
+		time.Sleep(10 * time.Millisecond)
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/lock_solaris.go b/vendor/go.etcd.io/etcd/pkg/fileutil/lock_solaris.go
new file mode 100644
index 0000000..352ca55
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/lock_solaris.go
@@ -0,0 +1,62 @@
+// Copyright 2015 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.
+
+// +build solaris
+
+package fileutil
+
+import (
+	"os"
+	"syscall"
+)
+
+func TryLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) {
+	var lock syscall.Flock_t
+	lock.Start = 0
+	lock.Len = 0
+	lock.Pid = 0
+	lock.Type = syscall.F_WRLCK
+	lock.Whence = 0
+	lock.Pid = 0
+	f, err := os.OpenFile(path, flag, perm)
+	if err != nil {
+		return nil, err
+	}
+	if err := syscall.FcntlFlock(f.Fd(), syscall.F_SETLK, &lock); err != nil {
+		f.Close()
+		if err == syscall.EAGAIN {
+			err = ErrLocked
+		}
+		return nil, err
+	}
+	return &LockedFile{f}, nil
+}
+
+func LockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) {
+	var lock syscall.Flock_t
+	lock.Start = 0
+	lock.Len = 0
+	lock.Pid = 0
+	lock.Type = syscall.F_WRLCK
+	lock.Whence = 0
+	f, err := os.OpenFile(path, flag, perm)
+	if err != nil {
+		return nil, err
+	}
+	if err = syscall.FcntlFlock(f.Fd(), syscall.F_SETLKW, &lock); err != nil {
+		f.Close()
+		return nil, err
+	}
+	return &LockedFile{f}, nil
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/lock_unix.go b/vendor/go.etcd.io/etcd/pkg/fileutil/lock_unix.go
new file mode 100644
index 0000000..ed01164
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/lock_unix.go
@@ -0,0 +1,29 @@
+// Copyright 2015 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.
+
+// +build !windows,!plan9,!solaris,!linux
+
+package fileutil
+
+import (
+	"os"
+)
+
+func TryLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) {
+	return flockTryLockFile(path, flag, perm)
+}
+
+func LockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) {
+	return flockLockFile(path, flag, perm)
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/lock_windows.go b/vendor/go.etcd.io/etcd/pkg/fileutil/lock_windows.go
new file mode 100644
index 0000000..b181723
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/lock_windows.go
@@ -0,0 +1,125 @@
+// Copyright 2015 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.
+
+// +build windows
+
+package fileutil
+
+import (
+	"errors"
+	"fmt"
+	"os"
+	"syscall"
+	"unsafe"
+)
+
+var (
+	modkernel32    = syscall.NewLazyDLL("kernel32.dll")
+	procLockFileEx = modkernel32.NewProc("LockFileEx")
+
+	errLocked = errors.New("The process cannot access the file because another process has locked a portion of the file.")
+)
+
+const (
+	// https://msdn.microsoft.com/en-us/library/windows/desktop/aa365203(v=vs.85).aspx
+	LOCKFILE_EXCLUSIVE_LOCK   = 2
+	LOCKFILE_FAIL_IMMEDIATELY = 1
+
+	// see https://msdn.microsoft.com/en-us/library/windows/desktop/ms681382(v=vs.85).aspx
+	errLockViolation syscall.Errno = 0x21
+)
+
+func TryLockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) {
+	f, err := open(path, flag, perm)
+	if err != nil {
+		return nil, err
+	}
+	if err := lockFile(syscall.Handle(f.Fd()), LOCKFILE_FAIL_IMMEDIATELY); err != nil {
+		f.Close()
+		return nil, err
+	}
+	return &LockedFile{f}, nil
+}
+
+func LockFile(path string, flag int, perm os.FileMode) (*LockedFile, error) {
+	f, err := open(path, flag, perm)
+	if err != nil {
+		return nil, err
+	}
+	if err := lockFile(syscall.Handle(f.Fd()), 0); err != nil {
+		f.Close()
+		return nil, err
+	}
+	return &LockedFile{f}, nil
+}
+
+func open(path string, flag int, perm os.FileMode) (*os.File, error) {
+	if path == "" {
+		return nil, fmt.Errorf("cannot open empty filename")
+	}
+	var access uint32
+	switch flag {
+	case syscall.O_RDONLY:
+		access = syscall.GENERIC_READ
+	case syscall.O_WRONLY:
+		access = syscall.GENERIC_WRITE
+	case syscall.O_RDWR:
+		access = syscall.GENERIC_READ | syscall.GENERIC_WRITE
+	case syscall.O_WRONLY | syscall.O_CREAT:
+		access = syscall.GENERIC_ALL
+	default:
+		panic(fmt.Errorf("flag %v is not supported", flag))
+	}
+	fd, err := syscall.CreateFile(&(syscall.StringToUTF16(path)[0]),
+		access,
+		syscall.FILE_SHARE_READ|syscall.FILE_SHARE_WRITE|syscall.FILE_SHARE_DELETE,
+		nil,
+		syscall.OPEN_ALWAYS,
+		syscall.FILE_ATTRIBUTE_NORMAL,
+		0)
+	if err != nil {
+		return nil, err
+	}
+	return os.NewFile(uintptr(fd), path), nil
+}
+
+func lockFile(fd syscall.Handle, flags uint32) error {
+	var flag uint32 = LOCKFILE_EXCLUSIVE_LOCK
+	flag |= flags
+	if fd == syscall.InvalidHandle {
+		return nil
+	}
+	err := lockFileEx(fd, flag, 1, 0, &syscall.Overlapped{})
+	if err == nil {
+		return nil
+	} else if err.Error() == errLocked.Error() {
+		return ErrLocked
+	} else if err != errLockViolation {
+		return err
+	}
+	return nil
+}
+
+func lockFileEx(h syscall.Handle, flags, locklow, lockhigh uint32, ol *syscall.Overlapped) (err error) {
+	var reserved uint32 = 0
+	r1, _, e1 := syscall.Syscall6(procLockFileEx.Addr(), 6, uintptr(h), uintptr(flags), uintptr(reserved), uintptr(locklow), uintptr(lockhigh), uintptr(unsafe.Pointer(ol)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = error(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return err
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/preallocate.go b/vendor/go.etcd.io/etcd/pkg/fileutil/preallocate.go
new file mode 100644
index 0000000..c747b7c
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/preallocate.go
@@ -0,0 +1,54 @@
+// Copyright 2015 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 fileutil
+
+import (
+	"io"
+	"os"
+)
+
+// Preallocate tries to allocate the space for given
+// file. This operation is only supported on linux by a
+// few filesystems (btrfs, ext4, etc.).
+// If the operation is unsupported, no error will be returned.
+// Otherwise, the error encountered will be returned.
+func Preallocate(f *os.File, sizeInBytes int64, extendFile bool) error {
+	if sizeInBytes == 0 {
+		// fallocate will return EINVAL if length is 0; skip
+		return nil
+	}
+	if extendFile {
+		return preallocExtend(f, sizeInBytes)
+	}
+	return preallocFixed(f, sizeInBytes)
+}
+
+func preallocExtendTrunc(f *os.File, sizeInBytes int64) error {
+	curOff, err := f.Seek(0, io.SeekCurrent)
+	if err != nil {
+		return err
+	}
+	size, err := f.Seek(sizeInBytes, io.SeekEnd)
+	if err != nil {
+		return err
+	}
+	if _, err = f.Seek(curOff, io.SeekStart); err != nil {
+		return err
+	}
+	if sizeInBytes > size {
+		return nil
+	}
+	return f.Truncate(sizeInBytes)
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/preallocate_darwin.go b/vendor/go.etcd.io/etcd/pkg/fileutil/preallocate_darwin.go
new file mode 100644
index 0000000..5a6dccf
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/preallocate_darwin.go
@@ -0,0 +1,65 @@
+// Copyright 2016 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.
+
+// +build darwin
+
+package fileutil
+
+import (
+	"os"
+	"syscall"
+	"unsafe"
+)
+
+func preallocExtend(f *os.File, sizeInBytes int64) error {
+	if err := preallocFixed(f, sizeInBytes); err != nil {
+		return err
+	}
+	return preallocExtendTrunc(f, sizeInBytes)
+}
+
+func preallocFixed(f *os.File, sizeInBytes int64) error {
+	// allocate all requested space or no space at all
+	// TODO: allocate contiguous space on disk with F_ALLOCATECONTIG flag
+	fstore := &syscall.Fstore_t{
+		Flags:   syscall.F_ALLOCATEALL,
+		Posmode: syscall.F_PEOFPOSMODE,
+		Length:  sizeInBytes}
+	p := unsafe.Pointer(fstore)
+	_, _, errno := syscall.Syscall(syscall.SYS_FCNTL, f.Fd(), uintptr(syscall.F_PREALLOCATE), uintptr(p))
+	if errno == 0 || errno == syscall.ENOTSUP {
+		return nil
+	}
+
+	// wrong argument to fallocate syscall
+	if errno == syscall.EINVAL {
+		// filesystem "st_blocks" are allocated in the units of
+		// "Allocation Block Size" (run "diskutil info /" command)
+		var stat syscall.Stat_t
+		syscall.Fstat(int(f.Fd()), &stat)
+
+		// syscall.Statfs_t.Bsize is "optimal transfer block size"
+		// and contains matching 4096 value when latest OS X kernel
+		// supports 4,096 KB filesystem block size
+		var statfs syscall.Statfs_t
+		syscall.Fstatfs(int(f.Fd()), &statfs)
+		blockSize := int64(statfs.Bsize)
+
+		if stat.Blocks*blockSize >= sizeInBytes {
+			// enough blocks are already allocated
+			return nil
+		}
+	}
+	return errno
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/preallocate_unix.go b/vendor/go.etcd.io/etcd/pkg/fileutil/preallocate_unix.go
new file mode 100644
index 0000000..50bd84f
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/preallocate_unix.go
@@ -0,0 +1,49 @@
+// Copyright 2016 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.
+
+// +build linux
+
+package fileutil
+
+import (
+	"os"
+	"syscall"
+)
+
+func preallocExtend(f *os.File, sizeInBytes int64) error {
+	// use mode = 0 to change size
+	err := syscall.Fallocate(int(f.Fd()), 0, 0, sizeInBytes)
+	if err != nil {
+		errno, ok := err.(syscall.Errno)
+		// not supported; fallback
+		// fallocate EINTRs frequently in some environments; fallback
+		if ok && (errno == syscall.ENOTSUP || errno == syscall.EINTR) {
+			return preallocExtendTrunc(f, sizeInBytes)
+		}
+	}
+	return err
+}
+
+func preallocFixed(f *os.File, sizeInBytes int64) error {
+	// use mode = 1 to keep size; see FALLOC_FL_KEEP_SIZE
+	err := syscall.Fallocate(int(f.Fd()), 1, 0, sizeInBytes)
+	if err != nil {
+		errno, ok := err.(syscall.Errno)
+		// treat not supported as nil error
+		if ok && errno == syscall.ENOTSUP {
+			return nil
+		}
+	}
+	return err
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/preallocate_unsupported.go b/vendor/go.etcd.io/etcd/pkg/fileutil/preallocate_unsupported.go
new file mode 100644
index 0000000..162fbc5
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/preallocate_unsupported.go
@@ -0,0 +1,25 @@
+// Copyright 2015 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.
+
+// +build !linux,!darwin
+
+package fileutil
+
+import "os"
+
+func preallocExtend(f *os.File, sizeInBytes int64) error {
+	return preallocExtendTrunc(f, sizeInBytes)
+}
+
+func preallocFixed(f *os.File, sizeInBytes int64) error { return nil }
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/purge.go b/vendor/go.etcd.io/etcd/pkg/fileutil/purge.go
new file mode 100644
index 0000000..fda96c3
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/purge.go
@@ -0,0 +1,88 @@
+// Copyright 2015 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 fileutil
+
+import (
+	"os"
+	"path/filepath"
+	"sort"
+	"strings"
+	"time"
+
+	"go.uber.org/zap"
+)
+
+func PurgeFile(lg *zap.Logger, dirname string, suffix string, max uint, interval time.Duration, stop <-chan struct{}) <-chan error {
+	return purgeFile(lg, dirname, suffix, max, interval, stop, nil)
+}
+
+// purgeFile is the internal implementation for PurgeFile which can post purged files to purgec if non-nil.
+func purgeFile(lg *zap.Logger, dirname string, suffix string, max uint, interval time.Duration, stop <-chan struct{}, purgec chan<- string) <-chan error {
+	errC := make(chan error, 1)
+	go func() {
+		for {
+			fnames, err := ReadDir(dirname)
+			if err != nil {
+				errC <- err
+				return
+			}
+			newfnames := make([]string, 0)
+			for _, fname := range fnames {
+				if strings.HasSuffix(fname, suffix) {
+					newfnames = append(newfnames, fname)
+				}
+			}
+			sort.Strings(newfnames)
+			fnames = newfnames
+			for len(newfnames) > int(max) {
+				f := filepath.Join(dirname, newfnames[0])
+				l, err := TryLockFile(f, os.O_WRONLY, PrivateFileMode)
+				if err != nil {
+					break
+				}
+				if err = os.Remove(f); err != nil {
+					errC <- err
+					return
+				}
+				if err = l.Close(); err != nil {
+					if lg != nil {
+						lg.Warn("failed to unlock/close", zap.String("path", l.Name()), zap.Error(err))
+					} else {
+						plog.Errorf("error unlocking %s when purging file (%v)", l.Name(), err)
+					}
+					errC <- err
+					return
+				}
+				if lg != nil {
+					lg.Info("purged", zap.String("path", f))
+				} else {
+					plog.Infof("purged file %s successfully", f)
+				}
+				newfnames = newfnames[1:]
+			}
+			if purgec != nil {
+				for i := 0; i < len(fnames)-len(newfnames); i++ {
+					purgec <- fnames[i]
+				}
+			}
+			select {
+			case <-time.After(interval):
+			case <-stop:
+				return
+			}
+		}
+	}()
+	return errC
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/read_dir.go b/vendor/go.etcd.io/etcd/pkg/fileutil/read_dir.go
new file mode 100644
index 0000000..2eeaa89
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/read_dir.go
@@ -0,0 +1,70 @@
+// Copyright 2018 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 fileutil
+
+import (
+	"os"
+	"path/filepath"
+	"sort"
+)
+
+// ReadDirOp represents an read-directory operation.
+type ReadDirOp struct {
+	ext string
+}
+
+// ReadDirOption configures archiver operations.
+type ReadDirOption func(*ReadDirOp)
+
+// WithExt filters file names by their extensions.
+// (e.g. WithExt(".wal") to list only WAL files)
+func WithExt(ext string) ReadDirOption {
+	return func(op *ReadDirOp) { op.ext = ext }
+}
+
+func (op *ReadDirOp) applyOpts(opts []ReadDirOption) {
+	for _, opt := range opts {
+		opt(op)
+	}
+}
+
+// ReadDir returns the filenames in the given directory in sorted order.
+func ReadDir(d string, opts ...ReadDirOption) ([]string, error) {
+	op := &ReadDirOp{}
+	op.applyOpts(opts)
+
+	dir, err := os.Open(d)
+	if err != nil {
+		return nil, err
+	}
+	defer dir.Close()
+
+	names, err := dir.Readdirnames(-1)
+	if err != nil {
+		return nil, err
+	}
+	sort.Strings(names)
+
+	if op.ext != "" {
+		tss := make([]string, 0)
+		for _, v := range names {
+			if filepath.Ext(v) == op.ext {
+				tss = append(tss, v)
+			}
+		}
+		names = tss
+	}
+	return names, nil
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/sync.go b/vendor/go.etcd.io/etcd/pkg/fileutil/sync.go
new file mode 100644
index 0000000..54dd41f
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/sync.go
@@ -0,0 +1,29 @@
+// Copyright 2016 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.
+
+// +build !linux,!darwin
+
+package fileutil
+
+import "os"
+
+// Fsync is a wrapper around file.Sync(). Special handling is needed on darwin platform.
+func Fsync(f *os.File) error {
+	return f.Sync()
+}
+
+// Fdatasync is a wrapper around file.Sync(). Special handling is needed on linux platform.
+func Fdatasync(f *os.File) error {
+	return f.Sync()
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/sync_darwin.go b/vendor/go.etcd.io/etcd/pkg/fileutil/sync_darwin.go
new file mode 100644
index 0000000..c2f39bf
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/sync_darwin.go
@@ -0,0 +1,40 @@
+// Copyright 2016 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.
+
+// +build darwin
+
+package fileutil
+
+import (
+	"os"
+	"syscall"
+)
+
+// Fsync on HFS/OSX flushes the data on to the physical drive but the drive
+// may not write it to the persistent media for quite sometime and it may be
+// written in out-of-order sequence. Using F_FULLFSYNC ensures that the
+// physical drive's buffer will also get flushed to the media.
+func Fsync(f *os.File) error {
+	_, _, errno := syscall.Syscall(syscall.SYS_FCNTL, f.Fd(), uintptr(syscall.F_FULLFSYNC), uintptr(0))
+	if errno == 0 {
+		return nil
+	}
+	return errno
+}
+
+// Fdatasync on darwin platform invokes fcntl(F_FULLFSYNC) for actual persistence
+// on physical drive media.
+func Fdatasync(f *os.File) error {
+	return Fsync(f)
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/fileutil/sync_linux.go b/vendor/go.etcd.io/etcd/pkg/fileutil/sync_linux.go
new file mode 100644
index 0000000..1bbced9
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/fileutil/sync_linux.go
@@ -0,0 +1,34 @@
+// Copyright 2016 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.
+
+// +build linux
+
+package fileutil
+
+import (
+	"os"
+	"syscall"
+)
+
+// Fsync is a wrapper around file.Sync(). Special handling is needed on darwin platform.
+func Fsync(f *os.File) error {
+	return f.Sync()
+}
+
+// Fdatasync is similar to fsync(), but does not flush modified metadata
+// unless that metadata is needed in order to allow a subsequent data retrieval
+// to be correctly handled.
+func Fdatasync(f *os.File) error {
+	return syscall.Fdatasync(int(f.Fd()))
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/flags/flag.go b/vendor/go.etcd.io/etcd/pkg/flags/flag.go
new file mode 100644
index 0000000..215902c
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/flags/flag.go
@@ -0,0 +1,121 @@
+// Copyright 2015 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 flags implements command-line flag parsing.
+package flags
+
+import (
+	"flag"
+	"fmt"
+	"os"
+	"strings"
+
+	"github.com/coreos/pkg/capnslog"
+	"github.com/spf13/pflag"
+)
+
+var plog = capnslog.NewPackageLogger("go.etcd.io/etcd", "pkg/flags")
+
+// SetFlagsFromEnv parses all registered flags in the given flagset,
+// and if they are not already set it attempts to set their values from
+// environment variables. Environment variables take the name of the flag but
+// are UPPERCASE, have the given prefix  and any dashes are replaced by
+// underscores - for example: some-flag => ETCD_SOME_FLAG
+func SetFlagsFromEnv(prefix string, fs *flag.FlagSet) error {
+	var err error
+	alreadySet := make(map[string]bool)
+	fs.Visit(func(f *flag.Flag) {
+		alreadySet[FlagToEnv(prefix, f.Name)] = true
+	})
+	usedEnvKey := make(map[string]bool)
+	fs.VisitAll(func(f *flag.Flag) {
+		if serr := setFlagFromEnv(fs, prefix, f.Name, usedEnvKey, alreadySet, true); serr != nil {
+			err = serr
+		}
+	})
+	verifyEnv(prefix, usedEnvKey, alreadySet)
+	return err
+}
+
+// SetPflagsFromEnv is similar to SetFlagsFromEnv. However, the accepted flagset type is pflag.FlagSet
+// and it does not do any logging.
+func SetPflagsFromEnv(prefix string, fs *pflag.FlagSet) error {
+	var err error
+	alreadySet := make(map[string]bool)
+	usedEnvKey := make(map[string]bool)
+	fs.VisitAll(func(f *pflag.Flag) {
+		if f.Changed {
+			alreadySet[FlagToEnv(prefix, f.Name)] = true
+		}
+		if serr := setFlagFromEnv(fs, prefix, f.Name, usedEnvKey, alreadySet, false); serr != nil {
+			err = serr
+		}
+	})
+	verifyEnv(prefix, usedEnvKey, alreadySet)
+	return err
+}
+
+// FlagToEnv converts flag string to upper-case environment variable key string.
+func FlagToEnv(prefix, name string) string {
+	return prefix + "_" + strings.ToUpper(strings.Replace(name, "-", "_", -1))
+}
+
+func verifyEnv(prefix string, usedEnvKey, alreadySet map[string]bool) {
+	for _, env := range os.Environ() {
+		kv := strings.SplitN(env, "=", 2)
+		if len(kv) != 2 {
+			plog.Warningf("found invalid env %s", env)
+		}
+		if usedEnvKey[kv[0]] {
+			continue
+		}
+		if alreadySet[kv[0]] {
+			plog.Fatalf("conflicting environment variable %q is shadowed by corresponding command-line flag (either unset environment variable or disable flag)", kv[0])
+		}
+		if strings.HasPrefix(env, prefix+"_") {
+			plog.Warningf("unrecognized environment variable %s", env)
+		}
+	}
+}
+
+type flagSetter interface {
+	Set(fk string, fv string) error
+}
+
+func setFlagFromEnv(fs flagSetter, prefix, fname string, usedEnvKey, alreadySet map[string]bool, log bool) error {
+	key := FlagToEnv(prefix, fname)
+	if !alreadySet[key] {
+		val := os.Getenv(key)
+		if val != "" {
+			usedEnvKey[key] = true
+			if serr := fs.Set(fname, val); serr != nil {
+				return fmt.Errorf("invalid value %q for %s: %v", val, key, serr)
+			}
+			if log {
+				plog.Infof("recognized and used environment variable %s=%s", key, val)
+			}
+		}
+	}
+	return nil
+}
+
+func IsSet(fs *flag.FlagSet, name string) bool {
+	set := false
+	fs.Visit(func(f *flag.Flag) {
+		if f.Name == name {
+			set = true
+		}
+	})
+	return set
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/flags/ignored.go b/vendor/go.etcd.io/etcd/pkg/flags/ignored.go
new file mode 100644
index 0000000..9953049
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/flags/ignored.go
@@ -0,0 +1,36 @@
+// Copyright 2018 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 flags
+
+// IgnoredFlag encapsulates a flag that may have been previously valid but is
+// now ignored. If an IgnoredFlag is set, a warning is printed and
+// operation continues.
+type IgnoredFlag struct {
+	Name string
+}
+
+// IsBoolFlag is defined to allow the flag to be defined without an argument
+func (f *IgnoredFlag) IsBoolFlag() bool {
+	return true
+}
+
+func (f *IgnoredFlag) Set(s string) error {
+	plog.Warningf(`flag "-%s" is no longer supported - ignoring.`, f.Name)
+	return nil
+}
+
+func (f *IgnoredFlag) String() string {
+	return ""
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/flags/selective_string.go b/vendor/go.etcd.io/etcd/pkg/flags/selective_string.go
new file mode 100644
index 0000000..4b90fbf
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/flags/selective_string.go
@@ -0,0 +1,114 @@
+// Copyright 2018 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 flags
+
+import (
+	"errors"
+	"fmt"
+	"sort"
+	"strings"
+)
+
+// SelectiveStringValue implements the flag.Value interface.
+type SelectiveStringValue struct {
+	v      string
+	valids map[string]struct{}
+}
+
+// Set verifies the argument to be a valid member of the allowed values
+// before setting the underlying flag value.
+func (ss *SelectiveStringValue) Set(s string) error {
+	if _, ok := ss.valids[s]; ok {
+		ss.v = s
+		return nil
+	}
+	return errors.New("invalid value")
+}
+
+// String returns the set value (if any) of the SelectiveStringValue
+func (ss *SelectiveStringValue) String() string {
+	return ss.v
+}
+
+// Valids returns the list of valid strings.
+func (ss *SelectiveStringValue) Valids() []string {
+	s := make([]string, 0, len(ss.valids))
+	for k := range ss.valids {
+		s = append(s, k)
+	}
+	sort.Strings(s)
+	return s
+}
+
+// NewSelectiveStringValue creates a new string flag
+// for which any one of the given strings is a valid value,
+// and any other value is an error.
+//
+// valids[0] will be default value. Caller must be sure
+// len(valids) != 0 or it will panic.
+func NewSelectiveStringValue(valids ...string) *SelectiveStringValue {
+	vm := make(map[string]struct{})
+	for _, v := range valids {
+		vm[v] = struct{}{}
+	}
+	return &SelectiveStringValue{valids: vm, v: valids[0]}
+}
+
+// SelectiveStringsValue implements the flag.Value interface.
+type SelectiveStringsValue struct {
+	vs     []string
+	valids map[string]struct{}
+}
+
+// Set verifies the argument to be a valid member of the allowed values
+// before setting the underlying flag value.
+func (ss *SelectiveStringsValue) Set(s string) error {
+	vs := strings.Split(s, ",")
+	for i := range vs {
+		if _, ok := ss.valids[vs[i]]; ok {
+			ss.vs = append(ss.vs, vs[i])
+		} else {
+			return fmt.Errorf("invalid value %q", vs[i])
+		}
+	}
+	sort.Strings(ss.vs)
+	return nil
+}
+
+// String returns the set value (if any) of the SelectiveStringsValue.
+func (ss *SelectiveStringsValue) String() string {
+	return strings.Join(ss.vs, ",")
+}
+
+// Valids returns the list of valid strings.
+func (ss *SelectiveStringsValue) Valids() []string {
+	s := make([]string, 0, len(ss.valids))
+	for k := range ss.valids {
+		s = append(s, k)
+	}
+	sort.Strings(s)
+	return s
+}
+
+// NewSelectiveStringsValue creates a new string slice flag
+// for which any one of the given strings is a valid value,
+// and any other value is an error.
+func NewSelectiveStringsValue(valids ...string) *SelectiveStringsValue {
+	vm := make(map[string]struct{})
+	for _, v := range valids {
+		vm[v] = struct{}{}
+	}
+	return &SelectiveStringsValue{valids: vm, vs: []string{}}
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/flags/strings.go b/vendor/go.etcd.io/etcd/pkg/flags/strings.go
new file mode 100644
index 0000000..3e47fb3
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/flags/strings.go
@@ -0,0 +1,52 @@
+// Copyright 2018 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 flags
+
+import (
+	"flag"
+	"sort"
+	"strings"
+)
+
+// StringsValue wraps "sort.StringSlice".
+type StringsValue sort.StringSlice
+
+// Set parses a command line set of strings, separated by comma.
+// Implements "flag.Value" interface.
+func (ss *StringsValue) Set(s string) error {
+	*ss = strings.Split(s, ",")
+	return nil
+}
+
+// String implements "flag.Value" interface.
+func (ss *StringsValue) String() string { return strings.Join(*ss, ",") }
+
+// NewStringsValue implements string slice as "flag.Value" interface.
+// Given value is to be separated by comma.
+func NewStringsValue(s string) (ss *StringsValue) {
+	if s == "" {
+		return &StringsValue{}
+	}
+	ss = new(StringsValue)
+	if err := ss.Set(s); err != nil {
+		plog.Panicf("new StringsValue should never fail: %v", err)
+	}
+	return ss
+}
+
+// StringsFromFlag returns a string slice from the flag.
+func StringsFromFlag(fs *flag.FlagSet, flagName string) []string {
+	return []string(*fs.Lookup(flagName).Value.(*StringsValue))
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/flags/unique_strings.go b/vendor/go.etcd.io/etcd/pkg/flags/unique_strings.go
new file mode 100644
index 0000000..e220ee0
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/flags/unique_strings.go
@@ -0,0 +1,76 @@
+// Copyright 2018 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 flags
+
+import (
+	"flag"
+	"sort"
+	"strings"
+)
+
+// UniqueStringsValue wraps a list of unique strings.
+// The values are set in order.
+type UniqueStringsValue struct {
+	Values map[string]struct{}
+}
+
+// Set parses a command line set of strings, separated by comma.
+// Implements "flag.Value" interface.
+// The values are set in order.
+func (us *UniqueStringsValue) Set(s string) error {
+	us.Values = make(map[string]struct{})
+	for _, v := range strings.Split(s, ",") {
+		us.Values[v] = struct{}{}
+	}
+	return nil
+}
+
+// String implements "flag.Value" interface.
+func (us *UniqueStringsValue) String() string {
+	return strings.Join(us.stringSlice(), ",")
+}
+
+func (us *UniqueStringsValue) stringSlice() []string {
+	ss := make([]string, 0, len(us.Values))
+	for v := range us.Values {
+		ss = append(ss, v)
+	}
+	sort.Strings(ss)
+	return ss
+}
+
+// NewUniqueStringsValue implements string slice as "flag.Value" interface.
+// Given value is to be separated by comma.
+// The values are set in order.
+func NewUniqueStringsValue(s string) (us *UniqueStringsValue) {
+	us = &UniqueStringsValue{Values: make(map[string]struct{})}
+	if s == "" {
+		return us
+	}
+	if err := us.Set(s); err != nil {
+		plog.Panicf("new UniqueStringsValue should never fail: %v", err)
+	}
+	return us
+}
+
+// UniqueStringsFromFlag returns a string slice from the flag.
+func UniqueStringsFromFlag(fs *flag.FlagSet, flagName string) []string {
+	return (*fs.Lookup(flagName).Value.(*UniqueStringsValue)).stringSlice()
+}
+
+// UniqueStringsMapFromFlag returns a map of strings from the flag.
+func UniqueStringsMapFromFlag(fs *flag.FlagSet, flagName string) map[string]struct{} {
+	return (*fs.Lookup(flagName).Value.(*UniqueStringsValue)).Values
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/flags/unique_urls.go b/vendor/go.etcd.io/etcd/pkg/flags/unique_urls.go
new file mode 100644
index 0000000..9b4178c
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/flags/unique_urls.go
@@ -0,0 +1,92 @@
+// Copyright 2018 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 flags
+
+import (
+	"flag"
+	"net/url"
+	"sort"
+	"strings"
+
+	"go.etcd.io/etcd/pkg/types"
+)
+
+// UniqueURLs contains unique URLs
+// with non-URL exceptions.
+type UniqueURLs struct {
+	Values  map[string]struct{}
+	uss     []url.URL
+	Allowed map[string]struct{}
+}
+
+// Set parses a command line set of URLs formatted like:
+// http://127.0.0.1:2380,http://10.1.1.2:80
+// Implements "flag.Value" interface.
+func (us *UniqueURLs) Set(s string) error {
+	if _, ok := us.Values[s]; ok {
+		return nil
+	}
+	if _, ok := us.Allowed[s]; ok {
+		us.Values[s] = struct{}{}
+		return nil
+	}
+	ss, err := types.NewURLs(strings.Split(s, ","))
+	if err != nil {
+		return err
+	}
+	us.Values = make(map[string]struct{})
+	us.uss = make([]url.URL, 0)
+	for _, v := range ss {
+		us.Values[v.String()] = struct{}{}
+		us.uss = append(us.uss, v)
+	}
+	return nil
+}
+
+// String implements "flag.Value" interface.
+func (us *UniqueURLs) String() string {
+	all := make([]string, 0, len(us.Values))
+	for u := range us.Values {
+		all = append(all, u)
+	}
+	sort.Strings(all)
+	return strings.Join(all, ",")
+}
+
+// NewUniqueURLsWithExceptions implements "url.URL" slice as flag.Value interface.
+// Given value is to be separated by comma.
+func NewUniqueURLsWithExceptions(s string, exceptions ...string) *UniqueURLs {
+	us := &UniqueURLs{Values: make(map[string]struct{}), Allowed: make(map[string]struct{})}
+	for _, v := range exceptions {
+		us.Allowed[v] = struct{}{}
+	}
+	if s == "" {
+		return us
+	}
+	if err := us.Set(s); err != nil {
+		plog.Panicf("new UniqueURLs should never fail: %v", err)
+	}
+	return us
+}
+
+// UniqueURLsFromFlag returns a slice from urls got from the flag.
+func UniqueURLsFromFlag(fs *flag.FlagSet, urlsFlagName string) []url.URL {
+	return (*fs.Lookup(urlsFlagName).Value.(*UniqueURLs)).uss
+}
+
+// UniqueURLsMapFromFlag returns a map from url strings got from the flag.
+func UniqueURLsMapFromFlag(fs *flag.FlagSet, urlsFlagName string) map[string]struct{} {
+	return (*fs.Lookup(urlsFlagName).Value.(*UniqueURLs)).Values
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/flags/urls.go b/vendor/go.etcd.io/etcd/pkg/flags/urls.go
new file mode 100644
index 0000000..ca90970
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/flags/urls.go
@@ -0,0 +1,65 @@
+// Copyright 2015 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 flags
+
+import (
+	"flag"
+	"net/url"
+	"strings"
+
+	"go.etcd.io/etcd/pkg/types"
+)
+
+// URLsValue wraps "types.URLs".
+type URLsValue types.URLs
+
+// Set parses a command line set of URLs formatted like:
+// http://127.0.0.1:2380,http://10.1.1.2:80
+// Implements "flag.Value" interface.
+func (us *URLsValue) Set(s string) error {
+	ss, err := types.NewURLs(strings.Split(s, ","))
+	if err != nil {
+		return err
+	}
+	*us = URLsValue(ss)
+	return nil
+}
+
+// String implements "flag.Value" interface.
+func (us *URLsValue) String() string {
+	all := make([]string, len(*us))
+	for i, u := range *us {
+		all[i] = u.String()
+	}
+	return strings.Join(all, ",")
+}
+
+// NewURLsValue implements "url.URL" slice as flag.Value interface.
+// Given value is to be separated by comma.
+func NewURLsValue(s string) *URLsValue {
+	if s == "" {
+		return &URLsValue{}
+	}
+	v := &URLsValue{}
+	if err := v.Set(s); err != nil {
+		plog.Panicf("new URLsValue should never fail: %v", err)
+	}
+	return v
+}
+
+// URLsFromFlag returns a slices from url got from the flag.
+func URLsFromFlag(fs *flag.FlagSet, urlsFlagName string) []url.URL {
+	return []url.URL(*fs.Lookup(urlsFlagName).Value.(*URLsValue))
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/httputil/httputil.go b/vendor/go.etcd.io/etcd/pkg/httputil/httputil.go
new file mode 100644
index 0000000..3bf58a3
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/httputil/httputil.go
@@ -0,0 +1,50 @@
+// Copyright 2018 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.
+
+// Copyright 2015 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// Package httputil provides HTTP utility functions.
+package httputil
+
+import (
+	"io"
+	"io/ioutil"
+	"net"
+	"net/http"
+)
+
+// GracefulClose drains http.Response.Body until it hits EOF
+// and closes it. This prevents TCP/TLS connections from closing,
+// therefore available for reuse.
+// Borrowed from golang/net/context/ctxhttp/cancelreq.go.
+func GracefulClose(resp *http.Response) {
+	io.Copy(ioutil.Discard, resp.Body)
+	resp.Body.Close()
+}
+
+// GetHostname returns the hostname from request Host field.
+// It returns empty string, if Host field contains invalid
+// value (e.g. "localhost:::" with too many colons).
+func GetHostname(req *http.Request) string {
+	if req == nil {
+		return ""
+	}
+	h, _, err := net.SplitHostPort(req.Host)
+	if err != nil {
+		return req.Host
+	}
+	return h
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/idutil/id.go b/vendor/go.etcd.io/etcd/pkg/idutil/id.go
new file mode 100644
index 0000000..63a02cd
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/idutil/id.go
@@ -0,0 +1,75 @@
+// Copyright 2015 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 idutil implements utility functions for generating unique,
+// randomized ids.
+package idutil
+
+import (
+	"math"
+	"sync/atomic"
+	"time"
+)
+
+const (
+	tsLen     = 5 * 8
+	cntLen    = 8
+	suffixLen = tsLen + cntLen
+)
+
+// Generator generates unique identifiers based on counters, timestamps, and
+// a node member ID.
+//
+// The initial id is in this format:
+// High order 2 bytes are from memberID, next 5 bytes are from timestamp,
+// and low order one byte is a counter.
+// | prefix   | suffix              |
+// | 2 bytes  | 5 bytes   | 1 byte  |
+// | memberID | timestamp | cnt     |
+//
+// The timestamp 5 bytes is different when the machine is restart
+// after 1 ms and before 35 years.
+//
+// It increases suffix to generate the next id.
+// The count field may overflow to timestamp field, which is intentional.
+// It helps to extend the event window to 2^56. This doesn't break that
+// id generated after restart is unique because etcd throughput is <<
+// 256req/ms(250k reqs/second).
+type Generator struct {
+	// high order 2 bytes
+	prefix uint64
+	// low order 6 bytes
+	suffix uint64
+}
+
+func NewGenerator(memberID uint16, now time.Time) *Generator {
+	prefix := uint64(memberID) << suffixLen
+	unixMilli := uint64(now.UnixNano()) / uint64(time.Millisecond/time.Nanosecond)
+	suffix := lowbit(unixMilli, tsLen) << cntLen
+	return &Generator{
+		prefix: prefix,
+		suffix: suffix,
+	}
+}
+
+// Next generates a id that is unique.
+func (g *Generator) Next() uint64 {
+	suffix := atomic.AddUint64(&g.suffix, 1)
+	id := g.prefix | lowbit(suffix, suffixLen)
+	return id
+}
+
+func lowbit(x uint64, n uint) uint64 {
+	return x & (math.MaxUint64 >> (64 - n))
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/ioutil/pagewriter.go b/vendor/go.etcd.io/etcd/pkg/ioutil/pagewriter.go
new file mode 100644
index 0000000..72de159
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/ioutil/pagewriter.go
@@ -0,0 +1,106 @@
+// Copyright 2016 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 ioutil
+
+import (
+	"io"
+)
+
+var defaultBufferBytes = 128 * 1024
+
+// PageWriter implements the io.Writer interface so that writes will
+// either be in page chunks or from flushing.
+type PageWriter struct {
+	w io.Writer
+	// pageOffset tracks the page offset of the base of the buffer
+	pageOffset int
+	// pageBytes is the number of bytes per page
+	pageBytes int
+	// bufferedBytes counts the number of bytes pending for write in the buffer
+	bufferedBytes int
+	// buf holds the write buffer
+	buf []byte
+	// bufWatermarkBytes is the number of bytes the buffer can hold before it needs
+	// to be flushed. It is less than len(buf) so there is space for slack writes
+	// to bring the writer to page alignment.
+	bufWatermarkBytes int
+}
+
+// NewPageWriter creates a new PageWriter. pageBytes is the number of bytes
+// to write per page. pageOffset is the starting offset of io.Writer.
+func NewPageWriter(w io.Writer, pageBytes, pageOffset int) *PageWriter {
+	return &PageWriter{
+		w:                 w,
+		pageOffset:        pageOffset,
+		pageBytes:         pageBytes,
+		buf:               make([]byte, defaultBufferBytes+pageBytes),
+		bufWatermarkBytes: defaultBufferBytes,
+	}
+}
+
+func (pw *PageWriter) Write(p []byte) (n int, err error) {
+	if len(p)+pw.bufferedBytes <= pw.bufWatermarkBytes {
+		// no overflow
+		copy(pw.buf[pw.bufferedBytes:], p)
+		pw.bufferedBytes += len(p)
+		return len(p), nil
+	}
+	// complete the slack page in the buffer if unaligned
+	slack := pw.pageBytes - ((pw.pageOffset + pw.bufferedBytes) % pw.pageBytes)
+	if slack != pw.pageBytes {
+		partial := slack > len(p)
+		if partial {
+			// not enough data to complete the slack page
+			slack = len(p)
+		}
+		// special case: writing to slack page in buffer
+		copy(pw.buf[pw.bufferedBytes:], p[:slack])
+		pw.bufferedBytes += slack
+		n = slack
+		p = p[slack:]
+		if partial {
+			// avoid forcing an unaligned flush
+			return n, nil
+		}
+	}
+	// buffer contents are now page-aligned; clear out
+	if err = pw.Flush(); err != nil {
+		return n, err
+	}
+	// directly write all complete pages without copying
+	if len(p) > pw.pageBytes {
+		pages := len(p) / pw.pageBytes
+		c, werr := pw.w.Write(p[:pages*pw.pageBytes])
+		n += c
+		if werr != nil {
+			return n, werr
+		}
+		p = p[pages*pw.pageBytes:]
+	}
+	// write remaining tail to buffer
+	c, werr := pw.Write(p)
+	n += c
+	return n, werr
+}
+
+func (pw *PageWriter) Flush() error {
+	if pw.bufferedBytes == 0 {
+		return nil
+	}
+	_, err := pw.w.Write(pw.buf[:pw.bufferedBytes])
+	pw.pageOffset = (pw.pageOffset + pw.bufferedBytes) % pw.pageBytes
+	pw.bufferedBytes = 0
+	return err
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/ioutil/readcloser.go b/vendor/go.etcd.io/etcd/pkg/ioutil/readcloser.go
new file mode 100644
index 0000000..d3efcfe
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/ioutil/readcloser.go
@@ -0,0 +1,66 @@
+// Copyright 2015 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 ioutil
+
+import (
+	"fmt"
+	"io"
+)
+
+// ReaderAndCloser implements io.ReadCloser interface by combining
+// reader and closer together.
+type ReaderAndCloser struct {
+	io.Reader
+	io.Closer
+}
+
+var (
+	ErrShortRead = fmt.Errorf("ioutil: short read")
+	ErrExpectEOF = fmt.Errorf("ioutil: expect EOF")
+)
+
+// NewExactReadCloser returns a ReadCloser that returns errors if the underlying
+// reader does not read back exactly the requested number of bytes.
+func NewExactReadCloser(rc io.ReadCloser, totalBytes int64) io.ReadCloser {
+	return &exactReadCloser{rc: rc, totalBytes: totalBytes}
+}
+
+type exactReadCloser struct {
+	rc         io.ReadCloser
+	br         int64
+	totalBytes int64
+}
+
+func (e *exactReadCloser) Read(p []byte) (int, error) {
+	n, err := e.rc.Read(p)
+	e.br += int64(n)
+	if e.br > e.totalBytes {
+		return 0, ErrExpectEOF
+	}
+	if e.br < e.totalBytes && n == 0 {
+		return 0, ErrShortRead
+	}
+	return n, err
+}
+
+func (e *exactReadCloser) Close() error {
+	if err := e.rc.Close(); err != nil {
+		return err
+	}
+	if e.br < e.totalBytes {
+		return ErrShortRead
+	}
+	return nil
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/ioutil/reader.go b/vendor/go.etcd.io/etcd/pkg/ioutil/reader.go
new file mode 100644
index 0000000..0703ed4
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/ioutil/reader.go
@@ -0,0 +1,40 @@
+// Copyright 2015 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 ioutil implements I/O utility functions.
+package ioutil
+
+import "io"
+
+// NewLimitedBufferReader returns a reader that reads from the given reader
+// but limits the amount of data returned to at most n bytes.
+func NewLimitedBufferReader(r io.Reader, n int) io.Reader {
+	return &limitedBufferReader{
+		r: r,
+		n: n,
+	}
+}
+
+type limitedBufferReader struct {
+	r io.Reader
+	n int
+}
+
+func (r *limitedBufferReader) Read(p []byte) (n int, err error) {
+	np := p
+	if len(np) > r.n {
+		np = np[:r.n]
+	}
+	return r.r.Read(np)
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/ioutil/util.go b/vendor/go.etcd.io/etcd/pkg/ioutil/util.go
new file mode 100644
index 0000000..6a6746e
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/ioutil/util.go
@@ -0,0 +1,43 @@
+// Copyright 2015 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 ioutil
+
+import (
+	"io"
+	"os"
+
+	"go.etcd.io/etcd/pkg/fileutil"
+)
+
+// WriteAndSyncFile behaves just like ioutil.WriteFile in the standard library,
+// but calls Sync before closing the file. WriteAndSyncFile guarantees the data
+// is synced if there is no error returned.
+func WriteAndSyncFile(filename string, data []byte, perm os.FileMode) error {
+	f, err := os.OpenFile(filename, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, perm)
+	if err != nil {
+		return err
+	}
+	n, err := f.Write(data)
+	if err == nil && n < len(data) {
+		err = io.ErrShortWrite
+	}
+	if err == nil {
+		err = fileutil.Fsync(f)
+	}
+	if err1 := f.Close(); err == nil {
+		err = err1
+	}
+	return err
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/netutil/doc.go b/vendor/go.etcd.io/etcd/pkg/netutil/doc.go
new file mode 100644
index 0000000..5d92d03
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/netutil/doc.go
@@ -0,0 +1,16 @@
+// Copyright 2018 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 netutil implements network-related utility functions.
+package netutil
diff --git a/vendor/go.etcd.io/etcd/pkg/netutil/isolate_linux.go b/vendor/go.etcd.io/etcd/pkg/netutil/isolate_linux.go
new file mode 100644
index 0000000..418580a
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/netutil/isolate_linux.go
@@ -0,0 +1,82 @@
+// Copyright 2015 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 netutil
+
+import (
+	"fmt"
+	"os/exec"
+)
+
+// DropPort drops all tcp packets that are received from the given port and sent to the given port.
+func DropPort(port int) error {
+	cmdStr := fmt.Sprintf("sudo iptables -A OUTPUT -p tcp --destination-port %d -j DROP", port)
+	if _, err := exec.Command("/bin/sh", "-c", cmdStr).Output(); err != nil {
+		return err
+	}
+	cmdStr = fmt.Sprintf("sudo iptables -A INPUT -p tcp --destination-port %d -j DROP", port)
+	_, err := exec.Command("/bin/sh", "-c", cmdStr).Output()
+	return err
+}
+
+// RecoverPort stops dropping tcp packets at given port.
+func RecoverPort(port int) error {
+	cmdStr := fmt.Sprintf("sudo iptables -D OUTPUT -p tcp --destination-port %d -j DROP", port)
+	if _, err := exec.Command("/bin/sh", "-c", cmdStr).Output(); err != nil {
+		return err
+	}
+	cmdStr = fmt.Sprintf("sudo iptables -D INPUT -p tcp --destination-port %d -j DROP", port)
+	_, err := exec.Command("/bin/sh", "-c", cmdStr).Output()
+	return err
+}
+
+// SetLatency adds latency in millisecond scale with random variations.
+func SetLatency(ms, rv int) error {
+	ifces, err := GetDefaultInterfaces()
+	if err != nil {
+		return err
+	}
+
+	if rv > ms {
+		rv = 1
+	}
+	for ifce := range ifces {
+		cmdStr := fmt.Sprintf("sudo tc qdisc add dev %s root netem delay %dms %dms distribution normal", ifce, ms, rv)
+		_, err = exec.Command("/bin/sh", "-c", cmdStr).Output()
+		if err != nil {
+			// the rule has already been added. Overwrite it.
+			cmdStr = fmt.Sprintf("sudo tc qdisc change dev %s root netem delay %dms %dms distribution normal", ifce, ms, rv)
+			_, err = exec.Command("/bin/sh", "-c", cmdStr).Output()
+			if err != nil {
+				return err
+			}
+		}
+	}
+	return nil
+}
+
+// RemoveLatency resets latency configurations.
+func RemoveLatency() error {
+	ifces, err := GetDefaultInterfaces()
+	if err != nil {
+		return err
+	}
+	for ifce := range ifces {
+		_, err = exec.Command("/bin/sh", "-c", fmt.Sprintf("sudo tc qdisc del dev %s root netem", ifce)).Output()
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/netutil/isolate_stub.go b/vendor/go.etcd.io/etcd/pkg/netutil/isolate_stub.go
new file mode 100644
index 0000000..7f4c3e6
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/netutil/isolate_stub.go
@@ -0,0 +1,25 @@
+// Copyright 2015 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.
+
+// +build !linux
+
+package netutil
+
+func DropPort(port int) error { return nil }
+
+func RecoverPort(port int) error { return nil }
+
+func SetLatency(ms, rv int) error { return nil }
+
+func RemoveLatency() error { return nil }
diff --git a/vendor/go.etcd.io/etcd/pkg/netutil/netutil.go b/vendor/go.etcd.io/etcd/pkg/netutil/netutil.go
new file mode 100644
index 0000000..faef646
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/netutil/netutil.go
@@ -0,0 +1,213 @@
+// Copyright 2015 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 netutil
+
+import (
+	"context"
+	"fmt"
+	"net"
+	"net/url"
+	"reflect"
+	"sort"
+	"time"
+
+	"go.etcd.io/etcd/pkg/types"
+
+	"go.uber.org/zap"
+)
+
+// indirection for testing
+var resolveTCPAddr = resolveTCPAddrDefault
+
+const retryInterval = time.Second
+
+// taken from go's ResolveTCP code but uses configurable ctx
+func resolveTCPAddrDefault(ctx context.Context, addr string) (*net.TCPAddr, error) {
+	host, port, serr := net.SplitHostPort(addr)
+	if serr != nil {
+		return nil, serr
+	}
+	portnum, perr := net.DefaultResolver.LookupPort(ctx, "tcp", port)
+	if perr != nil {
+		return nil, perr
+	}
+
+	var ips []net.IPAddr
+	if ip := net.ParseIP(host); ip != nil {
+		ips = []net.IPAddr{{IP: ip}}
+	} else {
+		// Try as a DNS name.
+		ipss, err := net.DefaultResolver.LookupIPAddr(ctx, host)
+		if err != nil {
+			return nil, err
+		}
+		ips = ipss
+	}
+	// randomize?
+	ip := ips[0]
+	return &net.TCPAddr{IP: ip.IP, Port: portnum, Zone: ip.Zone}, nil
+}
+
+// resolveTCPAddrs is a convenience wrapper for net.ResolveTCPAddr.
+// resolveTCPAddrs return a new set of url.URLs, in which all DNS hostnames
+// are resolved.
+func resolveTCPAddrs(ctx context.Context, lg *zap.Logger, urls [][]url.URL) ([][]url.URL, error) {
+	newurls := make([][]url.URL, 0)
+	for _, us := range urls {
+		nus := make([]url.URL, len(us))
+		for i, u := range us {
+			nu, err := url.Parse(u.String())
+			if err != nil {
+				return nil, fmt.Errorf("failed to parse %q (%v)", u.String(), err)
+			}
+			nus[i] = *nu
+		}
+		for i, u := range nus {
+			h, err := resolveURL(ctx, lg, u)
+			if err != nil {
+				return nil, fmt.Errorf("failed to resolve %q (%v)", u.String(), err)
+			}
+			if h != "" {
+				nus[i].Host = h
+			}
+		}
+		newurls = append(newurls, nus)
+	}
+	return newurls, nil
+}
+
+func resolveURL(ctx context.Context, lg *zap.Logger, u url.URL) (string, error) {
+	if u.Scheme == "unix" || u.Scheme == "unixs" {
+		// unix sockets don't resolve over TCP
+		return "", nil
+	}
+	host, _, err := net.SplitHostPort(u.Host)
+	if err != nil {
+		lg.Warn(
+			"failed to parse URL Host while resolving URL",
+			zap.String("url", u.String()),
+			zap.String("host", u.Host),
+			zap.Error(err),
+		)
+		return "", err
+	}
+	if host == "localhost" || net.ParseIP(host) != nil {
+		return "", nil
+	}
+	for ctx.Err() == nil {
+		tcpAddr, err := resolveTCPAddr(ctx, u.Host)
+		if err == nil {
+			lg.Info(
+				"resolved URL Host",
+				zap.String("url", u.String()),
+				zap.String("host", u.Host),
+				zap.String("resolved-addr", tcpAddr.String()),
+			)
+			return tcpAddr.String(), nil
+		}
+
+		lg.Warn(
+			"failed to resolve URL Host",
+			zap.String("url", u.String()),
+			zap.String("host", u.Host),
+			zap.Duration("retry-interval", retryInterval),
+			zap.Error(err),
+		)
+
+		select {
+		case <-ctx.Done():
+			lg.Warn(
+				"failed to resolve URL Host; returning",
+				zap.String("url", u.String()),
+				zap.String("host", u.Host),
+				zap.Duration("retry-interval", retryInterval),
+				zap.Error(err),
+			)
+			return "", err
+		case <-time.After(retryInterval):
+		}
+	}
+	return "", ctx.Err()
+}
+
+// urlsEqual checks equality of url.URLS between two arrays.
+// This check pass even if an URL is in hostname and opposite is in IP address.
+func urlsEqual(ctx context.Context, lg *zap.Logger, a []url.URL, b []url.URL) (bool, error) {
+	if len(a) != len(b) {
+		return false, fmt.Errorf("len(%q) != len(%q)", urlsToStrings(a), urlsToStrings(b))
+	}
+	urls, err := resolveTCPAddrs(ctx, lg, [][]url.URL{a, b})
+	if err != nil {
+		return false, err
+	}
+	preva, prevb := a, b
+	a, b = urls[0], urls[1]
+	sort.Sort(types.URLs(a))
+	sort.Sort(types.URLs(b))
+	for i := range a {
+		if !reflect.DeepEqual(a[i], b[i]) {
+			return false, fmt.Errorf("%q(resolved from %q) != %q(resolved from %q)",
+				a[i].String(), preva[i].String(),
+				b[i].String(), prevb[i].String(),
+			)
+		}
+	}
+	return true, nil
+}
+
+// URLStringsEqual returns "true" if given URLs are valid
+// and resolved to same IP addresses. Otherwise, return "false"
+// and error, if any.
+func URLStringsEqual(ctx context.Context, lg *zap.Logger, a []string, b []string) (bool, error) {
+	if len(a) != len(b) {
+		return false, fmt.Errorf("len(%q) != len(%q)", a, b)
+	}
+	urlsA := make([]url.URL, 0)
+	for _, str := range a {
+		u, err := url.Parse(str)
+		if err != nil {
+			return false, fmt.Errorf("failed to parse %q", str)
+		}
+		urlsA = append(urlsA, *u)
+	}
+	urlsB := make([]url.URL, 0)
+	for _, str := range b {
+		u, err := url.Parse(str)
+		if err != nil {
+			return false, fmt.Errorf("failed to parse %q", str)
+		}
+		urlsB = append(urlsB, *u)
+	}
+	if lg == nil {
+		lg, _ = zap.NewProduction()
+		if lg == nil {
+			lg = zap.NewExample()
+		}
+	}
+	return urlsEqual(ctx, lg, urlsA, urlsB)
+}
+
+func urlsToStrings(us []url.URL) []string {
+	rs := make([]string, len(us))
+	for i := range us {
+		rs[i] = us[i].String()
+	}
+	return rs
+}
+
+func IsNetworkTimeoutError(err error) bool {
+	nerr, ok := err.(net.Error)
+	return ok && nerr.Timeout()
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/netutil/routes.go b/vendor/go.etcd.io/etcd/pkg/netutil/routes.go
new file mode 100644
index 0000000..3eb6a19
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/netutil/routes.go
@@ -0,0 +1,33 @@
+// Copyright 2016 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.
+
+// +build !linux
+
+package netutil
+
+import (
+	"fmt"
+	"runtime"
+)
+
+// GetDefaultHost fetches the a resolvable name that corresponds
+// to the machine's default routable interface
+func GetDefaultHost() (string, error) {
+	return "", fmt.Errorf("default host not supported on %s_%s", runtime.GOOS, runtime.GOARCH)
+}
+
+// GetDefaultInterfaces fetches the device name of default routable interface.
+func GetDefaultInterfaces() (map[string]uint8, error) {
+	return nil, fmt.Errorf("default host not supported on %s_%s", runtime.GOOS, runtime.GOARCH)
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/netutil/routes_linux.go b/vendor/go.etcd.io/etcd/pkg/netutil/routes_linux.go
new file mode 100644
index 0000000..5118d3d
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/netutil/routes_linux.go
@@ -0,0 +1,250 @@
+// Copyright 2016 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.
+
+// +build linux
+
+package netutil
+
+import (
+	"bytes"
+	"encoding/binary"
+	"fmt"
+	"net"
+	"sort"
+	"syscall"
+
+	"go.etcd.io/etcd/pkg/cpuutil"
+)
+
+var errNoDefaultRoute = fmt.Errorf("could not find default route")
+var errNoDefaultHost = fmt.Errorf("could not find default host")
+var errNoDefaultInterface = fmt.Errorf("could not find default interface")
+
+// GetDefaultHost obtains the first IP address of machine from the routing table and returns the IP address as string.
+// An IPv4 address is preferred to an IPv6 address for backward compatibility.
+func GetDefaultHost() (string, error) {
+	rmsgs, rerr := getDefaultRoutes()
+	if rerr != nil {
+		return "", rerr
+	}
+
+	// prioritize IPv4
+	if rmsg, ok := rmsgs[syscall.AF_INET]; ok {
+		if host, err := chooseHost(syscall.AF_INET, rmsg); host != "" || err != nil {
+			return host, err
+		}
+		delete(rmsgs, syscall.AF_INET)
+	}
+
+	// sort so choice is deterministic
+	var families []int
+	for family := range rmsgs {
+		families = append(families, int(family))
+	}
+	sort.Ints(families)
+
+	for _, f := range families {
+		family := uint8(f)
+		if host, err := chooseHost(family, rmsgs[family]); host != "" || err != nil {
+			return host, err
+		}
+	}
+
+	return "", errNoDefaultHost
+}
+
+func chooseHost(family uint8, rmsg *syscall.NetlinkMessage) (string, error) {
+	host, oif, err := parsePREFSRC(rmsg)
+	if host != "" || err != nil {
+		return host, err
+	}
+
+	// prefsrc not detected, fall back to getting address from iface
+	ifmsg, ierr := getIfaceAddr(oif, family)
+	if ierr != nil {
+		return "", ierr
+	}
+
+	attrs, aerr := syscall.ParseNetlinkRouteAttr(ifmsg)
+	if aerr != nil {
+		return "", aerr
+	}
+
+	for _, attr := range attrs {
+		// search for RTA_DST because ipv6 doesn't have RTA_SRC
+		if attr.Attr.Type == syscall.RTA_DST {
+			return net.IP(attr.Value).String(), nil
+		}
+	}
+
+	return "", nil
+}
+
+func getDefaultRoutes() (map[uint8]*syscall.NetlinkMessage, error) {
+	dat, err := syscall.NetlinkRIB(syscall.RTM_GETROUTE, syscall.AF_UNSPEC)
+	if err != nil {
+		return nil, err
+	}
+
+	msgs, msgErr := syscall.ParseNetlinkMessage(dat)
+	if msgErr != nil {
+		return nil, msgErr
+	}
+
+	routes := make(map[uint8]*syscall.NetlinkMessage)
+	rtmsg := syscall.RtMsg{}
+	for _, m := range msgs {
+		if m.Header.Type != syscall.RTM_NEWROUTE {
+			continue
+		}
+		buf := bytes.NewBuffer(m.Data[:syscall.SizeofRtMsg])
+		if rerr := binary.Read(buf, cpuutil.ByteOrder(), &rtmsg); rerr != nil {
+			continue
+		}
+		if rtmsg.Dst_len == 0 && rtmsg.Table == syscall.RT_TABLE_MAIN {
+			// zero-length Dst_len implies default route
+			msg := m
+			routes[rtmsg.Family] = &msg
+		}
+	}
+
+	if len(routes) > 0 {
+		return routes, nil
+	}
+
+	return nil, errNoDefaultRoute
+}
+
+// Used to get an address of interface.
+func getIfaceAddr(idx uint32, family uint8) (*syscall.NetlinkMessage, error) {
+	dat, err := syscall.NetlinkRIB(syscall.RTM_GETADDR, int(family))
+	if err != nil {
+		return nil, err
+	}
+
+	msgs, msgErr := syscall.ParseNetlinkMessage(dat)
+	if msgErr != nil {
+		return nil, msgErr
+	}
+
+	ifaddrmsg := syscall.IfAddrmsg{}
+	for _, m := range msgs {
+		if m.Header.Type != syscall.RTM_NEWADDR {
+			continue
+		}
+		buf := bytes.NewBuffer(m.Data[:syscall.SizeofIfAddrmsg])
+		if rerr := binary.Read(buf, cpuutil.ByteOrder(), &ifaddrmsg); rerr != nil {
+			continue
+		}
+		if ifaddrmsg.Index == idx {
+			return &m, nil
+		}
+	}
+
+	return nil, fmt.Errorf("could not find address for interface index %v", idx)
+
+}
+
+// Used to get a name of interface.
+func getIfaceLink(idx uint32) (*syscall.NetlinkMessage, error) {
+	dat, err := syscall.NetlinkRIB(syscall.RTM_GETLINK, syscall.AF_UNSPEC)
+	if err != nil {
+		return nil, err
+	}
+
+	msgs, msgErr := syscall.ParseNetlinkMessage(dat)
+	if msgErr != nil {
+		return nil, msgErr
+	}
+
+	ifinfomsg := syscall.IfInfomsg{}
+	for _, m := range msgs {
+		if m.Header.Type != syscall.RTM_NEWLINK {
+			continue
+		}
+		buf := bytes.NewBuffer(m.Data[:syscall.SizeofIfInfomsg])
+		if rerr := binary.Read(buf, cpuutil.ByteOrder(), &ifinfomsg); rerr != nil {
+			continue
+		}
+		if ifinfomsg.Index == int32(idx) {
+			return &m, nil
+		}
+	}
+
+	return nil, fmt.Errorf("could not find link for interface index %v", idx)
+}
+
+// GetDefaultInterfaces gets names of interfaces and returns a map[interface]families.
+func GetDefaultInterfaces() (map[string]uint8, error) {
+	interfaces := make(map[string]uint8)
+	rmsgs, rerr := getDefaultRoutes()
+	if rerr != nil {
+		return interfaces, rerr
+	}
+
+	for family, rmsg := range rmsgs {
+		_, oif, err := parsePREFSRC(rmsg)
+		if err != nil {
+			return interfaces, err
+		}
+
+		ifmsg, ierr := getIfaceLink(oif)
+		if ierr != nil {
+			return interfaces, ierr
+		}
+
+		attrs, aerr := syscall.ParseNetlinkRouteAttr(ifmsg)
+		if aerr != nil {
+			return interfaces, aerr
+		}
+
+		for _, attr := range attrs {
+			if attr.Attr.Type == syscall.IFLA_IFNAME {
+				// key is an interface name
+				// possible values: 2 - AF_INET, 10 - AF_INET6, 12 - dualstack
+				interfaces[string(attr.Value[:len(attr.Value)-1])] += family
+			}
+		}
+	}
+	if len(interfaces) > 0 {
+		return interfaces, nil
+	}
+	return interfaces, errNoDefaultInterface
+}
+
+// parsePREFSRC returns preferred source address and output interface index (RTA_OIF).
+func parsePREFSRC(m *syscall.NetlinkMessage) (host string, oif uint32, err error) {
+	var attrs []syscall.NetlinkRouteAttr
+	attrs, err = syscall.ParseNetlinkRouteAttr(m)
+	if err != nil {
+		return "", 0, err
+	}
+
+	for _, attr := range attrs {
+		if attr.Attr.Type == syscall.RTA_PREFSRC {
+			host = net.IP(attr.Value).String()
+		}
+		if attr.Attr.Type == syscall.RTA_OIF {
+			oif = cpuutil.ByteOrder().Uint32(attr.Value)
+		}
+		if host != "" && oif != uint32(0) {
+			break
+		}
+	}
+
+	if oif == 0 {
+		err = errNoDefaultRoute
+	}
+	return host, oif, err
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/pathutil/path.go b/vendor/go.etcd.io/etcd/pkg/pathutil/path.go
new file mode 100644
index 0000000..f26254b
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/pathutil/path.go
@@ -0,0 +1,31 @@
+// Copyright 2009 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// Package pathutil implements utility functions for handling slash-separated
+// paths.
+package pathutil
+
+import "path"
+
+// CanonicalURLPath returns the canonical url path for p, which follows the rules:
+// 1. the path always starts with "/"
+// 2. replace multiple slashes with a single slash
+// 3. replace each '.' '..' path name element with equivalent one
+// 4. keep the trailing slash
+// The function is borrowed from stdlib http.cleanPath in server.go.
+func CanonicalURLPath(p string) string {
+	if p == "" {
+		return "/"
+	}
+	if p[0] != '/' {
+		p = "/" + p
+	}
+	np := path.Clean(p)
+	// path.Clean removes trailing slash except for root,
+	// put the trailing slash back if necessary.
+	if p[len(p)-1] == '/' && np != "/" {
+		np += "/"
+	}
+	return np
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/pbutil/pbutil.go b/vendor/go.etcd.io/etcd/pkg/pbutil/pbutil.go
new file mode 100644
index 0000000..53167ff
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/pbutil/pbutil.go
@@ -0,0 +1,60 @@
+// Copyright 2015 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 pbutil defines interfaces for handling Protocol Buffer objects.
+package pbutil
+
+import "github.com/coreos/pkg/capnslog"
+
+var (
+	plog = capnslog.NewPackageLogger("go.etcd.io/etcd", "pkg/pbutil")
+)
+
+type Marshaler interface {
+	Marshal() (data []byte, err error)
+}
+
+type Unmarshaler interface {
+	Unmarshal(data []byte) error
+}
+
+func MustMarshal(m Marshaler) []byte {
+	d, err := m.Marshal()
+	if err != nil {
+		plog.Panicf("marshal should never fail (%v)", err)
+	}
+	return d
+}
+
+func MustUnmarshal(um Unmarshaler, data []byte) {
+	if err := um.Unmarshal(data); err != nil {
+		plog.Panicf("unmarshal should never fail (%v)", err)
+	}
+}
+
+func MaybeUnmarshal(um Unmarshaler, data []byte) bool {
+	if err := um.Unmarshal(data); err != nil {
+		return false
+	}
+	return true
+}
+
+func GetBool(v *bool) (vv bool, set bool) {
+	if v == nil {
+		return false, false
+	}
+	return *v, true
+}
+
+func Boolp(b bool) *bool { return &b }
diff --git a/vendor/go.etcd.io/etcd/pkg/runtime/fds_linux.go b/vendor/go.etcd.io/etcd/pkg/runtime/fds_linux.go
new file mode 100644
index 0000000..8e9359d
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/runtime/fds_linux.go
@@ -0,0 +1,37 @@
+// Copyright 2015 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 runtime implements utility functions for runtime systems.
+package runtime
+
+import (
+	"io/ioutil"
+	"syscall"
+)
+
+func FDLimit() (uint64, error) {
+	var rlimit syscall.Rlimit
+	if err := syscall.Getrlimit(syscall.RLIMIT_NOFILE, &rlimit); err != nil {
+		return 0, err
+	}
+	return rlimit.Cur, nil
+}
+
+func FDUsage() (uint64, error) {
+	fds, err := ioutil.ReadDir("/proc/self/fd")
+	if err != nil {
+		return 0, err
+	}
+	return uint64(len(fds)), nil
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/runtime/fds_other.go b/vendor/go.etcd.io/etcd/pkg/runtime/fds_other.go
new file mode 100644
index 0000000..0cbdb88
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/runtime/fds_other.go
@@ -0,0 +1,30 @@
+// Copyright 2015 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.
+
+// +build !linux
+
+package runtime
+
+import (
+	"fmt"
+	"runtime"
+)
+
+func FDLimit() (uint64, error) {
+	return 0, fmt.Errorf("cannot get FDLimit on %s", runtime.GOOS)
+}
+
+func FDUsage() (uint64, error) {
+	return 0, fmt.Errorf("cannot get FDUsage on %s", runtime.GOOS)
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/schedule/doc.go b/vendor/go.etcd.io/etcd/pkg/schedule/doc.go
new file mode 100644
index 0000000..cca2c75
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/schedule/doc.go
@@ -0,0 +1,16 @@
+// Copyright 2016 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 schedule provides mechanisms and policies for scheduling units of work.
+package schedule
diff --git a/vendor/go.etcd.io/etcd/pkg/schedule/schedule.go b/vendor/go.etcd.io/etcd/pkg/schedule/schedule.go
new file mode 100644
index 0000000..234d019
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/schedule/schedule.go
@@ -0,0 +1,165 @@
+// Copyright 2016 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 schedule
+
+import (
+	"context"
+	"sync"
+)
+
+type Job func(context.Context)
+
+// Scheduler can schedule jobs.
+type Scheduler interface {
+	// Schedule asks the scheduler to schedule a job defined by the given func.
+	// Schedule to a stopped scheduler might panic.
+	Schedule(j Job)
+
+	// Pending returns number of pending jobs
+	Pending() int
+
+	// Scheduled returns the number of scheduled jobs (excluding pending jobs)
+	Scheduled() int
+
+	// Finished returns the number of finished jobs
+	Finished() int
+
+	// WaitFinish waits until at least n job are finished and all pending jobs are finished.
+	WaitFinish(n int)
+
+	// Stop stops the scheduler.
+	Stop()
+}
+
+type fifo struct {
+	mu sync.Mutex
+
+	resume    chan struct{}
+	scheduled int
+	finished  int
+	pendings  []Job
+
+	ctx    context.Context
+	cancel context.CancelFunc
+
+	finishCond *sync.Cond
+	donec      chan struct{}
+}
+
+// NewFIFOScheduler returns a Scheduler that schedules jobs in FIFO
+// order sequentially
+func NewFIFOScheduler() Scheduler {
+	f := &fifo{
+		resume: make(chan struct{}, 1),
+		donec:  make(chan struct{}, 1),
+	}
+	f.finishCond = sync.NewCond(&f.mu)
+	f.ctx, f.cancel = context.WithCancel(context.Background())
+	go f.run()
+	return f
+}
+
+// Schedule schedules a job that will be ran in FIFO order sequentially.
+func (f *fifo) Schedule(j Job) {
+	f.mu.Lock()
+	defer f.mu.Unlock()
+
+	if f.cancel == nil {
+		panic("schedule: schedule to stopped scheduler")
+	}
+
+	if len(f.pendings) == 0 {
+		select {
+		case f.resume <- struct{}{}:
+		default:
+		}
+	}
+	f.pendings = append(f.pendings, j)
+}
+
+func (f *fifo) Pending() int {
+	f.mu.Lock()
+	defer f.mu.Unlock()
+	return len(f.pendings)
+}
+
+func (f *fifo) Scheduled() int {
+	f.mu.Lock()
+	defer f.mu.Unlock()
+	return f.scheduled
+}
+
+func (f *fifo) Finished() int {
+	f.finishCond.L.Lock()
+	defer f.finishCond.L.Unlock()
+	return f.finished
+}
+
+func (f *fifo) WaitFinish(n int) {
+	f.finishCond.L.Lock()
+	for f.finished < n || len(f.pendings) != 0 {
+		f.finishCond.Wait()
+	}
+	f.finishCond.L.Unlock()
+}
+
+// Stop stops the scheduler and cancels all pending jobs.
+func (f *fifo) Stop() {
+	f.mu.Lock()
+	f.cancel()
+	f.cancel = nil
+	f.mu.Unlock()
+	<-f.donec
+}
+
+func (f *fifo) run() {
+	// TODO: recover from job panic?
+	defer func() {
+		close(f.donec)
+		close(f.resume)
+	}()
+
+	for {
+		var todo Job
+		f.mu.Lock()
+		if len(f.pendings) != 0 {
+			f.scheduled++
+			todo = f.pendings[0]
+		}
+		f.mu.Unlock()
+		if todo == nil {
+			select {
+			case <-f.resume:
+			case <-f.ctx.Done():
+				f.mu.Lock()
+				pendings := f.pendings
+				f.pendings = nil
+				f.mu.Unlock()
+				// clean up pending jobs
+				for _, todo := range pendings {
+					todo(f.ctx)
+				}
+				return
+			}
+		} else {
+			todo(f.ctx)
+			f.finishCond.L.Lock()
+			f.finished++
+			f.pendings = f.pendings[1:]
+			f.finishCond.Broadcast()
+			f.finishCond.L.Unlock()
+		}
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/srv/srv.go b/vendor/go.etcd.io/etcd/pkg/srv/srv.go
new file mode 100644
index 0000000..c356002
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/srv/srv.go
@@ -0,0 +1,142 @@
+// Copyright 2015 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 srv looks up DNS SRV records.
+package srv
+
+import (
+	"fmt"
+	"net"
+	"net/url"
+	"strings"
+
+	"go.etcd.io/etcd/pkg/types"
+)
+
+var (
+	// indirection for testing
+	lookupSRV      = net.LookupSRV // net.DefaultResolver.LookupSRV when ctxs don't conflict
+	resolveTCPAddr = net.ResolveTCPAddr
+)
+
+// GetCluster gets the cluster information via DNS discovery.
+// Also sees each entry as a separate instance.
+func GetCluster(serviceScheme, service, name, dns string, apurls types.URLs) ([]string, error) {
+	tempName := int(0)
+	tcp2ap := make(map[string]url.URL)
+
+	// First, resolve the apurls
+	for _, url := range apurls {
+		tcpAddr, err := resolveTCPAddr("tcp", url.Host)
+		if err != nil {
+			return nil, err
+		}
+		tcp2ap[tcpAddr.String()] = url
+	}
+
+	stringParts := []string{}
+	updateNodeMap := func(service, scheme string) error {
+		_, addrs, err := lookupSRV(service, "tcp", dns)
+		if err != nil {
+			return err
+		}
+		for _, srv := range addrs {
+			port := fmt.Sprintf("%d", srv.Port)
+			host := net.JoinHostPort(srv.Target, port)
+			tcpAddr, terr := resolveTCPAddr("tcp", host)
+			if terr != nil {
+				err = terr
+				continue
+			}
+			n := ""
+			url, ok := tcp2ap[tcpAddr.String()]
+			if ok {
+				n = name
+			}
+			if n == "" {
+				n = fmt.Sprintf("%d", tempName)
+				tempName++
+			}
+			// SRV records have a trailing dot but URL shouldn't.
+			shortHost := strings.TrimSuffix(srv.Target, ".")
+			urlHost := net.JoinHostPort(shortHost, port)
+			if ok && url.Scheme != scheme {
+				err = fmt.Errorf("bootstrap at %s from DNS for %s has scheme mismatch with expected peer %s", scheme+"://"+urlHost, service, url.String())
+			} else {
+				stringParts = append(stringParts, fmt.Sprintf("%s=%s://%s", n, scheme, urlHost))
+			}
+		}
+		if len(stringParts) == 0 {
+			return err
+		}
+		return nil
+	}
+
+	err := updateNodeMap(service, serviceScheme)
+	if err != nil {
+		return nil, fmt.Errorf("error querying DNS SRV records for _%s %s", service, err)
+	}
+	return stringParts, nil
+}
+
+type SRVClients struct {
+	Endpoints []string
+	SRVs      []*net.SRV
+}
+
+// GetClient looks up the client endpoints for a service and domain.
+func GetClient(service, domain string, serviceName string) (*SRVClients, error) {
+	var urls []*url.URL
+	var srvs []*net.SRV
+
+	updateURLs := func(service, scheme string) error {
+		_, addrs, err := lookupSRV(service, "tcp", domain)
+		if err != nil {
+			return err
+		}
+		for _, srv := range addrs {
+			urls = append(urls, &url.URL{
+				Scheme: scheme,
+				Host:   net.JoinHostPort(srv.Target, fmt.Sprintf("%d", srv.Port)),
+			})
+		}
+		srvs = append(srvs, addrs...)
+		return nil
+	}
+
+	errHTTPS := updateURLs(GetSRVService(service, serviceName, "https"), "https")
+	errHTTP := updateURLs(GetSRVService(service, serviceName, "http"), "http")
+
+	if errHTTPS != nil && errHTTP != nil {
+		return nil, fmt.Errorf("dns lookup errors: %s and %s", errHTTPS, errHTTP)
+	}
+
+	endpoints := make([]string, len(urls))
+	for i := range urls {
+		endpoints[i] = urls[i].String()
+	}
+	return &SRVClients{Endpoints: endpoints, SRVs: srvs}, nil
+}
+
+// GetSRVService generates a SRV service including an optional suffix.
+func GetSRVService(service, serviceName string, scheme string) (SRVService string) {
+	if scheme == "https" {
+		service = fmt.Sprintf("%s-ssl", service)
+	}
+
+	if serviceName != "" {
+		return fmt.Sprintf("%s-%s", service, serviceName)
+	}
+	return service
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/tlsutil/cipher_suites.go b/vendor/go.etcd.io/etcd/pkg/tlsutil/cipher_suites.go
new file mode 100644
index 0000000..b5916bb
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/tlsutil/cipher_suites.go
@@ -0,0 +1,51 @@
+// Copyright 2018 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 tlsutil
+
+import "crypto/tls"
+
+// cipher suites implemented by Go
+// https://github.com/golang/go/blob/dev.boringcrypto.go1.10/src/crypto/tls/cipher_suites.go
+var cipherSuites = map[string]uint16{
+	"TLS_RSA_WITH_RC4_128_SHA":                tls.TLS_RSA_WITH_RC4_128_SHA,
+	"TLS_RSA_WITH_3DES_EDE_CBC_SHA":           tls.TLS_RSA_WITH_3DES_EDE_CBC_SHA,
+	"TLS_RSA_WITH_AES_128_CBC_SHA":            tls.TLS_RSA_WITH_AES_128_CBC_SHA,
+	"TLS_RSA_WITH_AES_256_CBC_SHA":            tls.TLS_RSA_WITH_AES_256_CBC_SHA,
+	"TLS_RSA_WITH_AES_128_CBC_SHA256":         tls.TLS_RSA_WITH_AES_128_CBC_SHA256,
+	"TLS_RSA_WITH_AES_128_GCM_SHA256":         tls.TLS_RSA_WITH_AES_128_GCM_SHA256,
+	"TLS_RSA_WITH_AES_256_GCM_SHA384":         tls.TLS_RSA_WITH_AES_256_GCM_SHA384,
+	"TLS_ECDHE_ECDSA_WITH_RC4_128_SHA":        tls.TLS_ECDHE_ECDSA_WITH_RC4_128_SHA,
+	"TLS_ECDHE_ECDSA_WITH_AES_128_CBC_SHA":    tls.TLS_ECDHE_ECDSA_WITH_AES_128_CBC_SHA,
+	"TLS_ECDHE_ECDSA_WITH_AES_256_CBC_SHA":    tls.TLS_ECDHE_ECDSA_WITH_AES_256_CBC_SHA,
+	"TLS_ECDHE_RSA_WITH_RC4_128_SHA":          tls.TLS_ECDHE_RSA_WITH_RC4_128_SHA,
+	"TLS_ECDHE_RSA_WITH_3DES_EDE_CBC_SHA":     tls.TLS_ECDHE_RSA_WITH_3DES_EDE_CBC_SHA,
+	"TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA":      tls.TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA,
+	"TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA":      tls.TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA,
+	"TLS_ECDHE_ECDSA_WITH_AES_128_CBC_SHA256": tls.TLS_ECDHE_ECDSA_WITH_AES_128_CBC_SHA256,
+	"TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256":   tls.TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256,
+	"TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256":   tls.TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256,
+	"TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256": tls.TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256,
+	"TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384":   tls.TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384,
+	"TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384": tls.TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384,
+	"TLS_ECDHE_RSA_WITH_CHACHA20_POLY1305":    tls.TLS_ECDHE_RSA_WITH_CHACHA20_POLY1305,
+	"TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305":  tls.TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305,
+}
+
+// GetCipherSuite returns the corresponding cipher suite,
+// and boolean value if it is supported.
+func GetCipherSuite(s string) (uint16, bool) {
+	v, ok := cipherSuites[s]
+	return v, ok
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/tlsutil/doc.go b/vendor/go.etcd.io/etcd/pkg/tlsutil/doc.go
new file mode 100644
index 0000000..3b6aa67
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/tlsutil/doc.go
@@ -0,0 +1,16 @@
+// Copyright 2016 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 tlsutil provides utility functions for handling TLS.
+package tlsutil
diff --git a/vendor/go.etcd.io/etcd/pkg/tlsutil/tlsutil.go b/vendor/go.etcd.io/etcd/pkg/tlsutil/tlsutil.go
new file mode 100644
index 0000000..3a5aef0
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/tlsutil/tlsutil.go
@@ -0,0 +1,73 @@
+// Copyright 2016 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 tlsutil
+
+import (
+	"crypto/tls"
+	"crypto/x509"
+	"encoding/pem"
+	"io/ioutil"
+)
+
+// NewCertPool creates x509 certPool with provided CA files.
+func NewCertPool(CAFiles []string) (*x509.CertPool, error) {
+	certPool := x509.NewCertPool()
+
+	for _, CAFile := range CAFiles {
+		pemByte, err := ioutil.ReadFile(CAFile)
+		if err != nil {
+			return nil, err
+		}
+
+		for {
+			var block *pem.Block
+			block, pemByte = pem.Decode(pemByte)
+			if block == nil {
+				break
+			}
+			cert, err := x509.ParseCertificate(block.Bytes)
+			if err != nil {
+				return nil, err
+			}
+
+			certPool.AddCert(cert)
+		}
+	}
+
+	return certPool, nil
+}
+
+// NewCert generates TLS cert by using the given cert,key and parse function.
+func NewCert(certfile, keyfile string, parseFunc func([]byte, []byte) (tls.Certificate, error)) (*tls.Certificate, error) {
+	cert, err := ioutil.ReadFile(certfile)
+	if err != nil {
+		return nil, err
+	}
+
+	key, err := ioutil.ReadFile(keyfile)
+	if err != nil {
+		return nil, err
+	}
+
+	if parseFunc == nil {
+		parseFunc = tls.X509KeyPair
+	}
+
+	tlsCert, err := parseFunc(cert, key)
+	if err != nil {
+		return nil, err
+	}
+	return &tlsCert, nil
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/transport/doc.go b/vendor/go.etcd.io/etcd/pkg/transport/doc.go
new file mode 100644
index 0000000..37658ce
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/transport/doc.go
@@ -0,0 +1,17 @@
+// Copyright 2015 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 transport implements various HTTP transport utilities based on Go
+// net package.
+package transport
diff --git a/vendor/go.etcd.io/etcd/pkg/transport/keepalive_listener.go b/vendor/go.etcd.io/etcd/pkg/transport/keepalive_listener.go
new file mode 100644
index 0000000..4ff8e7f
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/transport/keepalive_listener.go
@@ -0,0 +1,94 @@
+// Copyright 2015 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 transport
+
+import (
+	"crypto/tls"
+	"fmt"
+	"net"
+	"time"
+)
+
+type keepAliveConn interface {
+	SetKeepAlive(bool) error
+	SetKeepAlivePeriod(d time.Duration) error
+}
+
+// NewKeepAliveListener returns a listener that listens on the given address.
+// Be careful when wrap around KeepAliveListener with another Listener if TLSInfo is not nil.
+// Some pkgs (like go/http) might expect Listener to return TLSConn type to start TLS handshake.
+// http://tldp.org/HOWTO/TCP-Keepalive-HOWTO/overview.html
+func NewKeepAliveListener(l net.Listener, scheme string, tlscfg *tls.Config) (net.Listener, error) {
+	if scheme == "https" {
+		if tlscfg == nil {
+			return nil, fmt.Errorf("cannot listen on TLS for given listener: KeyFile and CertFile are not presented")
+		}
+		return newTLSKeepaliveListener(l, tlscfg), nil
+	}
+
+	return &keepaliveListener{
+		Listener: l,
+	}, nil
+}
+
+type keepaliveListener struct{ net.Listener }
+
+func (kln *keepaliveListener) Accept() (net.Conn, error) {
+	c, err := kln.Listener.Accept()
+	if err != nil {
+		return nil, err
+	}
+	kac := c.(keepAliveConn)
+	// detection time: tcp_keepalive_time + tcp_keepalive_probes + tcp_keepalive_intvl
+	// default on linux:  30 + 8 * 30
+	// default on osx:    30 + 8 * 75
+	kac.SetKeepAlive(true)
+	kac.SetKeepAlivePeriod(30 * time.Second)
+	return c, nil
+}
+
+// A tlsKeepaliveListener implements a network listener (net.Listener) for TLS connections.
+type tlsKeepaliveListener struct {
+	net.Listener
+	config *tls.Config
+}
+
+// Accept waits for and returns the next incoming TLS connection.
+// The returned connection c is a *tls.Conn.
+func (l *tlsKeepaliveListener) Accept() (c net.Conn, err error) {
+	c, err = l.Listener.Accept()
+	if err != nil {
+		return
+	}
+	kac := c.(keepAliveConn)
+	// detection time: tcp_keepalive_time + tcp_keepalive_probes + tcp_keepalive_intvl
+	// default on linux:  30 + 8 * 30
+	// default on osx:    30 + 8 * 75
+	kac.SetKeepAlive(true)
+	kac.SetKeepAlivePeriod(30 * time.Second)
+	c = tls.Server(c, l.config)
+	return c, nil
+}
+
+// NewListener creates a Listener which accepts connections from an inner
+// Listener and wraps each connection with Server.
+// The configuration config must be non-nil and must have
+// at least one certificate.
+func newTLSKeepaliveListener(inner net.Listener, config *tls.Config) net.Listener {
+	l := &tlsKeepaliveListener{}
+	l.Listener = inner
+	l.config = config
+	return l
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/transport/limit_listen.go b/vendor/go.etcd.io/etcd/pkg/transport/limit_listen.go
new file mode 100644
index 0000000..930c542
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/transport/limit_listen.go
@@ -0,0 +1,80 @@
+// Copyright 2013 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 transport provides network utility functions, complementing the more
+// common ones in the net package.
+package transport
+
+import (
+	"errors"
+	"net"
+	"sync"
+	"time"
+)
+
+var (
+	ErrNotTCP = errors.New("only tcp connections have keepalive")
+)
+
+// LimitListener returns a Listener that accepts at most n simultaneous
+// connections from the provided Listener.
+func LimitListener(l net.Listener, n int) net.Listener {
+	return &limitListener{l, make(chan struct{}, n)}
+}
+
+type limitListener struct {
+	net.Listener
+	sem chan struct{}
+}
+
+func (l *limitListener) acquire() { l.sem <- struct{}{} }
+func (l *limitListener) release() { <-l.sem }
+
+func (l *limitListener) Accept() (net.Conn, error) {
+	l.acquire()
+	c, err := l.Listener.Accept()
+	if err != nil {
+		l.release()
+		return nil, err
+	}
+	return &limitListenerConn{Conn: c, release: l.release}, nil
+}
+
+type limitListenerConn struct {
+	net.Conn
+	releaseOnce sync.Once
+	release     func()
+}
+
+func (l *limitListenerConn) Close() error {
+	err := l.Conn.Close()
+	l.releaseOnce.Do(l.release)
+	return err
+}
+
+func (l *limitListenerConn) SetKeepAlive(doKeepAlive bool) error {
+	tcpc, ok := l.Conn.(*net.TCPConn)
+	if !ok {
+		return ErrNotTCP
+	}
+	return tcpc.SetKeepAlive(doKeepAlive)
+}
+
+func (l *limitListenerConn) SetKeepAlivePeriod(d time.Duration) error {
+	tcpc, ok := l.Conn.(*net.TCPConn)
+	if !ok {
+		return ErrNotTCP
+	}
+	return tcpc.SetKeepAlivePeriod(d)
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/transport/listener.go b/vendor/go.etcd.io/etcd/pkg/transport/listener.go
new file mode 100644
index 0000000..fa12911
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/transport/listener.go
@@ -0,0 +1,451 @@
+// Copyright 2015 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 transport
+
+import (
+	"crypto/ecdsa"
+	"crypto/elliptic"
+	"crypto/rand"
+	"crypto/tls"
+	"crypto/x509"
+	"crypto/x509/pkix"
+	"encoding/pem"
+	"errors"
+	"fmt"
+	"math/big"
+	"net"
+	"os"
+	"path/filepath"
+	"strings"
+	"time"
+
+	"go.etcd.io/etcd/pkg/tlsutil"
+
+	"go.uber.org/zap"
+)
+
+// NewListener creates a new listner.
+func NewListener(addr, scheme string, tlsinfo *TLSInfo) (l net.Listener, err error) {
+	if l, err = newListener(addr, scheme); err != nil {
+		return nil, err
+	}
+	return wrapTLS(scheme, tlsinfo, l)
+}
+
+func newListener(addr string, scheme string) (net.Listener, error) {
+	if scheme == "unix" || scheme == "unixs" {
+		// unix sockets via unix://laddr
+		return NewUnixListener(addr)
+	}
+	return net.Listen("tcp", addr)
+}
+
+func wrapTLS(scheme string, tlsinfo *TLSInfo, l net.Listener) (net.Listener, error) {
+	if scheme != "https" && scheme != "unixs" {
+		return l, nil
+	}
+	if tlsinfo != nil && tlsinfo.SkipClientSANVerify {
+		return NewTLSListener(l, tlsinfo)
+	}
+	return newTLSListener(l, tlsinfo, checkSAN)
+}
+
+type TLSInfo struct {
+	CertFile            string
+	KeyFile             string
+	TrustedCAFile       string
+	ClientCertAuth      bool
+	CRLFile             string
+	InsecureSkipVerify  bool
+	SkipClientSANVerify bool
+
+	// ServerName ensures the cert matches the given host in case of discovery / virtual hosting
+	ServerName string
+
+	// HandshakeFailure is optionally called when a connection fails to handshake. The
+	// connection will be closed immediately afterwards.
+	HandshakeFailure func(*tls.Conn, error)
+
+	// CipherSuites is a list of supported cipher suites.
+	// If empty, Go auto-populates it by default.
+	// Note that cipher suites are prioritized in the given order.
+	CipherSuites []uint16
+
+	selfCert bool
+
+	// parseFunc exists to simplify testing. Typically, parseFunc
+	// should be left nil. In that case, tls.X509KeyPair will be used.
+	parseFunc func([]byte, []byte) (tls.Certificate, error)
+
+	// AllowedCN is a CN which must be provided by a client.
+	AllowedCN string
+
+	// AllowedHostname is an IP address or hostname that must match the TLS
+	// certificate provided by a client.
+	AllowedHostname string
+
+	// Logger logs TLS errors.
+	// If nil, all logs are discarded.
+	Logger *zap.Logger
+
+	// EmptyCN indicates that the cert must have empty CN.
+	// If true, ClientConfig() will return an error for a cert with non empty CN.
+	EmptyCN bool
+}
+
+func (info TLSInfo) String() string {
+	return fmt.Sprintf("cert = %s, key = %s, trusted-ca = %s, client-cert-auth = %v, crl-file = %s", info.CertFile, info.KeyFile, info.TrustedCAFile, info.ClientCertAuth, info.CRLFile)
+}
+
+func (info TLSInfo) Empty() bool {
+	return info.CertFile == "" && info.KeyFile == ""
+}
+
+func SelfCert(lg *zap.Logger, dirpath string, hosts []string, additionalUsages ...x509.ExtKeyUsage) (info TLSInfo, err error) {
+	if err = os.MkdirAll(dirpath, 0700); err != nil {
+		return
+	}
+	info.Logger = lg
+
+	certPath := filepath.Join(dirpath, "cert.pem")
+	keyPath := filepath.Join(dirpath, "key.pem")
+	_, errcert := os.Stat(certPath)
+	_, errkey := os.Stat(keyPath)
+	if errcert == nil && errkey == nil {
+		info.CertFile = certPath
+		info.KeyFile = keyPath
+		info.selfCert = true
+		return
+	}
+
+	serialNumberLimit := new(big.Int).Lsh(big.NewInt(1), 128)
+	serialNumber, err := rand.Int(rand.Reader, serialNumberLimit)
+	if err != nil {
+		if info.Logger != nil {
+			info.Logger.Warn(
+				"cannot generate random number",
+				zap.Error(err),
+			)
+		}
+		return
+	}
+
+	tmpl := x509.Certificate{
+		SerialNumber: serialNumber,
+		Subject:      pkix.Name{Organization: []string{"etcd"}},
+		NotBefore:    time.Now(),
+		NotAfter:     time.Now().Add(365 * (24 * time.Hour)),
+
+		KeyUsage:              x509.KeyUsageKeyEncipherment | x509.KeyUsageDigitalSignature,
+		ExtKeyUsage:           append([]x509.ExtKeyUsage{x509.ExtKeyUsageServerAuth}, additionalUsages...),
+		BasicConstraintsValid: true,
+	}
+
+	for _, host := range hosts {
+		h, _, _ := net.SplitHostPort(host)
+		if ip := net.ParseIP(h); ip != nil {
+			tmpl.IPAddresses = append(tmpl.IPAddresses, ip)
+		} else {
+			tmpl.DNSNames = append(tmpl.DNSNames, h)
+		}
+	}
+
+	priv, err := ecdsa.GenerateKey(elliptic.P521(), rand.Reader)
+	if err != nil {
+		if info.Logger != nil {
+			info.Logger.Warn(
+				"cannot generate ECDSA key",
+				zap.Error(err),
+			)
+		}
+		return
+	}
+
+	derBytes, err := x509.CreateCertificate(rand.Reader, &tmpl, &tmpl, &priv.PublicKey, priv)
+	if err != nil {
+		if info.Logger != nil {
+			info.Logger.Warn(
+				"cannot generate x509 certificate",
+				zap.Error(err),
+			)
+		}
+		return
+	}
+
+	certOut, err := os.Create(certPath)
+	if err != nil {
+		info.Logger.Warn(
+			"cannot cert file",
+			zap.String("path", certPath),
+			zap.Error(err),
+		)
+		return
+	}
+	pem.Encode(certOut, &pem.Block{Type: "CERTIFICATE", Bytes: derBytes})
+	certOut.Close()
+	if info.Logger != nil {
+		info.Logger.Info("created cert file", zap.String("path", certPath))
+	}
+
+	b, err := x509.MarshalECPrivateKey(priv)
+	if err != nil {
+		return
+	}
+	keyOut, err := os.OpenFile(keyPath, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600)
+	if err != nil {
+		if info.Logger != nil {
+			info.Logger.Warn(
+				"cannot key file",
+				zap.String("path", keyPath),
+				zap.Error(err),
+			)
+		}
+		return
+	}
+	pem.Encode(keyOut, &pem.Block{Type: "EC PRIVATE KEY", Bytes: b})
+	keyOut.Close()
+	if info.Logger != nil {
+		info.Logger.Info("created key file", zap.String("path", keyPath))
+	}
+	return SelfCert(lg, dirpath, hosts)
+}
+
+// baseConfig is called on initial TLS handshake start.
+//
+// Previously,
+// 1. Server has non-empty (*tls.Config).Certificates on client hello
+// 2. Server calls (*tls.Config).GetCertificate iff:
+//    - Server's (*tls.Config).Certificates is not empty, or
+//    - Client supplies SNI; non-empty (*tls.ClientHelloInfo).ServerName
+//
+// When (*tls.Config).Certificates is always populated on initial handshake,
+// client is expected to provide a valid matching SNI to pass the TLS
+// verification, thus trigger server (*tls.Config).GetCertificate to reload
+// TLS assets. However, a cert whose SAN field does not include domain names
+// but only IP addresses, has empty (*tls.ClientHelloInfo).ServerName, thus
+// it was never able to trigger TLS reload on initial handshake; first
+// ceritifcate object was being used, never being updated.
+//
+// Now, (*tls.Config).Certificates is created empty on initial TLS client
+// handshake, in order to trigger (*tls.Config).GetCertificate and populate
+// rest of the certificates on every new TLS connection, even when client
+// SNI is empty (e.g. cert only includes IPs).
+func (info TLSInfo) baseConfig() (*tls.Config, error) {
+	if info.KeyFile == "" || info.CertFile == "" {
+		return nil, fmt.Errorf("KeyFile and CertFile must both be present[key: %v, cert: %v]", info.KeyFile, info.CertFile)
+	}
+	if info.Logger == nil {
+		info.Logger = zap.NewNop()
+	}
+
+	_, err := tlsutil.NewCert(info.CertFile, info.KeyFile, info.parseFunc)
+	if err != nil {
+		return nil, err
+	}
+
+	cfg := &tls.Config{
+		MinVersion: tls.VersionTLS12,
+		ServerName: info.ServerName,
+	}
+
+	if len(info.CipherSuites) > 0 {
+		cfg.CipherSuites = info.CipherSuites
+	}
+
+	// Client certificates may be verified by either an exact match on the CN,
+	// or a more general check of the CN and SANs.
+	var verifyCertificate func(*x509.Certificate) bool
+	if info.AllowedCN != "" {
+		if info.AllowedHostname != "" {
+			return nil, fmt.Errorf("AllowedCN and AllowedHostname are mutually exclusive (cn=%q, hostname=%q)", info.AllowedCN, info.AllowedHostname)
+		}
+		verifyCertificate = func(cert *x509.Certificate) bool {
+			return info.AllowedCN == cert.Subject.CommonName
+		}
+	}
+	if info.AllowedHostname != "" {
+		verifyCertificate = func(cert *x509.Certificate) bool {
+			return cert.VerifyHostname(info.AllowedHostname) == nil
+		}
+	}
+	if verifyCertificate != nil {
+		cfg.VerifyPeerCertificate = func(rawCerts [][]byte, verifiedChains [][]*x509.Certificate) error {
+			for _, chains := range verifiedChains {
+				if len(chains) != 0 {
+					if verifyCertificate(chains[0]) {
+						return nil
+					}
+				}
+			}
+			return errors.New("client certificate authentication failed")
+		}
+	}
+
+	// this only reloads certs when there's a client request
+	// TODO: support server-side refresh (e.g. inotify, SIGHUP), caching
+	cfg.GetCertificate = func(clientHello *tls.ClientHelloInfo) (cert *tls.Certificate, err error) {
+		cert, err = tlsutil.NewCert(info.CertFile, info.KeyFile, info.parseFunc)
+		if os.IsNotExist(err) {
+			if info.Logger != nil {
+				info.Logger.Warn(
+					"failed to find peer cert files",
+					zap.String("cert-file", info.CertFile),
+					zap.String("key-file", info.KeyFile),
+					zap.Error(err),
+				)
+			}
+		} else if err != nil {
+			if info.Logger != nil {
+				info.Logger.Warn(
+					"failed to create peer certificate",
+					zap.String("cert-file", info.CertFile),
+					zap.String("key-file", info.KeyFile),
+					zap.Error(err),
+				)
+			}
+		}
+		return cert, err
+	}
+	cfg.GetClientCertificate = func(unused *tls.CertificateRequestInfo) (cert *tls.Certificate, err error) {
+		cert, err = tlsutil.NewCert(info.CertFile, info.KeyFile, info.parseFunc)
+		if os.IsNotExist(err) {
+			if info.Logger != nil {
+				info.Logger.Warn(
+					"failed to find client cert files",
+					zap.String("cert-file", info.CertFile),
+					zap.String("key-file", info.KeyFile),
+					zap.Error(err),
+				)
+			}
+		} else if err != nil {
+			if info.Logger != nil {
+				info.Logger.Warn(
+					"failed to create client certificate",
+					zap.String("cert-file", info.CertFile),
+					zap.String("key-file", info.KeyFile),
+					zap.Error(err),
+				)
+			}
+		}
+		return cert, err
+	}
+	return cfg, nil
+}
+
+// cafiles returns a list of CA file paths.
+func (info TLSInfo) cafiles() []string {
+	cs := make([]string, 0)
+	if info.TrustedCAFile != "" {
+		cs = append(cs, info.TrustedCAFile)
+	}
+	return cs
+}
+
+// ServerConfig generates a tls.Config object for use by an HTTP server.
+func (info TLSInfo) ServerConfig() (*tls.Config, error) {
+	cfg, err := info.baseConfig()
+	if err != nil {
+		return nil, err
+	}
+
+	cfg.ClientAuth = tls.NoClientCert
+	if info.TrustedCAFile != "" || info.ClientCertAuth {
+		cfg.ClientAuth = tls.RequireAndVerifyClientCert
+	}
+
+	cs := info.cafiles()
+	if len(cs) > 0 {
+		cp, err := tlsutil.NewCertPool(cs)
+		if err != nil {
+			return nil, err
+		}
+		cfg.ClientCAs = cp
+	}
+
+	// "h2" NextProtos is necessary for enabling HTTP2 for go's HTTP server
+	cfg.NextProtos = []string{"h2"}
+
+	// go1.13 enables TLS 1.3 by default
+	// and in TLS 1.3, cipher suites are not configurable
+	// setting Max TLS version to TLS 1.2 for go 1.13
+	cfg.MaxVersion = tls.VersionTLS12
+
+	return cfg, nil
+}
+
+// ClientConfig generates a tls.Config object for use by an HTTP client.
+func (info TLSInfo) ClientConfig() (*tls.Config, error) {
+	var cfg *tls.Config
+	var err error
+
+	if !info.Empty() {
+		cfg, err = info.baseConfig()
+		if err != nil {
+			return nil, err
+		}
+	} else {
+		cfg = &tls.Config{ServerName: info.ServerName}
+	}
+	cfg.InsecureSkipVerify = info.InsecureSkipVerify
+
+	cs := info.cafiles()
+	if len(cs) > 0 {
+		cfg.RootCAs, err = tlsutil.NewCertPool(cs)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	if info.selfCert {
+		cfg.InsecureSkipVerify = true
+	}
+
+	if info.EmptyCN {
+		hasNonEmptyCN := false
+		cn := ""
+		tlsutil.NewCert(info.CertFile, info.KeyFile, func(certPEMBlock []byte, keyPEMBlock []byte) (tls.Certificate, error) {
+			var block *pem.Block
+			block, _ = pem.Decode(certPEMBlock)
+			cert, err := x509.ParseCertificate(block.Bytes)
+			if err != nil {
+				return tls.Certificate{}, err
+			}
+			if len(cert.Subject.CommonName) != 0 {
+				hasNonEmptyCN = true
+				cn = cert.Subject.CommonName
+			}
+			return tls.X509KeyPair(certPEMBlock, keyPEMBlock)
+		})
+		if hasNonEmptyCN {
+			return nil, fmt.Errorf("cert has non empty Common Name (%s)", cn)
+		}
+	}
+
+	// go1.13 enables TLS 1.3 by default
+	// and in TLS 1.3, cipher suites are not configurable
+	// setting Max TLS version to TLS 1.2 for go 1.13
+	cfg.MaxVersion = tls.VersionTLS12
+
+	return cfg, nil
+}
+
+// IsClosedConnError returns true if the error is from closing listener, cmux.
+// copied from golang.org/x/net/http2/http2.go
+func IsClosedConnError(err error) bool {
+	// 'use of closed network connection' (Go <=1.8)
+	// 'use of closed file or network connection' (Go >1.8, internal/poll.ErrClosing)
+	// 'mux: listener closed' (cmux.ErrListenerClosed)
+	return err != nil && strings.Contains(err.Error(), "closed")
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/transport/listener_tls.go b/vendor/go.etcd.io/etcd/pkg/transport/listener_tls.go
new file mode 100644
index 0000000..6f16009
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/transport/listener_tls.go
@@ -0,0 +1,272 @@
+// 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 transport
+
+import (
+	"context"
+	"crypto/tls"
+	"crypto/x509"
+	"fmt"
+	"io/ioutil"
+	"net"
+	"strings"
+	"sync"
+)
+
+// tlsListener overrides a TLS listener so it will reject client
+// certificates with insufficient SAN credentials or CRL revoked
+// certificates.
+type tlsListener struct {
+	net.Listener
+	connc            chan net.Conn
+	donec            chan struct{}
+	err              error
+	handshakeFailure func(*tls.Conn, error)
+	check            tlsCheckFunc
+}
+
+type tlsCheckFunc func(context.Context, *tls.Conn) error
+
+// NewTLSListener handshakes TLS connections and performs optional CRL checking.
+func NewTLSListener(l net.Listener, tlsinfo *TLSInfo) (net.Listener, error) {
+	check := func(context.Context, *tls.Conn) error { return nil }
+	return newTLSListener(l, tlsinfo, check)
+}
+
+func newTLSListener(l net.Listener, tlsinfo *TLSInfo, check tlsCheckFunc) (net.Listener, error) {
+	if tlsinfo == nil || tlsinfo.Empty() {
+		l.Close()
+		return nil, fmt.Errorf("cannot listen on TLS for %s: KeyFile and CertFile are not presented", l.Addr().String())
+	}
+	tlscfg, err := tlsinfo.ServerConfig()
+	if err != nil {
+		return nil, err
+	}
+
+	hf := tlsinfo.HandshakeFailure
+	if hf == nil {
+		hf = func(*tls.Conn, error) {}
+	}
+
+	if len(tlsinfo.CRLFile) > 0 {
+		prevCheck := check
+		check = func(ctx context.Context, tlsConn *tls.Conn) error {
+			if err := prevCheck(ctx, tlsConn); err != nil {
+				return err
+			}
+			st := tlsConn.ConnectionState()
+			if certs := st.PeerCertificates; len(certs) > 0 {
+				return checkCRL(tlsinfo.CRLFile, certs)
+			}
+			return nil
+		}
+	}
+
+	tlsl := &tlsListener{
+		Listener:         tls.NewListener(l, tlscfg),
+		connc:            make(chan net.Conn),
+		donec:            make(chan struct{}),
+		handshakeFailure: hf,
+		check:            check,
+	}
+	go tlsl.acceptLoop()
+	return tlsl, nil
+}
+
+func (l *tlsListener) Accept() (net.Conn, error) {
+	select {
+	case conn := <-l.connc:
+		return conn, nil
+	case <-l.donec:
+		return nil, l.err
+	}
+}
+
+func checkSAN(ctx context.Context, tlsConn *tls.Conn) error {
+	st := tlsConn.ConnectionState()
+	if certs := st.PeerCertificates; len(certs) > 0 {
+		addr := tlsConn.RemoteAddr().String()
+		return checkCertSAN(ctx, certs[0], addr)
+	}
+	return nil
+}
+
+// acceptLoop launches each TLS handshake in a separate goroutine
+// to prevent a hanging TLS connection from blocking other connections.
+func (l *tlsListener) acceptLoop() {
+	var wg sync.WaitGroup
+	var pendingMu sync.Mutex
+
+	pending := make(map[net.Conn]struct{})
+	ctx, cancel := context.WithCancel(context.Background())
+	defer func() {
+		cancel()
+		pendingMu.Lock()
+		for c := range pending {
+			c.Close()
+		}
+		pendingMu.Unlock()
+		wg.Wait()
+		close(l.donec)
+	}()
+
+	for {
+		conn, err := l.Listener.Accept()
+		if err != nil {
+			l.err = err
+			return
+		}
+
+		pendingMu.Lock()
+		pending[conn] = struct{}{}
+		pendingMu.Unlock()
+
+		wg.Add(1)
+		go func() {
+			defer func() {
+				if conn != nil {
+					conn.Close()
+				}
+				wg.Done()
+			}()
+
+			tlsConn := conn.(*tls.Conn)
+			herr := tlsConn.Handshake()
+			pendingMu.Lock()
+			delete(pending, conn)
+			pendingMu.Unlock()
+
+			if herr != nil {
+				l.handshakeFailure(tlsConn, herr)
+				return
+			}
+			if err := l.check(ctx, tlsConn); err != nil {
+				l.handshakeFailure(tlsConn, err)
+				return
+			}
+
+			select {
+			case l.connc <- tlsConn:
+				conn = nil
+			case <-ctx.Done():
+			}
+		}()
+	}
+}
+
+func checkCRL(crlPath string, cert []*x509.Certificate) error {
+	// TODO: cache
+	crlBytes, err := ioutil.ReadFile(crlPath)
+	if err != nil {
+		return err
+	}
+	certList, err := x509.ParseCRL(crlBytes)
+	if err != nil {
+		return err
+	}
+	revokedSerials := make(map[string]struct{})
+	for _, rc := range certList.TBSCertList.RevokedCertificates {
+		revokedSerials[string(rc.SerialNumber.Bytes())] = struct{}{}
+	}
+	for _, c := range cert {
+		serial := string(c.SerialNumber.Bytes())
+		if _, ok := revokedSerials[serial]; ok {
+			return fmt.Errorf("transport: certificate serial %x revoked", serial)
+		}
+	}
+	return nil
+}
+
+func checkCertSAN(ctx context.Context, cert *x509.Certificate, remoteAddr string) error {
+	if len(cert.IPAddresses) == 0 && len(cert.DNSNames) == 0 {
+		return nil
+	}
+	h, _, herr := net.SplitHostPort(remoteAddr)
+	if herr != nil {
+		return herr
+	}
+	if len(cert.IPAddresses) > 0 {
+		cerr := cert.VerifyHostname(h)
+		if cerr == nil {
+			return nil
+		}
+		if len(cert.DNSNames) == 0 {
+			return cerr
+		}
+	}
+	if len(cert.DNSNames) > 0 {
+		ok, err := isHostInDNS(ctx, h, cert.DNSNames)
+		if ok {
+			return nil
+		}
+		errStr := ""
+		if err != nil {
+			errStr = " (" + err.Error() + ")"
+		}
+		return fmt.Errorf("tls: %q does not match any of DNSNames %q"+errStr, h, cert.DNSNames)
+	}
+	return nil
+}
+
+func isHostInDNS(ctx context.Context, host string, dnsNames []string) (ok bool, err error) {
+	// reverse lookup
+	wildcards, names := []string{}, []string{}
+	for _, dns := range dnsNames {
+		if strings.HasPrefix(dns, "*.") {
+			wildcards = append(wildcards, dns[1:])
+		} else {
+			names = append(names, dns)
+		}
+	}
+	lnames, lerr := net.DefaultResolver.LookupAddr(ctx, host)
+	for _, name := range lnames {
+		// strip trailing '.' from PTR record
+		if name[len(name)-1] == '.' {
+			name = name[:len(name)-1]
+		}
+		for _, wc := range wildcards {
+			if strings.HasSuffix(name, wc) {
+				return true, nil
+			}
+		}
+		for _, n := range names {
+			if n == name {
+				return true, nil
+			}
+		}
+	}
+	err = lerr
+
+	// forward lookup
+	for _, dns := range names {
+		addrs, lerr := net.DefaultResolver.LookupHost(ctx, dns)
+		if lerr != nil {
+			err = lerr
+			continue
+		}
+		for _, addr := range addrs {
+			if addr == host {
+				return true, nil
+			}
+		}
+	}
+	return false, err
+}
+
+func (l *tlsListener) Close() error {
+	err := l.Listener.Close()
+	<-l.donec
+	return err
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/transport/timeout_conn.go b/vendor/go.etcd.io/etcd/pkg/transport/timeout_conn.go
new file mode 100644
index 0000000..7e8c020
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/transport/timeout_conn.go
@@ -0,0 +1,44 @@
+// Copyright 2015 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 transport
+
+import (
+	"net"
+	"time"
+)
+
+type timeoutConn struct {
+	net.Conn
+	wtimeoutd  time.Duration
+	rdtimeoutd time.Duration
+}
+
+func (c timeoutConn) Write(b []byte) (n int, err error) {
+	if c.wtimeoutd > 0 {
+		if err := c.SetWriteDeadline(time.Now().Add(c.wtimeoutd)); err != nil {
+			return 0, err
+		}
+	}
+	return c.Conn.Write(b)
+}
+
+func (c timeoutConn) Read(b []byte) (n int, err error) {
+	if c.rdtimeoutd > 0 {
+		if err := c.SetReadDeadline(time.Now().Add(c.rdtimeoutd)); err != nil {
+			return 0, err
+		}
+	}
+	return c.Conn.Read(b)
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/transport/timeout_dialer.go b/vendor/go.etcd.io/etcd/pkg/transport/timeout_dialer.go
new file mode 100644
index 0000000..6ae39ec
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/transport/timeout_dialer.go
@@ -0,0 +1,36 @@
+// Copyright 2015 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 transport
+
+import (
+	"net"
+	"time"
+)
+
+type rwTimeoutDialer struct {
+	wtimeoutd  time.Duration
+	rdtimeoutd time.Duration
+	net.Dialer
+}
+
+func (d *rwTimeoutDialer) Dial(network, address string) (net.Conn, error) {
+	conn, err := d.Dialer.Dial(network, address)
+	tconn := &timeoutConn{
+		rdtimeoutd: d.rdtimeoutd,
+		wtimeoutd:  d.wtimeoutd,
+		Conn:       conn,
+	}
+	return tconn, err
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/transport/timeout_listener.go b/vendor/go.etcd.io/etcd/pkg/transport/timeout_listener.go
new file mode 100644
index 0000000..273e99f
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/transport/timeout_listener.go
@@ -0,0 +1,57 @@
+// Copyright 2015 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 transport
+
+import (
+	"net"
+	"time"
+)
+
+// NewTimeoutListener returns a listener that listens on the given address.
+// If read/write on the accepted connection blocks longer than its time limit,
+// it will return timeout error.
+func NewTimeoutListener(addr string, scheme string, tlsinfo *TLSInfo, rdtimeoutd, wtimeoutd time.Duration) (net.Listener, error) {
+	ln, err := newListener(addr, scheme)
+	if err != nil {
+		return nil, err
+	}
+	ln = &rwTimeoutListener{
+		Listener:   ln,
+		rdtimeoutd: rdtimeoutd,
+		wtimeoutd:  wtimeoutd,
+	}
+	if ln, err = wrapTLS(scheme, tlsinfo, ln); err != nil {
+		return nil, err
+	}
+	return ln, nil
+}
+
+type rwTimeoutListener struct {
+	net.Listener
+	wtimeoutd  time.Duration
+	rdtimeoutd time.Duration
+}
+
+func (rwln *rwTimeoutListener) Accept() (net.Conn, error) {
+	c, err := rwln.Listener.Accept()
+	if err != nil {
+		return nil, err
+	}
+	return timeoutConn{
+		Conn:       c,
+		wtimeoutd:  rwln.wtimeoutd,
+		rdtimeoutd: rwln.rdtimeoutd,
+	}, nil
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/transport/timeout_transport.go b/vendor/go.etcd.io/etcd/pkg/transport/timeout_transport.go
new file mode 100644
index 0000000..ea16b4c
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/transport/timeout_transport.go
@@ -0,0 +1,51 @@
+// Copyright 2015 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 transport
+
+import (
+	"net"
+	"net/http"
+	"time"
+)
+
+// NewTimeoutTransport returns a transport created using the given TLS info.
+// If read/write on the created connection blocks longer than its time limit,
+// it will return timeout error.
+// If read/write timeout is set, transport will not be able to reuse connection.
+func NewTimeoutTransport(info TLSInfo, dialtimeoutd, rdtimeoutd, wtimeoutd time.Duration) (*http.Transport, error) {
+	tr, err := NewTransport(info, dialtimeoutd)
+	if err != nil {
+		return nil, err
+	}
+
+	if rdtimeoutd != 0 || wtimeoutd != 0 {
+		// the timed out connection will timeout soon after it is idle.
+		// it should not be put back to http transport as an idle connection for future usage.
+		tr.MaxIdleConnsPerHost = -1
+	} else {
+		// allow more idle connections between peers to avoid unnecessary port allocation.
+		tr.MaxIdleConnsPerHost = 1024
+	}
+
+	tr.Dial = (&rwTimeoutDialer{
+		Dialer: net.Dialer{
+			Timeout:   dialtimeoutd,
+			KeepAlive: 30 * time.Second,
+		},
+		rdtimeoutd: rdtimeoutd,
+		wtimeoutd:  wtimeoutd,
+	}).Dial
+	return tr, nil
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/transport/tls.go b/vendor/go.etcd.io/etcd/pkg/transport/tls.go
new file mode 100644
index 0000000..62fe0d3
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/transport/tls.go
@@ -0,0 +1,49 @@
+// Copyright 2016 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 transport
+
+import (
+	"fmt"
+	"strings"
+	"time"
+)
+
+// ValidateSecureEndpoints scans the given endpoints against tls info, returning only those
+// endpoints that could be validated as secure.
+func ValidateSecureEndpoints(tlsInfo TLSInfo, eps []string) ([]string, error) {
+	t, err := NewTransport(tlsInfo, 5*time.Second)
+	if err != nil {
+		return nil, err
+	}
+	var errs []string
+	var endpoints []string
+	for _, ep := range eps {
+		if !strings.HasPrefix(ep, "https://") {
+			errs = append(errs, fmt.Sprintf("%q is insecure", ep))
+			continue
+		}
+		conn, cerr := t.Dial("tcp", ep[len("https://"):])
+		if cerr != nil {
+			errs = append(errs, fmt.Sprintf("%q failed to dial (%v)", ep, cerr))
+			continue
+		}
+		conn.Close()
+		endpoints = append(endpoints, ep)
+	}
+	if len(errs) != 0 {
+		err = fmt.Errorf("%s", strings.Join(errs, ","))
+	}
+	return endpoints, err
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/transport/transport.go b/vendor/go.etcd.io/etcd/pkg/transport/transport.go
new file mode 100644
index 0000000..4a7fe69
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/transport/transport.go
@@ -0,0 +1,71 @@
+// Copyright 2016 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 transport
+
+import (
+	"net"
+	"net/http"
+	"strings"
+	"time"
+)
+
+type unixTransport struct{ *http.Transport }
+
+func NewTransport(info TLSInfo, dialtimeoutd time.Duration) (*http.Transport, error) {
+	cfg, err := info.ClientConfig()
+	if err != nil {
+		return nil, err
+	}
+
+	t := &http.Transport{
+		Proxy: http.ProxyFromEnvironment,
+		Dial: (&net.Dialer{
+			Timeout: dialtimeoutd,
+			// value taken from http.DefaultTransport
+			KeepAlive: 30 * time.Second,
+		}).Dial,
+		// value taken from http.DefaultTransport
+		TLSHandshakeTimeout: 10 * time.Second,
+		TLSClientConfig:     cfg,
+	}
+
+	dialer := (&net.Dialer{
+		Timeout:   dialtimeoutd,
+		KeepAlive: 30 * time.Second,
+	})
+	dial := func(net, addr string) (net.Conn, error) {
+		return dialer.Dial("unix", addr)
+	}
+
+	tu := &http.Transport{
+		Proxy:               http.ProxyFromEnvironment,
+		Dial:                dial,
+		TLSHandshakeTimeout: 10 * time.Second,
+		TLSClientConfig:     cfg,
+	}
+	ut := &unixTransport{tu}
+
+	t.RegisterProtocol("unix", ut)
+	t.RegisterProtocol("unixs", ut)
+
+	return t, nil
+}
+
+func (urt *unixTransport) RoundTrip(req *http.Request) (*http.Response, error) {
+	url := *req.URL
+	req.URL = &url
+	req.URL.Scheme = strings.Replace(req.URL.Scheme, "unix", "http", 1)
+	return urt.Transport.RoundTrip(req)
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/transport/unix_listener.go b/vendor/go.etcd.io/etcd/pkg/transport/unix_listener.go
new file mode 100644
index 0000000..123e203
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/transport/unix_listener.go
@@ -0,0 +1,40 @@
+// Copyright 2016 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 transport
+
+import (
+	"net"
+	"os"
+)
+
+type unixListener struct{ net.Listener }
+
+func NewUnixListener(addr string) (net.Listener, error) {
+	if err := os.Remove(addr); err != nil && !os.IsNotExist(err) {
+		return nil, err
+	}
+	l, err := net.Listen("unix", addr)
+	if err != nil {
+		return nil, err
+	}
+	return &unixListener{l}, nil
+}
+
+func (ul *unixListener) Close() error {
+	if err := os.Remove(ul.Addr().String()); err != nil && !os.IsNotExist(err) {
+		return err
+	}
+	return ul.Listener.Close()
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/wait/wait.go b/vendor/go.etcd.io/etcd/pkg/wait/wait.go
new file mode 100644
index 0000000..9b1df41
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/wait/wait.go
@@ -0,0 +1,91 @@
+// Copyright 2015 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 wait provides utility functions for polling, listening using Go
+// channel.
+package wait
+
+import (
+	"log"
+	"sync"
+)
+
+// Wait is an interface that provides the ability to wait and trigger events that
+// are associated with IDs.
+type Wait interface {
+	// Register waits returns a chan that waits on the given ID.
+	// The chan will be triggered when Trigger is called with
+	// the same ID.
+	Register(id uint64) <-chan interface{}
+	// Trigger triggers the waiting chans with the given ID.
+	Trigger(id uint64, x interface{})
+	IsRegistered(id uint64) bool
+}
+
+type list struct {
+	l sync.RWMutex
+	m map[uint64]chan interface{}
+}
+
+// New creates a Wait.
+func New() Wait {
+	return &list{m: make(map[uint64]chan interface{})}
+}
+
+func (w *list) Register(id uint64) <-chan interface{} {
+	w.l.Lock()
+	defer w.l.Unlock()
+	ch := w.m[id]
+	if ch == nil {
+		ch = make(chan interface{}, 1)
+		w.m[id] = ch
+	} else {
+		log.Panicf("dup id %x", id)
+	}
+	return ch
+}
+
+func (w *list) Trigger(id uint64, x interface{}) {
+	w.l.Lock()
+	ch := w.m[id]
+	delete(w.m, id)
+	w.l.Unlock()
+	if ch != nil {
+		ch <- x
+		close(ch)
+	}
+}
+
+func (w *list) IsRegistered(id uint64) bool {
+	w.l.RLock()
+	defer w.l.RUnlock()
+	_, ok := w.m[id]
+	return ok
+}
+
+type waitWithResponse struct {
+	ch <-chan interface{}
+}
+
+func NewWithResponse(ch <-chan interface{}) Wait {
+	return &waitWithResponse{ch: ch}
+}
+
+func (w *waitWithResponse) Register(id uint64) <-chan interface{} {
+	return w.ch
+}
+func (w *waitWithResponse) Trigger(id uint64, x interface{}) {}
+func (w *waitWithResponse) IsRegistered(id uint64) bool {
+	panic("waitWithResponse.IsRegistered() shouldn't be called")
+}
diff --git a/vendor/go.etcd.io/etcd/pkg/wait/wait_time.go b/vendor/go.etcd.io/etcd/pkg/wait/wait_time.go
new file mode 100644
index 0000000..297e48a
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/pkg/wait/wait_time.go
@@ -0,0 +1,66 @@
+// Copyright 2015 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 wait
+
+import "sync"
+
+type WaitTime interface {
+	// Wait returns a chan that waits on the given logical deadline.
+	// The chan will be triggered when Trigger is called with a
+	// deadline that is later than the one it is waiting for.
+	Wait(deadline uint64) <-chan struct{}
+	// Trigger triggers all the waiting chans with an earlier logical deadline.
+	Trigger(deadline uint64)
+}
+
+var closec chan struct{}
+
+func init() { closec = make(chan struct{}); close(closec) }
+
+type timeList struct {
+	l                   sync.Mutex
+	lastTriggerDeadline uint64
+	m                   map[uint64]chan struct{}
+}
+
+func NewTimeList() *timeList {
+	return &timeList{m: make(map[uint64]chan struct{})}
+}
+
+func (tl *timeList) Wait(deadline uint64) <-chan struct{} {
+	tl.l.Lock()
+	defer tl.l.Unlock()
+	if tl.lastTriggerDeadline >= deadline {
+		return closec
+	}
+	ch := tl.m[deadline]
+	if ch == nil {
+		ch = make(chan struct{})
+		tl.m[deadline] = ch
+	}
+	return ch
+}
+
+func (tl *timeList) Trigger(deadline uint64) {
+	tl.l.Lock()
+	defer tl.l.Unlock()
+	tl.lastTriggerDeadline = deadline
+	for t, ch := range tl.m {
+		if t <= deadline {
+			delete(tl.m, t)
+			close(ch)
+		}
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/auth_client_adapter.go b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/auth_client_adapter.go
new file mode 100644
index 0000000..59dbe6b
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/auth_client_adapter.go
@@ -0,0 +1,93 @@
+// 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 adapter
+
+import (
+	"context"
+
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+
+	grpc "google.golang.org/grpc"
+)
+
+type as2ac struct{ as pb.AuthServer }
+
+func AuthServerToAuthClient(as pb.AuthServer) pb.AuthClient {
+	return &as2ac{as}
+}
+
+func (s *as2ac) AuthEnable(ctx context.Context, in *pb.AuthEnableRequest, opts ...grpc.CallOption) (*pb.AuthEnableResponse, error) {
+	return s.as.AuthEnable(ctx, in)
+}
+
+func (s *as2ac) AuthDisable(ctx context.Context, in *pb.AuthDisableRequest, opts ...grpc.CallOption) (*pb.AuthDisableResponse, error) {
+	return s.as.AuthDisable(ctx, in)
+}
+
+func (s *as2ac) Authenticate(ctx context.Context, in *pb.AuthenticateRequest, opts ...grpc.CallOption) (*pb.AuthenticateResponse, error) {
+	return s.as.Authenticate(ctx, in)
+}
+
+func (s *as2ac) RoleAdd(ctx context.Context, in *pb.AuthRoleAddRequest, opts ...grpc.CallOption) (*pb.AuthRoleAddResponse, error) {
+	return s.as.RoleAdd(ctx, in)
+}
+
+func (s *as2ac) RoleDelete(ctx context.Context, in *pb.AuthRoleDeleteRequest, opts ...grpc.CallOption) (*pb.AuthRoleDeleteResponse, error) {
+	return s.as.RoleDelete(ctx, in)
+}
+
+func (s *as2ac) RoleGet(ctx context.Context, in *pb.AuthRoleGetRequest, opts ...grpc.CallOption) (*pb.AuthRoleGetResponse, error) {
+	return s.as.RoleGet(ctx, in)
+}
+
+func (s *as2ac) RoleList(ctx context.Context, in *pb.AuthRoleListRequest, opts ...grpc.CallOption) (*pb.AuthRoleListResponse, error) {
+	return s.as.RoleList(ctx, in)
+}
+
+func (s *as2ac) RoleRevokePermission(ctx context.Context, in *pb.AuthRoleRevokePermissionRequest, opts ...grpc.CallOption) (*pb.AuthRoleRevokePermissionResponse, error) {
+	return s.as.RoleRevokePermission(ctx, in)
+}
+
+func (s *as2ac) RoleGrantPermission(ctx context.Context, in *pb.AuthRoleGrantPermissionRequest, opts ...grpc.CallOption) (*pb.AuthRoleGrantPermissionResponse, error) {
+	return s.as.RoleGrantPermission(ctx, in)
+}
+
+func (s *as2ac) UserDelete(ctx context.Context, in *pb.AuthUserDeleteRequest, opts ...grpc.CallOption) (*pb.AuthUserDeleteResponse, error) {
+	return s.as.UserDelete(ctx, in)
+}
+
+func (s *as2ac) UserAdd(ctx context.Context, in *pb.AuthUserAddRequest, opts ...grpc.CallOption) (*pb.AuthUserAddResponse, error) {
+	return s.as.UserAdd(ctx, in)
+}
+
+func (s *as2ac) UserGet(ctx context.Context, in *pb.AuthUserGetRequest, opts ...grpc.CallOption) (*pb.AuthUserGetResponse, error) {
+	return s.as.UserGet(ctx, in)
+}
+
+func (s *as2ac) UserList(ctx context.Context, in *pb.AuthUserListRequest, opts ...grpc.CallOption) (*pb.AuthUserListResponse, error) {
+	return s.as.UserList(ctx, in)
+}
+
+func (s *as2ac) UserGrantRole(ctx context.Context, in *pb.AuthUserGrantRoleRequest, opts ...grpc.CallOption) (*pb.AuthUserGrantRoleResponse, error) {
+	return s.as.UserGrantRole(ctx, in)
+}
+
+func (s *as2ac) UserRevokeRole(ctx context.Context, in *pb.AuthUserRevokeRoleRequest, opts ...grpc.CallOption) (*pb.AuthUserRevokeRoleResponse, error) {
+	return s.as.UserRevokeRole(ctx, in)
+}
+
+func (s *as2ac) UserChangePassword(ctx context.Context, in *pb.AuthUserChangePasswordRequest, opts ...grpc.CallOption) (*pb.AuthUserChangePasswordResponse, error) {
+	return s.as.UserChangePassword(ctx, in)
+}
diff --git a/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/chan_stream.go b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/chan_stream.go
new file mode 100644
index 0000000..1af514b
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/chan_stream.go
@@ -0,0 +1,167 @@
+// 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 adapter
+
+import (
+	"context"
+
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/metadata"
+	"google.golang.org/grpc/status"
+)
+
+// chanServerStream implements grpc.ServerStream with a chanStream
+type chanServerStream struct {
+	headerc  chan<- metadata.MD
+	trailerc chan<- metadata.MD
+	grpc.Stream
+
+	headers []metadata.MD
+}
+
+func (ss *chanServerStream) SendHeader(md metadata.MD) error {
+	if ss.headerc == nil {
+		return errAlreadySentHeader
+	}
+	outmd := make(map[string][]string)
+	for _, h := range append(ss.headers, md) {
+		for k, v := range h {
+			outmd[k] = v
+		}
+	}
+	select {
+	case ss.headerc <- outmd:
+		ss.headerc = nil
+		ss.headers = nil
+		return nil
+	case <-ss.Context().Done():
+	}
+	return ss.Context().Err()
+}
+
+func (ss *chanServerStream) SetHeader(md metadata.MD) error {
+	if ss.headerc == nil {
+		return errAlreadySentHeader
+	}
+	ss.headers = append(ss.headers, md)
+	return nil
+}
+
+func (ss *chanServerStream) SetTrailer(md metadata.MD) {
+	ss.trailerc <- md
+}
+
+// chanClientStream implements grpc.ClientStream with a chanStream
+type chanClientStream struct {
+	headerc  <-chan metadata.MD
+	trailerc <-chan metadata.MD
+	*chanStream
+}
+
+func (cs *chanClientStream) Header() (metadata.MD, error) {
+	select {
+	case md := <-cs.headerc:
+		return md, nil
+	case <-cs.Context().Done():
+	}
+	return nil, cs.Context().Err()
+}
+
+func (cs *chanClientStream) Trailer() metadata.MD {
+	select {
+	case md := <-cs.trailerc:
+		return md
+	case <-cs.Context().Done():
+		return nil
+	}
+}
+
+func (cs *chanClientStream) CloseSend() error {
+	close(cs.chanStream.sendc)
+	return nil
+}
+
+// chanStream implements grpc.Stream using channels
+type chanStream struct {
+	recvc  <-chan interface{}
+	sendc  chan<- interface{}
+	ctx    context.Context
+	cancel context.CancelFunc
+}
+
+func (s *chanStream) Context() context.Context { return s.ctx }
+
+func (s *chanStream) SendMsg(m interface{}) error {
+	select {
+	case s.sendc <- m:
+		if err, ok := m.(error); ok {
+			return err
+		}
+		return nil
+	case <-s.ctx.Done():
+	}
+	return s.ctx.Err()
+}
+
+func (s *chanStream) RecvMsg(m interface{}) error {
+	v := m.(*interface{})
+	for {
+		select {
+		case msg, ok := <-s.recvc:
+			if !ok {
+				return status.Error(codes.Canceled, "the client connection is closing")
+			}
+			if err, ok := msg.(error); ok {
+				return err
+			}
+			*v = msg
+			return nil
+		case <-s.ctx.Done():
+		}
+		if len(s.recvc) == 0 {
+			// prioritize any pending recv messages over canceled context
+			break
+		}
+	}
+	return s.ctx.Err()
+}
+
+func newPipeStream(ctx context.Context, ssHandler func(chanServerStream) error) chanClientStream {
+	// ch1 is buffered so server can send error on close
+	ch1, ch2 := make(chan interface{}, 1), make(chan interface{})
+	headerc, trailerc := make(chan metadata.MD, 1), make(chan metadata.MD, 1)
+
+	cctx, ccancel := context.WithCancel(ctx)
+	cli := &chanStream{recvc: ch1, sendc: ch2, ctx: cctx, cancel: ccancel}
+	cs := chanClientStream{headerc, trailerc, cli}
+
+	sctx, scancel := context.WithCancel(ctx)
+	srv := &chanStream{recvc: ch2, sendc: ch1, ctx: sctx, cancel: scancel}
+	ss := chanServerStream{headerc, trailerc, srv, nil}
+
+	go func() {
+		if err := ssHandler(ss); err != nil {
+			select {
+			case srv.sendc <- err:
+			case <-sctx.Done():
+			case <-cctx.Done():
+			}
+		}
+		scancel()
+		ccancel()
+	}()
+	return cs
+}
diff --git a/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/cluster_client_adapter.go b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/cluster_client_adapter.go
new file mode 100644
index 0000000..73a6fdf
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/cluster_client_adapter.go
@@ -0,0 +1,49 @@
+// 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 adapter
+
+import (
+	"context"
+
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+
+	"google.golang.org/grpc"
+)
+
+type cls2clc struct{ cls pb.ClusterServer }
+
+func ClusterServerToClusterClient(cls pb.ClusterServer) pb.ClusterClient {
+	return &cls2clc{cls}
+}
+
+func (s *cls2clc) MemberList(ctx context.Context, r *pb.MemberListRequest, opts ...grpc.CallOption) (*pb.MemberListResponse, error) {
+	return s.cls.MemberList(ctx, r)
+}
+
+func (s *cls2clc) MemberAdd(ctx context.Context, r *pb.MemberAddRequest, opts ...grpc.CallOption) (*pb.MemberAddResponse, error) {
+	return s.cls.MemberAdd(ctx, r)
+}
+
+func (s *cls2clc) MemberUpdate(ctx context.Context, r *pb.MemberUpdateRequest, opts ...grpc.CallOption) (*pb.MemberUpdateResponse, error) {
+	return s.cls.MemberUpdate(ctx, r)
+}
+
+func (s *cls2clc) MemberRemove(ctx context.Context, r *pb.MemberRemoveRequest, opts ...grpc.CallOption) (*pb.MemberRemoveResponse, error) {
+	return s.cls.MemberRemove(ctx, r)
+}
+
+func (s *cls2clc) MemberPromote(ctx context.Context, r *pb.MemberPromoteRequest, opts ...grpc.CallOption) (*pb.MemberPromoteResponse, error) {
+	return s.cls.MemberPromote(ctx, r)
+}
diff --git a/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/doc.go b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/doc.go
new file mode 100644
index 0000000..7170be2
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/doc.go
@@ -0,0 +1,17 @@
+// 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 adapter provides gRPC adapters between client and server
+// gRPC interfaces without needing to go through a gRPC connection.
+package adapter
diff --git a/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/election_client_adapter.go b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/election_client_adapter.go
new file mode 100644
index 0000000..4722be0
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/election_client_adapter.go
@@ -0,0 +1,80 @@
+// 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 adapter
+
+import (
+	"context"
+
+	"go.etcd.io/etcd/etcdserver/api/v3election/v3electionpb"
+
+	"google.golang.org/grpc"
+)
+
+type es2ec struct{ es v3electionpb.ElectionServer }
+
+func ElectionServerToElectionClient(es v3electionpb.ElectionServer) v3electionpb.ElectionClient {
+	return &es2ec{es}
+}
+
+func (s *es2ec) Campaign(ctx context.Context, r *v3electionpb.CampaignRequest, opts ...grpc.CallOption) (*v3electionpb.CampaignResponse, error) {
+	return s.es.Campaign(ctx, r)
+}
+
+func (s *es2ec) Proclaim(ctx context.Context, r *v3electionpb.ProclaimRequest, opts ...grpc.CallOption) (*v3electionpb.ProclaimResponse, error) {
+	return s.es.Proclaim(ctx, r)
+}
+
+func (s *es2ec) Leader(ctx context.Context, r *v3electionpb.LeaderRequest, opts ...grpc.CallOption) (*v3electionpb.LeaderResponse, error) {
+	return s.es.Leader(ctx, r)
+}
+
+func (s *es2ec) Resign(ctx context.Context, r *v3electionpb.ResignRequest, opts ...grpc.CallOption) (*v3electionpb.ResignResponse, error) {
+	return s.es.Resign(ctx, r)
+}
+
+func (s *es2ec) Observe(ctx context.Context, in *v3electionpb.LeaderRequest, opts ...grpc.CallOption) (v3electionpb.Election_ObserveClient, error) {
+	cs := newPipeStream(ctx, func(ss chanServerStream) error {
+		return s.es.Observe(in, &es2ecServerStream{ss})
+	})
+	return &es2ecClientStream{cs}, nil
+}
+
+// es2ecClientStream implements Election_ObserveClient
+type es2ecClientStream struct{ chanClientStream }
+
+// es2ecServerStream implements Election_ObserveServer
+type es2ecServerStream struct{ chanServerStream }
+
+func (s *es2ecClientStream) Send(rr *v3electionpb.LeaderRequest) error {
+	return s.SendMsg(rr)
+}
+func (s *es2ecClientStream) Recv() (*v3electionpb.LeaderResponse, error) {
+	var v interface{}
+	if err := s.RecvMsg(&v); err != nil {
+		return nil, err
+	}
+	return v.(*v3electionpb.LeaderResponse), nil
+}
+
+func (s *es2ecServerStream) Send(rr *v3electionpb.LeaderResponse) error {
+	return s.SendMsg(rr)
+}
+func (s *es2ecServerStream) Recv() (*v3electionpb.LeaderRequest, error) {
+	var v interface{}
+	if err := s.RecvMsg(&v); err != nil {
+		return nil, err
+	}
+	return v.(*v3electionpb.LeaderRequest), nil
+}
diff --git a/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/kv_client_adapter.go b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/kv_client_adapter.go
new file mode 100644
index 0000000..b1a7820
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/kv_client_adapter.go
@@ -0,0 +1,49 @@
+// Copyright 2016 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 adapter
+
+import (
+	"context"
+
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+
+	grpc "google.golang.org/grpc"
+)
+
+type kvs2kvc struct{ kvs pb.KVServer }
+
+func KvServerToKvClient(kvs pb.KVServer) pb.KVClient {
+	return &kvs2kvc{kvs}
+}
+
+func (s *kvs2kvc) Range(ctx context.Context, in *pb.RangeRequest, opts ...grpc.CallOption) (*pb.RangeResponse, error) {
+	return s.kvs.Range(ctx, in)
+}
+
+func (s *kvs2kvc) Put(ctx context.Context, in *pb.PutRequest, opts ...grpc.CallOption) (*pb.PutResponse, error) {
+	return s.kvs.Put(ctx, in)
+}
+
+func (s *kvs2kvc) DeleteRange(ctx context.Context, in *pb.DeleteRangeRequest, opts ...grpc.CallOption) (*pb.DeleteRangeResponse, error) {
+	return s.kvs.DeleteRange(ctx, in)
+}
+
+func (s *kvs2kvc) Txn(ctx context.Context, in *pb.TxnRequest, opts ...grpc.CallOption) (*pb.TxnResponse, error) {
+	return s.kvs.Txn(ctx, in)
+}
+
+func (s *kvs2kvc) Compact(ctx context.Context, in *pb.CompactionRequest, opts ...grpc.CallOption) (*pb.CompactionResponse, error) {
+	return s.kvs.Compact(ctx, in)
+}
diff --git a/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/lease_client_adapter.go b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/lease_client_adapter.go
new file mode 100644
index 0000000..a58408f
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/lease_client_adapter.go
@@ -0,0 +1,82 @@
+// 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 adapter
+
+import (
+	"context"
+
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+
+	"google.golang.org/grpc"
+)
+
+type ls2lc struct {
+	leaseServer pb.LeaseServer
+}
+
+func LeaseServerToLeaseClient(ls pb.LeaseServer) pb.LeaseClient {
+	return &ls2lc{ls}
+}
+
+func (c *ls2lc) LeaseGrant(ctx context.Context, in *pb.LeaseGrantRequest, opts ...grpc.CallOption) (*pb.LeaseGrantResponse, error) {
+	return c.leaseServer.LeaseGrant(ctx, in)
+}
+
+func (c *ls2lc) LeaseRevoke(ctx context.Context, in *pb.LeaseRevokeRequest, opts ...grpc.CallOption) (*pb.LeaseRevokeResponse, error) {
+	return c.leaseServer.LeaseRevoke(ctx, in)
+}
+
+func (c *ls2lc) LeaseKeepAlive(ctx context.Context, opts ...grpc.CallOption) (pb.Lease_LeaseKeepAliveClient, error) {
+	cs := newPipeStream(ctx, func(ss chanServerStream) error {
+		return c.leaseServer.LeaseKeepAlive(&ls2lcServerStream{ss})
+	})
+	return &ls2lcClientStream{cs}, nil
+}
+
+func (c *ls2lc) LeaseTimeToLive(ctx context.Context, in *pb.LeaseTimeToLiveRequest, opts ...grpc.CallOption) (*pb.LeaseTimeToLiveResponse, error) {
+	return c.leaseServer.LeaseTimeToLive(ctx, in)
+}
+
+func (c *ls2lc) LeaseLeases(ctx context.Context, in *pb.LeaseLeasesRequest, opts ...grpc.CallOption) (*pb.LeaseLeasesResponse, error) {
+	return c.leaseServer.LeaseLeases(ctx, in)
+}
+
+// ls2lcClientStream implements Lease_LeaseKeepAliveClient
+type ls2lcClientStream struct{ chanClientStream }
+
+// ls2lcServerStream implements Lease_LeaseKeepAliveServer
+type ls2lcServerStream struct{ chanServerStream }
+
+func (s *ls2lcClientStream) Send(rr *pb.LeaseKeepAliveRequest) error {
+	return s.SendMsg(rr)
+}
+func (s *ls2lcClientStream) Recv() (*pb.LeaseKeepAliveResponse, error) {
+	var v interface{}
+	if err := s.RecvMsg(&v); err != nil {
+		return nil, err
+	}
+	return v.(*pb.LeaseKeepAliveResponse), nil
+}
+
+func (s *ls2lcServerStream) Send(rr *pb.LeaseKeepAliveResponse) error {
+	return s.SendMsg(rr)
+}
+func (s *ls2lcServerStream) Recv() (*pb.LeaseKeepAliveRequest, error) {
+	var v interface{}
+	if err := s.RecvMsg(&v); err != nil {
+		return nil, err
+	}
+	return v.(*pb.LeaseKeepAliveRequest), nil
+}
diff --git a/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/lock_client_adapter.go b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/lock_client_adapter.go
new file mode 100644
index 0000000..65b5641
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/lock_client_adapter.go
@@ -0,0 +1,37 @@
+// 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 adapter
+
+import (
+	"context"
+
+	"go.etcd.io/etcd/etcdserver/api/v3lock/v3lockpb"
+
+	"google.golang.org/grpc"
+)
+
+type ls2lsc struct{ ls v3lockpb.LockServer }
+
+func LockServerToLockClient(ls v3lockpb.LockServer) v3lockpb.LockClient {
+	return &ls2lsc{ls}
+}
+
+func (s *ls2lsc) Lock(ctx context.Context, r *v3lockpb.LockRequest, opts ...grpc.CallOption) (*v3lockpb.LockResponse, error) {
+	return s.ls.Lock(ctx, r)
+}
+
+func (s *ls2lsc) Unlock(ctx context.Context, r *v3lockpb.UnlockRequest, opts ...grpc.CallOption) (*v3lockpb.UnlockResponse, error) {
+	return s.ls.Unlock(ctx, r)
+}
diff --git a/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/maintenance_client_adapter.go b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/maintenance_client_adapter.go
new file mode 100644
index 0000000..4a8781b
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/maintenance_client_adapter.go
@@ -0,0 +1,88 @@
+// 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 adapter
+
+import (
+	"context"
+
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+
+	"google.golang.org/grpc"
+)
+
+type mts2mtc struct{ mts pb.MaintenanceServer }
+
+func MaintenanceServerToMaintenanceClient(mts pb.MaintenanceServer) pb.MaintenanceClient {
+	return &mts2mtc{mts}
+}
+
+func (s *mts2mtc) Alarm(ctx context.Context, r *pb.AlarmRequest, opts ...grpc.CallOption) (*pb.AlarmResponse, error) {
+	return s.mts.Alarm(ctx, r)
+}
+
+func (s *mts2mtc) Status(ctx context.Context, r *pb.StatusRequest, opts ...grpc.CallOption) (*pb.StatusResponse, error) {
+	return s.mts.Status(ctx, r)
+}
+
+func (s *mts2mtc) Defragment(ctx context.Context, dr *pb.DefragmentRequest, opts ...grpc.CallOption) (*pb.DefragmentResponse, error) {
+	return s.mts.Defragment(ctx, dr)
+}
+
+func (s *mts2mtc) Hash(ctx context.Context, r *pb.HashRequest, opts ...grpc.CallOption) (*pb.HashResponse, error) {
+	return s.mts.Hash(ctx, r)
+}
+
+func (s *mts2mtc) HashKV(ctx context.Context, r *pb.HashKVRequest, opts ...grpc.CallOption) (*pb.HashKVResponse, error) {
+	return s.mts.HashKV(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})
+	})
+	return &ss2scClientStream{cs}, nil
+}
+
+// ss2scClientStream implements Maintenance_SnapshotClient
+type ss2scClientStream struct{ chanClientStream }
+
+// ss2scServerStream implements Maintenance_SnapshotServer
+type ss2scServerStream struct{ chanServerStream }
+
+func (s *ss2scClientStream) Send(rr *pb.SnapshotRequest) error {
+	return s.SendMsg(rr)
+}
+func (s *ss2scClientStream) Recv() (*pb.SnapshotResponse, error) {
+	var v interface{}
+	if err := s.RecvMsg(&v); err != nil {
+		return nil, err
+	}
+	return v.(*pb.SnapshotResponse), nil
+}
+
+func (s *ss2scServerStream) Send(rr *pb.SnapshotResponse) error {
+	return s.SendMsg(rr)
+}
+func (s *ss2scServerStream) Recv() (*pb.SnapshotRequest, error) {
+	var v interface{}
+	if err := s.RecvMsg(&v); err != nil {
+		return nil, err
+	}
+	return v.(*pb.SnapshotRequest), nil
+}
diff --git a/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/watch_client_adapter.go b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/watch_client_adapter.go
new file mode 100644
index 0000000..2f629cc
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/proxy/grpcproxy/adapter/watch_client_adapter.go
@@ -0,0 +1,66 @@
+// Copyright 2016 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 adapter
+
+import (
+	"context"
+	"errors"
+
+	pb "go.etcd.io/etcd/etcdserver/etcdserverpb"
+	"google.golang.org/grpc"
+)
+
+var errAlreadySentHeader = errors.New("adapter: already sent header")
+
+type ws2wc struct{ wserv pb.WatchServer }
+
+func WatchServerToWatchClient(wserv pb.WatchServer) pb.WatchClient {
+	return &ws2wc{wserv}
+}
+
+func (s *ws2wc) Watch(ctx context.Context, opts ...grpc.CallOption) (pb.Watch_WatchClient, error) {
+	cs := newPipeStream(ctx, func(ss chanServerStream) error {
+		return s.wserv.Watch(&ws2wcServerStream{ss})
+	})
+	return &ws2wcClientStream{cs}, nil
+}
+
+// ws2wcClientStream implements Watch_WatchClient
+type ws2wcClientStream struct{ chanClientStream }
+
+// ws2wcServerStream implements Watch_WatchServer
+type ws2wcServerStream struct{ chanServerStream }
+
+func (s *ws2wcClientStream) Send(wr *pb.WatchRequest) error {
+	return s.SendMsg(wr)
+}
+func (s *ws2wcClientStream) Recv() (*pb.WatchResponse, error) {
+	var v interface{}
+	if err := s.RecvMsg(&v); err != nil {
+		return nil, err
+	}
+	return v.(*pb.WatchResponse), nil
+}
+
+func (s *ws2wcServerStream) Send(wr *pb.WatchResponse) error {
+	return s.SendMsg(wr)
+}
+func (s *ws2wcServerStream) Recv() (*pb.WatchRequest, error) {
+	var v interface{}
+	if err := s.RecvMsg(&v); err != nil {
+		return nil, err
+	}
+	return v.(*pb.WatchRequest), nil
+}
diff --git a/vendor/go.etcd.io/etcd/version/version.go b/vendor/go.etcd.io/etcd/version/version.go
new file mode 100644
index 0000000..622e751
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/version/version.go
@@ -0,0 +1,56 @@
+// Copyright 2015 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 version implements etcd version parsing and contains latest version
+// information.
+package version
+
+import (
+	"fmt"
+	"strings"
+
+	"github.com/coreos/go-semver/semver"
+)
+
+var (
+	// MinClusterVersion is the min cluster version this etcd binary is compatible with.
+	MinClusterVersion = "3.0.0"
+	Version           = "3.5.0-pre"
+	APIVersion        = "unknown"
+
+	// Git SHA Value will be set during build
+	GitSHA = "Not provided (use ./build instead of go build)"
+)
+
+func init() {
+	ver, err := semver.NewVersion(Version)
+	if err == nil {
+		APIVersion = fmt.Sprintf("%d.%d", ver.Major, ver.Minor)
+	}
+}
+
+type Versions struct {
+	Server  string `json:"etcdserver"`
+	Cluster string `json:"etcdcluster"`
+	// TODO: raft state machine version
+}
+
+// Cluster only keeps the major.minor.
+func Cluster(v string) string {
+	vs := strings.Split(v, ".")
+	if len(vs) <= 2 {
+		return v
+	}
+	return fmt.Sprintf("%s.%s", vs[0], vs[1])
+}
diff --git a/vendor/go.etcd.io/etcd/wal/decoder.go b/vendor/go.etcd.io/etcd/wal/decoder.go
new file mode 100644
index 0000000..f2f01fd
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/wal/decoder.go
@@ -0,0 +1,188 @@
+// Copyright 2015 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 wal
+
+import (
+	"bufio"
+	"encoding/binary"
+	"hash"
+	"io"
+	"sync"
+
+	"go.etcd.io/etcd/pkg/crc"
+	"go.etcd.io/etcd/pkg/pbutil"
+	"go.etcd.io/etcd/raft/raftpb"
+	"go.etcd.io/etcd/wal/walpb"
+)
+
+const minSectorSize = 512
+
+// frameSizeBytes is frame size in bytes, including record size and padding size.
+const frameSizeBytes = 8
+
+type decoder struct {
+	mu  sync.Mutex
+	brs []*bufio.Reader
+
+	// lastValidOff file offset following the last valid decoded record
+	lastValidOff int64
+	crc          hash.Hash32
+}
+
+func newDecoder(r ...io.Reader) *decoder {
+	readers := make([]*bufio.Reader, len(r))
+	for i := range r {
+		readers[i] = bufio.NewReader(r[i])
+	}
+	return &decoder{
+		brs: readers,
+		crc: crc.New(0, crcTable),
+	}
+}
+
+func (d *decoder) decode(rec *walpb.Record) error {
+	rec.Reset()
+	d.mu.Lock()
+	defer d.mu.Unlock()
+	return d.decodeRecord(rec)
+}
+
+func (d *decoder) decodeRecord(rec *walpb.Record) error {
+	if len(d.brs) == 0 {
+		return io.EOF
+	}
+
+	l, err := readInt64(d.brs[0])
+	if err == io.EOF || (err == nil && l == 0) {
+		// hit end of file or preallocated space
+		d.brs = d.brs[1:]
+		if len(d.brs) == 0 {
+			return io.EOF
+		}
+		d.lastValidOff = 0
+		return d.decodeRecord(rec)
+	}
+	if err != nil {
+		return err
+	}
+
+	recBytes, padBytes := decodeFrameSize(l)
+
+	data := make([]byte, recBytes+padBytes)
+	if _, err = io.ReadFull(d.brs[0], data); err != nil {
+		// ReadFull returns io.EOF only if no bytes were read
+		// the decoder should treat this as an ErrUnexpectedEOF instead.
+		if err == io.EOF {
+			err = io.ErrUnexpectedEOF
+		}
+		return err
+	}
+	if err := rec.Unmarshal(data[:recBytes]); err != nil {
+		if d.isTornEntry(data) {
+			return io.ErrUnexpectedEOF
+		}
+		return err
+	}
+
+	// skip crc checking if the record type is crcType
+	if rec.Type != crcType {
+		d.crc.Write(rec.Data)
+		if err := rec.Validate(d.crc.Sum32()); err != nil {
+			if d.isTornEntry(data) {
+				return io.ErrUnexpectedEOF
+			}
+			return err
+		}
+	}
+	// record decoded as valid; point last valid offset to end of record
+	d.lastValidOff += frameSizeBytes + recBytes + padBytes
+	return nil
+}
+
+func decodeFrameSize(lenField int64) (recBytes int64, padBytes int64) {
+	// the record size is stored in the lower 56 bits of the 64-bit length
+	recBytes = int64(uint64(lenField) & ^(uint64(0xff) << 56))
+	// non-zero padding is indicated by set MSb / a negative length
+	if lenField < 0 {
+		// padding is stored in lower 3 bits of length MSB
+		padBytes = int64((uint64(lenField) >> 56) & 0x7)
+	}
+	return recBytes, padBytes
+}
+
+// isTornEntry determines whether the last entry of the WAL was partially written
+// and corrupted because of a torn write.
+func (d *decoder) isTornEntry(data []byte) bool {
+	if len(d.brs) != 1 {
+		return false
+	}
+
+	fileOff := d.lastValidOff + frameSizeBytes
+	curOff := 0
+	chunks := [][]byte{}
+	// split data on sector boundaries
+	for curOff < len(data) {
+		chunkLen := int(minSectorSize - (fileOff % minSectorSize))
+		if chunkLen > len(data)-curOff {
+			chunkLen = len(data) - curOff
+		}
+		chunks = append(chunks, data[curOff:curOff+chunkLen])
+		fileOff += int64(chunkLen)
+		curOff += chunkLen
+	}
+
+	// if any data for a sector chunk is all 0, it's a torn write
+	for _, sect := range chunks {
+		isZero := true
+		for _, v := range sect {
+			if v != 0 {
+				isZero = false
+				break
+			}
+		}
+		if isZero {
+			return true
+		}
+	}
+	return false
+}
+
+func (d *decoder) updateCRC(prevCrc uint32) {
+	d.crc = crc.New(prevCrc, crcTable)
+}
+
+func (d *decoder) lastCRC() uint32 {
+	return d.crc.Sum32()
+}
+
+func (d *decoder) lastOffset() int64 { return d.lastValidOff }
+
+func mustUnmarshalEntry(d []byte) raftpb.Entry {
+	var e raftpb.Entry
+	pbutil.MustUnmarshal(&e, d)
+	return e
+}
+
+func mustUnmarshalState(d []byte) raftpb.HardState {
+	var s raftpb.HardState
+	pbutil.MustUnmarshal(&s, d)
+	return s
+}
+
+func readInt64(r io.Reader) (int64, error) {
+	var n int64
+	err := binary.Read(r, binary.LittleEndian, &n)
+	return n, err
+}
diff --git a/vendor/go.etcd.io/etcd/wal/doc.go b/vendor/go.etcd.io/etcd/wal/doc.go
new file mode 100644
index 0000000..7ea348e
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/wal/doc.go
@@ -0,0 +1,75 @@
+// Copyright 2015 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 wal provides an implementation of a write ahead log that is used by
+etcd.
+
+A WAL is created at a particular directory and is made up of a number of
+segmented WAL files. Inside of each file the raft state and entries are appended
+to it with the Save method:
+
+	metadata := []byte{}
+	w, err := wal.Create(zap.NewExample(), "/var/lib/etcd", metadata)
+	...
+	err := w.Save(s, ents)
+
+After saving a raft snapshot to disk, SaveSnapshot method should be called to
+record it. So WAL can match with the saved snapshot when restarting.
+
+	err := w.SaveSnapshot(walpb.Snapshot{Index: 10, Term: 2})
+
+When a user has finished using a WAL it must be closed:
+
+	w.Close()
+
+Each WAL file is a stream of WAL records. A WAL record is a length field and a wal record
+protobuf. The record protobuf contains a CRC, a type, and a data payload. The length field is a
+64-bit packed structure holding the length of the remaining logical record data in its lower
+56 bits and its physical padding in the first three bits of the most significant byte. Each
+record is 8-byte aligned so that the length field is never torn. The CRC contains the CRC32
+value of all record protobufs preceding the current record.
+
+WAL files are placed inside of the directory in the following format:
+$seq-$index.wal
+
+The first WAL file to be created will be 0000000000000000-0000000000000000.wal
+indicating an initial sequence of 0 and an initial raft index of 0. The first
+entry written to WAL MUST have raft index 0.
+
+WAL will cut its current tail wal file if its size exceeds 64MB. This will increment an internal
+sequence number and cause a new file to be created. If the last raft index saved
+was 0x20 and this is the first time cut has been called on this WAL then the sequence will
+increment from 0x0 to 0x1. The new file will be: 0000000000000001-0000000000000021.wal.
+If a second cut issues 0x10 entries with incremental index later then the file will be called:
+0000000000000002-0000000000000031.wal.
+
+At a later time a WAL can be opened at a particular snapshot. If there is no
+snapshot, an empty snapshot should be passed in.
+
+	w, err := wal.Open("/var/lib/etcd", walpb.Snapshot{Index: 10, Term: 2})
+	...
+
+The snapshot must have been written to the WAL.
+
+Additional items cannot be Saved to this WAL until all of the items from the given
+snapshot to the end of the WAL are read first:
+
+	metadata, state, ents, err := w.ReadAll()
+
+This will give you the metadata, the last raft.State and the slice of
+raft.Entry items in the log.
+
+*/
+package wal
diff --git a/vendor/go.etcd.io/etcd/wal/encoder.go b/vendor/go.etcd.io/etcd/wal/encoder.go
new file mode 100644
index 0000000..d3877ed
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/wal/encoder.go
@@ -0,0 +1,120 @@
+// Copyright 2015 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 wal
+
+import (
+	"encoding/binary"
+	"hash"
+	"io"
+	"os"
+	"sync"
+
+	"go.etcd.io/etcd/pkg/crc"
+	"go.etcd.io/etcd/pkg/ioutil"
+	"go.etcd.io/etcd/wal/walpb"
+)
+
+// walPageBytes is the alignment for flushing records to the backing Writer.
+// It should be a multiple of the minimum sector size so that WAL can safely
+// distinguish between torn writes and ordinary data corruption.
+const walPageBytes = 8 * minSectorSize
+
+type encoder struct {
+	mu sync.Mutex
+	bw *ioutil.PageWriter
+
+	crc       hash.Hash32
+	buf       []byte
+	uint64buf []byte
+}
+
+func newEncoder(w io.Writer, prevCrc uint32, pageOffset int) *encoder {
+	return &encoder{
+		bw:  ioutil.NewPageWriter(w, walPageBytes, pageOffset),
+		crc: crc.New(prevCrc, crcTable),
+		// 1MB buffer
+		buf:       make([]byte, 1024*1024),
+		uint64buf: make([]byte, 8),
+	}
+}
+
+// newFileEncoder creates a new encoder with current file offset for the page writer.
+func newFileEncoder(f *os.File, prevCrc uint32) (*encoder, error) {
+	offset, err := f.Seek(0, io.SeekCurrent)
+	if err != nil {
+		return nil, err
+	}
+	return newEncoder(f, prevCrc, int(offset)), nil
+}
+
+func (e *encoder) encode(rec *walpb.Record) error {
+	e.mu.Lock()
+	defer e.mu.Unlock()
+
+	e.crc.Write(rec.Data)
+	rec.Crc = e.crc.Sum32()
+	var (
+		data []byte
+		err  error
+		n    int
+	)
+
+	if rec.Size() > len(e.buf) {
+		data, err = rec.Marshal()
+		if err != nil {
+			return err
+		}
+	} else {
+		n, err = rec.MarshalTo(e.buf)
+		if err != nil {
+			return err
+		}
+		data = e.buf[:n]
+	}
+
+	lenField, padBytes := encodeFrameSize(len(data))
+	if err = writeUint64(e.bw, lenField, e.uint64buf); err != nil {
+		return err
+	}
+
+	if padBytes != 0 {
+		data = append(data, make([]byte, padBytes)...)
+	}
+	_, err = e.bw.Write(data)
+	return err
+}
+
+func encodeFrameSize(dataBytes int) (lenField uint64, padBytes int) {
+	lenField = uint64(dataBytes)
+	// force 8 byte alignment so length never gets a torn write
+	padBytes = (8 - (dataBytes % 8)) % 8
+	if padBytes != 0 {
+		lenField |= uint64(0x80|padBytes) << 56
+	}
+	return lenField, padBytes
+}
+
+func (e *encoder) flush() error {
+	e.mu.Lock()
+	defer e.mu.Unlock()
+	return e.bw.Flush()
+}
+
+func writeUint64(w io.Writer, n uint64, buf []byte) error {
+	// http://golang.org/src/encoding/binary/binary.go
+	binary.LittleEndian.PutUint64(buf, n)
+	_, err := w.Write(buf)
+	return err
+}
diff --git a/vendor/go.etcd.io/etcd/wal/file_pipeline.go b/vendor/go.etcd.io/etcd/wal/file_pipeline.go
new file mode 100644
index 0000000..e1e1c55
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/wal/file_pipeline.go
@@ -0,0 +1,106 @@
+// Copyright 2016 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 wal
+
+import (
+	"fmt"
+	"os"
+	"path/filepath"
+
+	"go.etcd.io/etcd/pkg/fileutil"
+
+	"go.uber.org/zap"
+)
+
+// filePipeline pipelines allocating disk space
+type filePipeline struct {
+	lg *zap.Logger
+
+	// dir to put files
+	dir string
+	// size of files to make, in bytes
+	size int64
+	// count number of files generated
+	count int
+
+	filec chan *fileutil.LockedFile
+	errc  chan error
+	donec chan struct{}
+}
+
+func newFilePipeline(lg *zap.Logger, dir string, fileSize int64) *filePipeline {
+	fp := &filePipeline{
+		lg:    lg,
+		dir:   dir,
+		size:  fileSize,
+		filec: make(chan *fileutil.LockedFile),
+		errc:  make(chan error, 1),
+		donec: make(chan struct{}),
+	}
+	go fp.run()
+	return fp
+}
+
+// Open returns a fresh file for writing. Rename the file before calling
+// Open again or there will be file collisions.
+func (fp *filePipeline) Open() (f *fileutil.LockedFile, err error) {
+	select {
+	case f = <-fp.filec:
+	case err = <-fp.errc:
+	}
+	return f, err
+}
+
+func (fp *filePipeline) Close() error {
+	close(fp.donec)
+	return <-fp.errc
+}
+
+func (fp *filePipeline) alloc() (f *fileutil.LockedFile, err error) {
+	// count % 2 so this file isn't the same as the one last published
+	fpath := filepath.Join(fp.dir, fmt.Sprintf("%d.tmp", fp.count%2))
+	if f, err = fileutil.LockFile(fpath, os.O_CREATE|os.O_WRONLY, fileutil.PrivateFileMode); err != nil {
+		return nil, err
+	}
+	if err = fileutil.Preallocate(f.File, fp.size, true); err != nil {
+		if fp.lg != nil {
+			fp.lg.Warn("failed to preallocate space when creating a new WAL", zap.Int64("size", fp.size), zap.Error(err))
+		} else {
+			plog.Errorf("failed to allocate space when creating new wal file (%v)", err)
+		}
+		f.Close()
+		return nil, err
+	}
+	fp.count++
+	return f, nil
+}
+
+func (fp *filePipeline) run() {
+	defer close(fp.errc)
+	for {
+		f, err := fp.alloc()
+		if err != nil {
+			fp.errc <- err
+			return
+		}
+		select {
+		case fp.filec <- f:
+		case <-fp.donec:
+			os.Remove(f.Name())
+			f.Close()
+			return
+		}
+	}
+}
diff --git a/vendor/go.etcd.io/etcd/wal/metrics.go b/vendor/go.etcd.io/etcd/wal/metrics.go
new file mode 100644
index 0000000..22cb800
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/wal/metrics.go
@@ -0,0 +1,34 @@
+// Copyright 2015 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 wal
+
+import "github.com/prometheus/client_golang/prometheus"
+
+var (
+	walFsyncSec = prometheus.NewHistogram(prometheus.HistogramOpts{
+		Namespace: "etcd",
+		Subsystem: "disk",
+		Name:      "wal_fsync_duration_seconds",
+		Help:      "The latency distributions of fsync called by WAL.",
+
+		// lowest bucket start of upper bound 0.001 sec (1 ms) with factor 2
+		// highest bucket start of 0.001 sec * 2^13 == 8.192 sec
+		Buckets: prometheus.ExponentialBuckets(0.001, 2, 14),
+	})
+)
+
+func init() {
+	prometheus.MustRegister(walFsyncSec)
+}
diff --git a/vendor/go.etcd.io/etcd/wal/repair.go b/vendor/go.etcd.io/etcd/wal/repair.go
new file mode 100644
index 0000000..15afed0
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/wal/repair.go
@@ -0,0 +1,141 @@
+// Copyright 2015 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 wal
+
+import (
+	"io"
+	"os"
+	"path/filepath"
+
+	"go.etcd.io/etcd/pkg/fileutil"
+	"go.etcd.io/etcd/wal/walpb"
+
+	"go.uber.org/zap"
+)
+
+// Repair tries to repair ErrUnexpectedEOF in the
+// last wal file by truncating.
+func Repair(lg *zap.Logger, dirpath string) bool {
+	f, err := openLast(lg, dirpath)
+	if err != nil {
+		return false
+	}
+	defer f.Close()
+
+	if lg != nil {
+		lg.Info("repairing", zap.String("path", f.Name()))
+	} else {
+		plog.Noticef("repairing %v", f.Name())
+	}
+
+	rec := &walpb.Record{}
+	decoder := newDecoder(f)
+	for {
+		lastOffset := decoder.lastOffset()
+		err := decoder.decode(rec)
+		switch err {
+		case nil:
+			// update crc of the decoder when necessary
+			switch rec.Type {
+			case crcType:
+				crc := decoder.crc.Sum32()
+				// current crc of decoder must match the crc of the record.
+				// do no need to match 0 crc, since the decoder is a new one at this case.
+				if crc != 0 && rec.Validate(crc) != nil {
+					return false
+				}
+				decoder.updateCRC(rec.Crc)
+			}
+			continue
+
+		case io.EOF:
+			if lg != nil {
+				lg.Info("repaired", zap.String("path", f.Name()), zap.Error(io.EOF))
+			}
+			return true
+
+		case io.ErrUnexpectedEOF:
+			bf, bferr := os.Create(f.Name() + ".broken")
+			if bferr != nil {
+				if lg != nil {
+					lg.Warn("failed to create backup file", zap.String("path", f.Name()+".broken"), zap.Error(bferr))
+				} else {
+					plog.Errorf("could not repair %v, failed to create backup file", f.Name())
+				}
+				return false
+			}
+			defer bf.Close()
+
+			if _, err = f.Seek(0, io.SeekStart); err != nil {
+				if lg != nil {
+					lg.Warn("failed to read file", zap.String("path", f.Name()), zap.Error(err))
+				} else {
+					plog.Errorf("could not repair %v, failed to read file", f.Name())
+				}
+				return false
+			}
+
+			if _, err = io.Copy(bf, f); err != nil {
+				if lg != nil {
+					lg.Warn("failed to copy", zap.String("from", f.Name()+".broken"), zap.String("to", f.Name()), zap.Error(err))
+				} else {
+					plog.Errorf("could not repair %v, failed to copy file", f.Name())
+				}
+				return false
+			}
+
+			if err = f.Truncate(lastOffset); err != nil {
+				if lg != nil {
+					lg.Warn("failed to truncate", zap.String("path", f.Name()), zap.Error(err))
+				} else {
+					plog.Errorf("could not repair %v, failed to truncate file", f.Name())
+				}
+				return false
+			}
+
+			if err = fileutil.Fsync(f.File); err != nil {
+				if lg != nil {
+					lg.Warn("failed to fsync", zap.String("path", f.Name()), zap.Error(err))
+				} else {
+					plog.Errorf("could not repair %v, failed to sync file", f.Name())
+				}
+				return false
+			}
+
+			if lg != nil {
+				lg.Info("repaired", zap.String("path", f.Name()), zap.Error(io.ErrUnexpectedEOF))
+			}
+			return true
+
+		default:
+			if lg != nil {
+				lg.Warn("failed to repair", zap.String("path", f.Name()), zap.Error(err))
+			} else {
+				plog.Errorf("could not repair error (%v)", err)
+			}
+			return false
+		}
+	}
+}
+
+// openLast opens the last wal file for read and write.
+func openLast(lg *zap.Logger, dirpath string) (*fileutil.LockedFile, error) {
+	names, err := readWALNames(lg, dirpath)
+	if err != nil {
+		return nil, err
+	}
+	last := filepath.Join(dirpath, names[len(names)-1])
+	return fileutil.LockFile(last, os.O_RDWR, fileutil.PrivateFileMode)
+}
diff --git a/vendor/go.etcd.io/etcd/wal/util.go b/vendor/go.etcd.io/etcd/wal/util.go
new file mode 100644
index 0000000..a3f314b
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/wal/util.go
@@ -0,0 +1,124 @@
+// Copyright 2015 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 wal
+
+import (
+	"errors"
+	"fmt"
+	"strings"
+
+	"go.etcd.io/etcd/pkg/fileutil"
+
+	"go.uber.org/zap"
+)
+
+var errBadWALName = errors.New("bad wal name")
+
+// Exist returns true if there are any files in a given directory.
+func Exist(dir string) bool {
+	names, err := fileutil.ReadDir(dir, fileutil.WithExt(".wal"))
+	if err != nil {
+		return false
+	}
+	return len(names) != 0
+}
+
+// searchIndex returns the last array index of names whose raft index section is
+// equal to or smaller than the given index.
+// The given names MUST be sorted.
+func searchIndex(lg *zap.Logger, names []string, index uint64) (int, bool) {
+	for i := len(names) - 1; i >= 0; i-- {
+		name := names[i]
+		_, curIndex, err := parseWALName(name)
+		if err != nil {
+			if lg != nil {
+				lg.Panic("failed to parse WAL file name", zap.String("path", name), zap.Error(err))
+			} else {
+				plog.Panicf("parse correct name should never fail: %v", err)
+			}
+		}
+		if index >= curIndex {
+			return i, true
+		}
+	}
+	return -1, false
+}
+
+// names should have been sorted based on sequence number.
+// isValidSeq checks whether seq increases continuously.
+func isValidSeq(lg *zap.Logger, names []string) bool {
+	var lastSeq uint64
+	for _, name := range names {
+		curSeq, _, err := parseWALName(name)
+		if err != nil {
+			if lg != nil {
+				lg.Panic("failed to parse WAL file name", zap.String("path", name), zap.Error(err))
+			} else {
+				plog.Panicf("parse correct name should never fail: %v", err)
+			}
+		}
+		if lastSeq != 0 && lastSeq != curSeq-1 {
+			return false
+		}
+		lastSeq = curSeq
+	}
+	return true
+}
+
+func readWALNames(lg *zap.Logger, dirpath string) ([]string, error) {
+	names, err := fileutil.ReadDir(dirpath)
+	if err != nil {
+		return nil, err
+	}
+	wnames := checkWalNames(lg, names)
+	if len(wnames) == 0 {
+		return nil, ErrFileNotFound
+	}
+	return wnames, nil
+}
+
+func checkWalNames(lg *zap.Logger, names []string) []string {
+	wnames := make([]string, 0)
+	for _, name := range names {
+		if _, _, err := parseWALName(name); err != nil {
+			// don't complain about left over tmp files
+			if !strings.HasSuffix(name, ".tmp") {
+				if lg != nil {
+					lg.Warn(
+						"ignored file in WAL directory",
+						zap.String("path", name),
+					)
+				} else {
+					plog.Warningf("ignored file %v in wal", name)
+				}
+			}
+			continue
+		}
+		wnames = append(wnames, name)
+	}
+	return wnames
+}
+
+func parseWALName(str string) (seq, index uint64, err error) {
+	if !strings.HasSuffix(str, ".wal") {
+		return 0, 0, errBadWALName
+	}
+	_, err = fmt.Sscanf(str, "%016x-%016x.wal", &seq, &index)
+	return seq, index, err
+}
+
+func walName(seq, index uint64) string {
+	return fmt.Sprintf("%016x-%016x.wal", seq, index)
+}
diff --git a/vendor/go.etcd.io/etcd/wal/wal.go b/vendor/go.etcd.io/etcd/wal/wal.go
new file mode 100644
index 0000000..e177c8e
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/wal/wal.go
@@ -0,0 +1,912 @@
+// Copyright 2015 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 wal
+
+import (
+	"bytes"
+	"errors"
+	"fmt"
+	"hash/crc32"
+	"io"
+	"os"
+	"path/filepath"
+	"sync"
+	"time"
+
+	"go.etcd.io/etcd/pkg/fileutil"
+	"go.etcd.io/etcd/pkg/pbutil"
+	"go.etcd.io/etcd/raft"
+	"go.etcd.io/etcd/raft/raftpb"
+	"go.etcd.io/etcd/wal/walpb"
+
+	"github.com/coreos/pkg/capnslog"
+	"go.uber.org/zap"
+)
+
+const (
+	metadataType int64 = iota + 1
+	entryType
+	stateType
+	crcType
+	snapshotType
+
+	// warnSyncDuration is the amount of time allotted to an fsync before
+	// logging a warning
+	warnSyncDuration = time.Second
+)
+
+var (
+	// SegmentSizeBytes is the preallocated size of each wal segment file.
+	// The actual size might be larger than this. In general, the default
+	// value should be used, but this is defined as an exported variable
+	// so that tests can set a different segment size.
+	SegmentSizeBytes int64 = 64 * 1000 * 1000 // 64MB
+
+	plog = capnslog.NewPackageLogger("go.etcd.io/etcd", "wal")
+
+	ErrMetadataConflict = errors.New("wal: conflicting metadata found")
+	ErrFileNotFound     = errors.New("wal: file not found")
+	ErrCRCMismatch      = errors.New("wal: crc mismatch")
+	ErrSnapshotMismatch = errors.New("wal: snapshot mismatch")
+	ErrSnapshotNotFound = errors.New("wal: snapshot not found")
+	crcTable            = crc32.MakeTable(crc32.Castagnoli)
+)
+
+// WAL is a logical representation of the stable storage.
+// WAL is either in read mode or append mode but not both.
+// A newly created WAL is in append mode, and ready for appending records.
+// A just opened WAL is in read mode, and ready for reading records.
+// The WAL will be ready for appending after reading out all the previous records.
+type WAL struct {
+	lg *zap.Logger
+
+	dir string // the living directory of the underlay files
+
+	// dirFile is a fd for the wal directory for syncing on Rename
+	dirFile *os.File
+
+	metadata []byte           // metadata recorded at the head of each WAL
+	state    raftpb.HardState // hardstate recorded at the head of WAL
+
+	start     walpb.Snapshot // snapshot to start reading
+	decoder   *decoder       // decoder to decode records
+	readClose func() error   // closer for decode reader
+
+	mu      sync.Mutex
+	enti    uint64   // index of the last entry saved to the wal
+	encoder *encoder // encoder to encode records
+
+	locks []*fileutil.LockedFile // the locked files the WAL holds (the name is increasing)
+	fp    *filePipeline
+}
+
+// Create creates a WAL ready for appending records. The given metadata is
+// recorded at the head of each WAL file, and can be retrieved with ReadAll.
+func Create(lg *zap.Logger, dirpath string, metadata []byte) (*WAL, error) {
+	if Exist(dirpath) {
+		return nil, os.ErrExist
+	}
+
+	// keep temporary wal directory so WAL initialization appears atomic
+	tmpdirpath := filepath.Clean(dirpath) + ".tmp"
+	if fileutil.Exist(tmpdirpath) {
+		if err := os.RemoveAll(tmpdirpath); err != nil {
+			return nil, err
+		}
+	}
+	if err := fileutil.CreateDirAll(tmpdirpath); err != nil {
+		if lg != nil {
+			lg.Warn(
+				"failed to create a temporary WAL directory",
+				zap.String("tmp-dir-path", tmpdirpath),
+				zap.String("dir-path", dirpath),
+				zap.Error(err),
+			)
+		}
+		return nil, err
+	}
+
+	p := filepath.Join(tmpdirpath, walName(0, 0))
+	f, err := fileutil.LockFile(p, os.O_WRONLY|os.O_CREATE, fileutil.PrivateFileMode)
+	if err != nil {
+		if lg != nil {
+			lg.Warn(
+				"failed to flock an initial WAL file",
+				zap.String("path", p),
+				zap.Error(err),
+			)
+		}
+		return nil, err
+	}
+	if _, err = f.Seek(0, io.SeekEnd); err != nil {
+		if lg != nil {
+			lg.Warn(
+				"failed to seek an initial WAL file",
+				zap.String("path", p),
+				zap.Error(err),
+			)
+		}
+		return nil, err
+	}
+	if err = fileutil.Preallocate(f.File, SegmentSizeBytes, true); err != nil {
+		if lg != nil {
+			lg.Warn(
+				"failed to preallocate an initial WAL file",
+				zap.String("path", p),
+				zap.Int64("segment-bytes", SegmentSizeBytes),
+				zap.Error(err),
+			)
+		}
+		return nil, err
+	}
+
+	w := &WAL{
+		lg:       lg,
+		dir:      dirpath,
+		metadata: metadata,
+	}
+	w.encoder, err = newFileEncoder(f.File, 0)
+	if err != nil {
+		return nil, err
+	}
+	w.locks = append(w.locks, f)
+	if err = w.saveCrc(0); err != nil {
+		return nil, err
+	}
+	if err = w.encoder.encode(&walpb.Record{Type: metadataType, Data: metadata}); err != nil {
+		return nil, err
+	}
+	if err = w.SaveSnapshot(walpb.Snapshot{}); err != nil {
+		return nil, err
+	}
+
+	if w, err = w.renameWAL(tmpdirpath); err != nil {
+		if lg != nil {
+			lg.Warn(
+				"failed to rename the temporary WAL directory",
+				zap.String("tmp-dir-path", tmpdirpath),
+				zap.String("dir-path", w.dir),
+				zap.Error(err),
+			)
+		}
+		return nil, err
+	}
+
+	var perr error
+	defer func() {
+		if perr != nil {
+			w.cleanupWAL(lg)
+		}
+	}()
+
+	// directory was renamed; sync parent dir to persist rename
+	pdir, perr := fileutil.OpenDir(filepath.Dir(w.dir))
+	if perr != nil {
+		if lg != nil {
+			lg.Warn(
+				"failed to open the parent data directory",
+				zap.String("parent-dir-path", filepath.Dir(w.dir)),
+				zap.String("dir-path", w.dir),
+				zap.Error(perr),
+			)
+		}
+		return nil, perr
+	}
+	if perr = fileutil.Fsync(pdir); perr != nil {
+		if lg != nil {
+			lg.Warn(
+				"failed to fsync the parent data directory file",
+				zap.String("parent-dir-path", filepath.Dir(w.dir)),
+				zap.String("dir-path", w.dir),
+				zap.Error(perr),
+			)
+		}
+		return nil, perr
+	}
+	if perr = pdir.Close(); perr != nil {
+		if lg != nil {
+			lg.Warn(
+				"failed to close the parent data directory file",
+				zap.String("parent-dir-path", filepath.Dir(w.dir)),
+				zap.String("dir-path", w.dir),
+				zap.Error(perr),
+			)
+		}
+		return nil, perr
+	}
+
+	return w, nil
+}
+
+func (w *WAL) cleanupWAL(lg *zap.Logger) {
+	var err error
+	if err = w.Close(); err != nil {
+		if lg != nil {
+			lg.Panic("failed to close WAL during cleanup", zap.Error(err))
+		} else {
+			plog.Panicf("failed to close WAL during cleanup: %v", err)
+		}
+	}
+	brokenDirName := fmt.Sprintf("%s.broken.%v", w.dir, time.Now().Format("20060102.150405.999999"))
+	if err = os.Rename(w.dir, brokenDirName); err != nil {
+		if lg != nil {
+			lg.Panic(
+				"failed to rename WAL during cleanup",
+				zap.Error(err),
+				zap.String("source-path", w.dir),
+				zap.String("rename-path", brokenDirName),
+			)
+		} else {
+			plog.Panicf("failed to rename WAL during cleanup: %v", err)
+		}
+	}
+}
+
+func (w *WAL) renameWAL(tmpdirpath string) (*WAL, error) {
+	if err := os.RemoveAll(w.dir); err != nil {
+		return nil, err
+	}
+	// On non-Windows platforms, hold the lock while renaming. Releasing
+	// the lock and trying to reacquire it quickly can be flaky because
+	// it's possible the process will fork to spawn a process while this is
+	// happening. The fds are set up as close-on-exec by the Go runtime,
+	// but there is a window between the fork and the exec where another
+	// process holds the lock.
+	if err := os.Rename(tmpdirpath, w.dir); err != nil {
+		if _, ok := err.(*os.LinkError); ok {
+			return w.renameWALUnlock(tmpdirpath)
+		}
+		return nil, err
+	}
+	w.fp = newFilePipeline(w.lg, w.dir, SegmentSizeBytes)
+	df, err := fileutil.OpenDir(w.dir)
+	w.dirFile = df
+	return w, err
+}
+
+func (w *WAL) renameWALUnlock(tmpdirpath string) (*WAL, error) {
+	// rename of directory with locked files doesn't work on windows/cifs;
+	// close the WAL to release the locks so the directory can be renamed.
+	if w.lg != nil {
+		w.lg.Info(
+			"closing WAL to release flock and retry directory renaming",
+			zap.String("from", tmpdirpath),
+			zap.String("to", w.dir),
+		)
+	} else {
+		plog.Infof("releasing file lock to rename %q to %q", tmpdirpath, w.dir)
+	}
+	w.Close()
+
+	if err := os.Rename(tmpdirpath, w.dir); err != nil {
+		return nil, err
+	}
+
+	// reopen and relock
+	newWAL, oerr := Open(w.lg, w.dir, walpb.Snapshot{})
+	if oerr != nil {
+		return nil, oerr
+	}
+	if _, _, _, err := newWAL.ReadAll(); err != nil {
+		newWAL.Close()
+		return nil, err
+	}
+	return newWAL, nil
+}
+
+// Open opens the WAL at the given snap.
+// The snap SHOULD have been previously saved to the WAL, or the following
+// ReadAll will fail.
+// The returned WAL is ready to read and the first record will be the one after
+// the given snap. The WAL cannot be appended to before reading out all of its
+// previous records.
+func Open(lg *zap.Logger, dirpath string, snap walpb.Snapshot) (*WAL, error) {
+	w, err := openAtIndex(lg, dirpath, snap, true)
+	if err != nil {
+		return nil, err
+	}
+	if w.dirFile, err = fileutil.OpenDir(w.dir); err != nil {
+		return nil, err
+	}
+	return w, nil
+}
+
+// OpenForRead only opens the wal files for read.
+// Write on a read only wal panics.
+func OpenForRead(lg *zap.Logger, dirpath string, snap walpb.Snapshot) (*WAL, error) {
+	return openAtIndex(lg, dirpath, snap, false)
+}
+
+func openAtIndex(lg *zap.Logger, dirpath string, snap walpb.Snapshot, write bool) (*WAL, error) {
+	names, nameIndex, err := selectWALFiles(lg, dirpath, snap)
+	if err != nil {
+		return nil, err
+	}
+
+	rs, ls, closer, err := openWALFiles(lg, dirpath, names, nameIndex, write)
+	if err != nil {
+		return nil, err
+	}
+
+	// create a WAL ready for reading
+	w := &WAL{
+		lg:        lg,
+		dir:       dirpath,
+		start:     snap,
+		decoder:   newDecoder(rs...),
+		readClose: closer,
+		locks:     ls,
+	}
+
+	if write {
+		// write reuses the file descriptors from read; don't close so
+		// WAL can append without dropping the file lock
+		w.readClose = nil
+		if _, _, err := parseWALName(filepath.Base(w.tail().Name())); err != nil {
+			closer()
+			return nil, err
+		}
+		w.fp = newFilePipeline(lg, w.dir, SegmentSizeBytes)
+	}
+
+	return w, nil
+}
+
+func selectWALFiles(lg *zap.Logger, dirpath string, snap walpb.Snapshot) ([]string, int, error) {
+	names, err := readWALNames(lg, dirpath)
+	if err != nil {
+		return nil, -1, err
+	}
+
+	nameIndex, ok := searchIndex(lg, names, snap.Index)
+	if !ok || !isValidSeq(lg, names[nameIndex:]) {
+		err = ErrFileNotFound
+		return nil, -1, err
+	}
+
+	return names, nameIndex, nil
+}
+
+func openWALFiles(lg *zap.Logger, dirpath string, names []string, nameIndex int, write bool) ([]io.Reader, []*fileutil.LockedFile, func() error, error) {
+	rcs := make([]io.ReadCloser, 0)
+	rs := make([]io.Reader, 0)
+	ls := make([]*fileutil.LockedFile, 0)
+	for _, name := range names[nameIndex:] {
+		p := filepath.Join(dirpath, name)
+		if write {
+			l, err := fileutil.TryLockFile(p, os.O_RDWR, fileutil.PrivateFileMode)
+			if err != nil {
+				closeAll(rcs...)
+				return nil, nil, nil, err
+			}
+			ls = append(ls, l)
+			rcs = append(rcs, l)
+		} else {
+			rf, err := os.OpenFile(p, os.O_RDONLY, fileutil.PrivateFileMode)
+			if err != nil {
+				closeAll(rcs...)
+				return nil, nil, nil, err
+			}
+			ls = append(ls, nil)
+			rcs = append(rcs, rf)
+		}
+		rs = append(rs, rcs[len(rcs)-1])
+	}
+
+	closer := func() error { return closeAll(rcs...) }
+
+	return rs, ls, closer, nil
+}
+
+// ReadAll reads out records of the current WAL.
+// If opened in write mode, it must read out all records until EOF. Or an error
+// will be returned.
+// If opened in read mode, it will try to read all records if possible.
+// If it cannot read out the expected snap, it will return ErrSnapshotNotFound.
+// If loaded snap doesn't match with the expected one, it will return
+// all the records and error ErrSnapshotMismatch.
+// TODO: detect not-last-snap error.
+// TODO: maybe loose the checking of match.
+// After ReadAll, the WAL will be ready for appending new records.
+func (w *WAL) ReadAll() (metadata []byte, state raftpb.HardState, ents []raftpb.Entry, err error) {
+	w.mu.Lock()
+	defer w.mu.Unlock()
+
+	rec := &walpb.Record{}
+	decoder := w.decoder
+
+	var match bool
+	for err = decoder.decode(rec); err == nil; err = decoder.decode(rec) {
+		switch rec.Type {
+		case entryType:
+			e := mustUnmarshalEntry(rec.Data)
+			if e.Index > w.start.Index {
+				ents = append(ents[:e.Index-w.start.Index-1], e)
+			}
+			w.enti = e.Index
+
+		case stateType:
+			state = mustUnmarshalState(rec.Data)
+
+		case metadataType:
+			if metadata != nil && !bytes.Equal(metadata, rec.Data) {
+				state.Reset()
+				return nil, state, nil, ErrMetadataConflict
+			}
+			metadata = rec.Data
+
+		case crcType:
+			crc := decoder.crc.Sum32()
+			// current crc of decoder must match the crc of the record.
+			// do no need to match 0 crc, since the decoder is a new one at this case.
+			if crc != 0 && rec.Validate(crc) != nil {
+				state.Reset()
+				return nil, state, nil, ErrCRCMismatch
+			}
+			decoder.updateCRC(rec.Crc)
+
+		case snapshotType:
+			var snap walpb.Snapshot
+			pbutil.MustUnmarshal(&snap, rec.Data)
+			if snap.Index == w.start.Index {
+				if snap.Term != w.start.Term {
+					state.Reset()
+					return nil, state, nil, ErrSnapshotMismatch
+				}
+				match = true
+			}
+
+		default:
+			state.Reset()
+			return nil, state, nil, fmt.Errorf("unexpected block type %d", rec.Type)
+		}
+	}
+
+	switch w.tail() {
+	case nil:
+		// We do not have to read out all entries in read mode.
+		// The last record maybe a partial written one, so
+		// ErrunexpectedEOF might be returned.
+		if err != io.EOF && err != io.ErrUnexpectedEOF {
+			state.Reset()
+			return nil, state, nil, err
+		}
+	default:
+		// We must read all of the entries if WAL is opened in write mode.
+		if err != io.EOF {
+			state.Reset()
+			return nil, state, nil, err
+		}
+		// decodeRecord() will return io.EOF if it detects a zero record,
+		// but this zero record may be followed by non-zero records from
+		// a torn write. Overwriting some of these non-zero records, but
+		// not all, will cause CRC errors on WAL open. Since the records
+		// were never fully synced to disk in the first place, it's safe
+		// to zero them out to avoid any CRC errors from new writes.
+		if _, err = w.tail().Seek(w.decoder.lastOffset(), io.SeekStart); err != nil {
+			return nil, state, nil, err
+		}
+		if err = fileutil.ZeroToEnd(w.tail().File); err != nil {
+			return nil, state, nil, err
+		}
+	}
+
+	err = nil
+	if !match {
+		err = ErrSnapshotNotFound
+	}
+
+	// close decoder, disable reading
+	if w.readClose != nil {
+		w.readClose()
+		w.readClose = nil
+	}
+	w.start = walpb.Snapshot{}
+
+	w.metadata = metadata
+
+	if w.tail() != nil {
+		// create encoder (chain crc with the decoder), enable appending
+		w.encoder, err = newFileEncoder(w.tail().File, w.decoder.lastCRC())
+		if err != nil {
+			return
+		}
+	}
+	w.decoder = nil
+
+	return metadata, state, ents, err
+}
+
+// Verify reads through the given WAL and verifies that it is not corrupted.
+// It creates a new decoder to read through the records of the given WAL.
+// It does not conflict with any open WAL, but it is recommended not to
+// call this function after opening the WAL for writing.
+// If it cannot read out the expected snap, it will return ErrSnapshotNotFound.
+// If the loaded snap doesn't match with the expected one, it will
+// return error ErrSnapshotMismatch.
+func Verify(lg *zap.Logger, walDir string, snap walpb.Snapshot) error {
+	var metadata []byte
+	var err error
+	var match bool
+
+	rec := &walpb.Record{}
+
+	names, nameIndex, err := selectWALFiles(lg, walDir, snap)
+	if err != nil {
+		return err
+	}
+
+	// open wal files in read mode, so that there is no conflict
+	// when the same WAL is opened elsewhere in write mode
+	rs, _, closer, err := openWALFiles(lg, walDir, names, nameIndex, false)
+	if err != nil {
+		return err
+	}
+	defer func() {
+		if closer != nil {
+			closer()
+		}
+	}()
+
+	// create a new decoder from the readers on the WAL files
+	decoder := newDecoder(rs...)
+
+	for err = decoder.decode(rec); err == nil; err = decoder.decode(rec) {
+		switch rec.Type {
+		case metadataType:
+			if metadata != nil && !bytes.Equal(metadata, rec.Data) {
+				return ErrMetadataConflict
+			}
+			metadata = rec.Data
+		case crcType:
+			crc := decoder.crc.Sum32()
+			// Current crc of decoder must match the crc of the record.
+			// We need not match 0 crc, since the decoder is a new one at this point.
+			if crc != 0 && rec.Validate(crc) != nil {
+				return ErrCRCMismatch
+			}
+			decoder.updateCRC(rec.Crc)
+		case snapshotType:
+			var loadedSnap walpb.Snapshot
+			pbutil.MustUnmarshal(&loadedSnap, rec.Data)
+			if loadedSnap.Index == snap.Index {
+				if loadedSnap.Term != snap.Term {
+					return ErrSnapshotMismatch
+				}
+				match = true
+			}
+		// We ignore all entry and state type records as these
+		// are not necessary for validating the WAL contents
+		case entryType:
+		case stateType:
+		default:
+			return fmt.Errorf("unexpected block type %d", rec.Type)
+		}
+	}
+
+	// We do not have to read out all the WAL entries
+	// as the decoder is opened in read mode.
+	if err != io.EOF && err != io.ErrUnexpectedEOF {
+		return err
+	}
+
+	if !match {
+		return ErrSnapshotNotFound
+	}
+
+	return nil
+}
+
+// cut closes current file written and creates a new one ready to append.
+// cut first creates a temp wal file and writes necessary headers into it.
+// Then cut atomically rename temp wal file to a wal file.
+func (w *WAL) cut() error {
+	// close old wal file; truncate to avoid wasting space if an early cut
+	off, serr := w.tail().Seek(0, io.SeekCurrent)
+	if serr != nil {
+		return serr
+	}
+
+	if err := w.tail().Truncate(off); err != nil {
+		return err
+	}
+
+	if err := w.sync(); err != nil {
+		return err
+	}
+
+	fpath := filepath.Join(w.dir, walName(w.seq()+1, w.enti+1))
+
+	// create a temp wal file with name sequence + 1, or truncate the existing one
+	newTail, err := w.fp.Open()
+	if err != nil {
+		return err
+	}
+
+	// update writer and save the previous crc
+	w.locks = append(w.locks, newTail)
+	prevCrc := w.encoder.crc.Sum32()
+	w.encoder, err = newFileEncoder(w.tail().File, prevCrc)
+	if err != nil {
+		return err
+	}
+
+	if err = w.saveCrc(prevCrc); err != nil {
+		return err
+	}
+
+	if err = w.encoder.encode(&walpb.Record{Type: metadataType, Data: w.metadata}); err != nil {
+		return err
+	}
+
+	if err = w.saveState(&w.state); err != nil {
+		return err
+	}
+
+	// atomically move temp wal file to wal file
+	if err = w.sync(); err != nil {
+		return err
+	}
+
+	off, err = w.tail().Seek(0, io.SeekCurrent)
+	if err != nil {
+		return err
+	}
+
+	if err = os.Rename(newTail.Name(), fpath); err != nil {
+		return err
+	}
+	if err = fileutil.Fsync(w.dirFile); err != nil {
+		return err
+	}
+
+	// reopen newTail with its new path so calls to Name() match the wal filename format
+	newTail.Close()
+
+	if newTail, err = fileutil.LockFile(fpath, os.O_WRONLY, fileutil.PrivateFileMode); err != nil {
+		return err
+	}
+	if _, err = newTail.Seek(off, io.SeekStart); err != nil {
+		return err
+	}
+
+	w.locks[len(w.locks)-1] = newTail
+
+	prevCrc = w.encoder.crc.Sum32()
+	w.encoder, err = newFileEncoder(w.tail().File, prevCrc)
+	if err != nil {
+		return err
+	}
+
+	if w.lg != nil {
+		w.lg.Info("created a new WAL segment", zap.String("path", fpath))
+	} else {
+		plog.Infof("segmented wal file %v is created", fpath)
+	}
+	return nil
+}
+
+func (w *WAL) sync() error {
+	if w.encoder != nil {
+		if err := w.encoder.flush(); err != nil {
+			return err
+		}
+	}
+	start := time.Now()
+	err := fileutil.Fdatasync(w.tail().File)
+
+	took := time.Since(start)
+	if took > warnSyncDuration {
+		if w.lg != nil {
+			w.lg.Warn(
+				"slow fdatasync",
+				zap.Duration("took", took),
+				zap.Duration("expected-duration", warnSyncDuration),
+			)
+		} else {
+			plog.Warningf("sync duration of %v, expected less than %v", took, warnSyncDuration)
+		}
+	}
+	walFsyncSec.Observe(took.Seconds())
+
+	return err
+}
+
+// ReleaseLockTo releases the locks, which has smaller index than the given index
+// except the largest one among them.
+// For example, if WAL is holding lock 1,2,3,4,5,6, ReleaseLockTo(4) will release
+// lock 1,2 but keep 3. ReleaseLockTo(5) will release 1,2,3 but keep 4.
+func (w *WAL) ReleaseLockTo(index uint64) error {
+	w.mu.Lock()
+	defer w.mu.Unlock()
+
+	if len(w.locks) == 0 {
+		return nil
+	}
+
+	var smaller int
+	found := false
+	for i, l := range w.locks {
+		_, lockIndex, err := parseWALName(filepath.Base(l.Name()))
+		if err != nil {
+			return err
+		}
+		if lockIndex >= index {
+			smaller = i - 1
+			found = true
+			break
+		}
+	}
+
+	// if no lock index is greater than the release index, we can
+	// release lock up to the last one(excluding).
+	if !found {
+		smaller = len(w.locks) - 1
+	}
+
+	if smaller <= 0 {
+		return nil
+	}
+
+	for i := 0; i < smaller; i++ {
+		if w.locks[i] == nil {
+			continue
+		}
+		w.locks[i].Close()
+	}
+	w.locks = w.locks[smaller:]
+
+	return nil
+}
+
+// Close closes the current WAL file and directory.
+func (w *WAL) Close() error {
+	w.mu.Lock()
+	defer w.mu.Unlock()
+
+	if w.fp != nil {
+		w.fp.Close()
+		w.fp = nil
+	}
+
+	if w.tail() != nil {
+		if err := w.sync(); err != nil {
+			return err
+		}
+	}
+	for _, l := range w.locks {
+		if l == nil {
+			continue
+		}
+		if err := l.Close(); err != nil {
+			if w.lg != nil {
+				w.lg.Warn("failed to close WAL", zap.Error(err))
+			} else {
+				plog.Errorf("failed to unlock during closing wal: %s", err)
+			}
+		}
+	}
+
+	return w.dirFile.Close()
+}
+
+func (w *WAL) saveEntry(e *raftpb.Entry) error {
+	// TODO: add MustMarshalTo to reduce one allocation.
+	b := pbutil.MustMarshal(e)
+	rec := &walpb.Record{Type: entryType, Data: b}
+	if err := w.encoder.encode(rec); err != nil {
+		return err
+	}
+	w.enti = e.Index
+	return nil
+}
+
+func (w *WAL) saveState(s *raftpb.HardState) error {
+	if raft.IsEmptyHardState(*s) {
+		return nil
+	}
+	w.state = *s
+	b := pbutil.MustMarshal(s)
+	rec := &walpb.Record{Type: stateType, Data: b}
+	return w.encoder.encode(rec)
+}
+
+func (w *WAL) Save(st raftpb.HardState, ents []raftpb.Entry) error {
+	w.mu.Lock()
+	defer w.mu.Unlock()
+
+	// short cut, do not call sync
+	if raft.IsEmptyHardState(st) && len(ents) == 0 {
+		return nil
+	}
+
+	mustSync := raft.MustSync(st, w.state, len(ents))
+
+	// TODO(xiangli): no more reference operator
+	for i := range ents {
+		if err := w.saveEntry(&ents[i]); err != nil {
+			return err
+		}
+	}
+	if err := w.saveState(&st); err != nil {
+		return err
+	}
+
+	curOff, err := w.tail().Seek(0, io.SeekCurrent)
+	if err != nil {
+		return err
+	}
+	if curOff < SegmentSizeBytes {
+		if mustSync {
+			return w.sync()
+		}
+		return nil
+	}
+
+	return w.cut()
+}
+
+func (w *WAL) SaveSnapshot(e walpb.Snapshot) error {
+	b := pbutil.MustMarshal(&e)
+
+	w.mu.Lock()
+	defer w.mu.Unlock()
+
+	rec := &walpb.Record{Type: snapshotType, Data: b}
+	if err := w.encoder.encode(rec); err != nil {
+		return err
+	}
+	// update enti only when snapshot is ahead of last index
+	if w.enti < e.Index {
+		w.enti = e.Index
+	}
+	return w.sync()
+}
+
+func (w *WAL) saveCrc(prevCrc uint32) error {
+	return w.encoder.encode(&walpb.Record{Type: crcType, Crc: prevCrc})
+}
+
+func (w *WAL) tail() *fileutil.LockedFile {
+	if len(w.locks) > 0 {
+		return w.locks[len(w.locks)-1]
+	}
+	return nil
+}
+
+func (w *WAL) seq() uint64 {
+	t := w.tail()
+	if t == nil {
+		return 0
+	}
+	seq, _, err := parseWALName(filepath.Base(t.Name()))
+	if err != nil {
+		if w.lg != nil {
+			w.lg.Fatal("failed to parse WAL name", zap.String("name", t.Name()), zap.Error(err))
+		} else {
+			plog.Fatalf("bad wal name %s (%v)", t.Name(), err)
+		}
+	}
+	return seq
+}
+
+func closeAll(rcs ...io.ReadCloser) error {
+	for _, f := range rcs {
+		if err := f.Close(); err != nil {
+			return err
+		}
+	}
+	return nil
+}
diff --git a/vendor/go.etcd.io/etcd/wal/walpb/record.go b/vendor/go.etcd.io/etcd/wal/walpb/record.go
new file mode 100644
index 0000000..30a05e0
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/wal/walpb/record.go
@@ -0,0 +1,29 @@
+// Copyright 2015 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 walpb
+
+import "errors"
+
+var (
+	ErrCRCMismatch = errors.New("walpb: crc mismatch")
+)
+
+func (rec *Record) Validate(crc uint32) error {
+	if rec.Crc == crc {
+		return nil
+	}
+	rec.Reset()
+	return ErrCRCMismatch
+}
diff --git a/vendor/go.etcd.io/etcd/wal/walpb/record.pb.go b/vendor/go.etcd.io/etcd/wal/walpb/record.pb.go
new file mode 100644
index 0000000..3ce63dd
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/wal/walpb/record.pb.go
@@ -0,0 +1,504 @@
+// Code generated by protoc-gen-gogo. DO NOT EDIT.
+// source: record.proto
+
+/*
+	Package walpb is a generated protocol buffer package.
+
+	It is generated from these files:
+		record.proto
+
+	It has these top-level messages:
+		Record
+		Snapshot
+*/
+package walpb
+
+import (
+	"fmt"
+
+	proto "github.com/golang/protobuf/proto"
+
+	math "math"
+
+	_ "github.com/gogo/protobuf/gogoproto"
+
+	io "io"
+)
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
+
+type Record struct {
+	Type             int64  `protobuf:"varint,1,opt,name=type" json:"type"`
+	Crc              uint32 `protobuf:"varint,2,opt,name=crc" json:"crc"`
+	Data             []byte `protobuf:"bytes,3,opt,name=data" json:"data,omitempty"`
+	XXX_unrecognized []byte `json:"-"`
+}
+
+func (m *Record) Reset()                    { *m = Record{} }
+func (m *Record) String() string            { return proto.CompactTextString(m) }
+func (*Record) ProtoMessage()               {}
+func (*Record) Descriptor() ([]byte, []int) { return fileDescriptorRecord, []int{0} }
+
+type Snapshot struct {
+	Index            uint64 `protobuf:"varint,1,opt,name=index" json:"index"`
+	Term             uint64 `protobuf:"varint,2,opt,name=term" json:"term"`
+	XXX_unrecognized []byte `json:"-"`
+}
+
+func (m *Snapshot) Reset()                    { *m = Snapshot{} }
+func (m *Snapshot) String() string            { return proto.CompactTextString(m) }
+func (*Snapshot) ProtoMessage()               {}
+func (*Snapshot) Descriptor() ([]byte, []int) { return fileDescriptorRecord, []int{1} }
+
+func init() {
+	proto.RegisterType((*Record)(nil), "walpb.Record")
+	proto.RegisterType((*Snapshot)(nil), "walpb.Snapshot")
+}
+func (m *Record) 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 *Record) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	dAtA[i] = 0x8
+	i++
+	i = encodeVarintRecord(dAtA, i, uint64(m.Type))
+	dAtA[i] = 0x10
+	i++
+	i = encodeVarintRecord(dAtA, i, uint64(m.Crc))
+	if m.Data != nil {
+		dAtA[i] = 0x1a
+		i++
+		i = encodeVarintRecord(dAtA, i, uint64(len(m.Data)))
+		i += copy(dAtA[i:], m.Data)
+	}
+	if m.XXX_unrecognized != nil {
+		i += copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	return i, nil
+}
+
+func (m *Snapshot) 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 *Snapshot) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	dAtA[i] = 0x8
+	i++
+	i = encodeVarintRecord(dAtA, i, uint64(m.Index))
+	dAtA[i] = 0x10
+	i++
+	i = encodeVarintRecord(dAtA, i, uint64(m.Term))
+	if m.XXX_unrecognized != nil {
+		i += copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	return i, nil
+}
+
+func encodeVarintRecord(dAtA []byte, offset int, v uint64) int {
+	for v >= 1<<7 {
+		dAtA[offset] = uint8(v&0x7f | 0x80)
+		v >>= 7
+		offset++
+	}
+	dAtA[offset] = uint8(v)
+	return offset + 1
+}
+func (m *Record) Size() (n int) {
+	var l int
+	_ = l
+	n += 1 + sovRecord(uint64(m.Type))
+	n += 1 + sovRecord(uint64(m.Crc))
+	if m.Data != nil {
+		l = len(m.Data)
+		n += 1 + l + sovRecord(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *Snapshot) Size() (n int) {
+	var l int
+	_ = l
+	n += 1 + sovRecord(uint64(m.Index))
+	n += 1 + sovRecord(uint64(m.Term))
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func sovRecord(x uint64) (n int) {
+	for {
+		n++
+		x >>= 7
+		if x == 0 {
+			break
+		}
+	}
+	return n
+}
+func sozRecord(x uint64) (n int) {
+	return sovRecord(uint64((x << 1) ^ uint64((int64(x) >> 63))))
+}
+func (m *Record) 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 ErrIntOverflowRecord
+			}
+			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: Record: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: Record: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType)
+			}
+			m.Type = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRecord
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Type |= (int64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Crc", wireType)
+			}
+			m.Crc = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRecord
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Crc |= (uint32(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRecord
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthRecord
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Data = append(m.Data[:0], dAtA[iNdEx:postIndex]...)
+			if m.Data == nil {
+				m.Data = []byte{}
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRecord(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRecord
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...)
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *Snapshot) 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 ErrIntOverflowRecord
+			}
+			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: Snapshot: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: Snapshot: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType)
+			}
+			m.Index = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRecord
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Index |= (uint64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Term", wireType)
+			}
+			m.Term = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRecord
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Term |= (uint64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRecord(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRecord
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...)
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func skipRecord(dAtA []byte) (n int, err error) {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return 0, ErrIntOverflowRecord
+			}
+			if iNdEx >= l {
+				return 0, io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= (uint64(b) & 0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		wireType := int(wire & 0x7)
+		switch wireType {
+		case 0:
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return 0, ErrIntOverflowRecord
+				}
+				if iNdEx >= l {
+					return 0, io.ErrUnexpectedEOF
+				}
+				iNdEx++
+				if dAtA[iNdEx-1] < 0x80 {
+					break
+				}
+			}
+			return iNdEx, nil
+		case 1:
+			iNdEx += 8
+			return iNdEx, nil
+		case 2:
+			var length int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return 0, ErrIntOverflowRecord
+				}
+				if iNdEx >= l {
+					return 0, io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				length |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			iNdEx += length
+			if length < 0 {
+				return 0, ErrInvalidLengthRecord
+			}
+			return iNdEx, nil
+		case 3:
+			for {
+				var innerWire uint64
+				var start int = iNdEx
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return 0, ErrIntOverflowRecord
+					}
+					if iNdEx >= l {
+						return 0, io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					innerWire |= (uint64(b) & 0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				innerWireType := int(innerWire & 0x7)
+				if innerWireType == 4 {
+					break
+				}
+				next, err := skipRecord(dAtA[start:])
+				if err != nil {
+					return 0, err
+				}
+				iNdEx = start + next
+			}
+			return iNdEx, nil
+		case 4:
+			return iNdEx, nil
+		case 5:
+			iNdEx += 4
+			return iNdEx, nil
+		default:
+			return 0, fmt.Errorf("proto: illegal wireType %d", wireType)
+		}
+	}
+	panic("unreachable")
+}
+
+var (
+	ErrInvalidLengthRecord = fmt.Errorf("proto: negative length found during unmarshaling")
+	ErrIntOverflowRecord   = fmt.Errorf("proto: integer overflow")
+)
+
+func init() { proto.RegisterFile("record.proto", fileDescriptorRecord) }
+
+var fileDescriptorRecord = []byte{
+	// 186 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0xe2, 0x29, 0x4a, 0x4d, 0xce,
+	0x2f, 0x4a, 0xd1, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x62, 0x2d, 0x4f, 0xcc, 0x29, 0x48, 0x92,
+	0x12, 0x49, 0xcf, 0x4f, 0xcf, 0x07, 0x8b, 0xe8, 0x83, 0x58, 0x10, 0x49, 0x25, 0x3f, 0x2e, 0xb6,
+	0x20, 0xb0, 0x62, 0x21, 0x09, 0x2e, 0x96, 0x92, 0xca, 0x82, 0x54, 0x09, 0x46, 0x05, 0x46, 0x0d,
+	0x66, 0x27, 0x96, 0x13, 0xf7, 0xe4, 0x19, 0x82, 0xc0, 0x22, 0x42, 0x62, 0x5c, 0xcc, 0xc9, 0x45,
+	0xc9, 0x12, 0x4c, 0x0a, 0x8c, 0x1a, 0xbc, 0x50, 0x09, 0x90, 0x80, 0x90, 0x10, 0x17, 0x4b, 0x4a,
+	0x62, 0x49, 0xa2, 0x04, 0xb3, 0x02, 0xa3, 0x06, 0x4f, 0x10, 0x98, 0xad, 0xe4, 0xc0, 0xc5, 0x11,
+	0x9c, 0x97, 0x58, 0x50, 0x9c, 0x91, 0x5f, 0x22, 0x24, 0xc5, 0xc5, 0x9a, 0x99, 0x97, 0x92, 0x5a,
+	0x01, 0x36, 0x92, 0x05, 0xaa, 0x13, 0x22, 0x04, 0xb6, 0x2d, 0xb5, 0x28, 0x17, 0x6c, 0x28, 0x0b,
+	0xdc, 0xb6, 0xd4, 0xa2, 0x5c, 0x27, 0x91, 0x13, 0x0f, 0xe5, 0x18, 0x4e, 0x3c, 0x92, 0x63, 0xbc,
+	0xf0, 0x48, 0x8e, 0xf1, 0xc1, 0x23, 0x39, 0xc6, 0x19, 0x8f, 0xe5, 0x18, 0x00, 0x01, 0x00, 0x00,
+	0xff, 0xff, 0x7f, 0x5e, 0x5c, 0x46, 0xd3, 0x00, 0x00, 0x00,
+}
diff --git a/vendor/go.etcd.io/etcd/wal/walpb/record.proto b/vendor/go.etcd.io/etcd/wal/walpb/record.proto
new file mode 100644
index 0000000..b694cb2
--- /dev/null
+++ b/vendor/go.etcd.io/etcd/wal/walpb/record.proto
@@ -0,0 +1,20 @@
+syntax = "proto2";
+package walpb;
+
+import "gogoproto/gogo.proto";
+
+option (gogoproto.marshaler_all) = true;
+option (gogoproto.sizer_all) = true;
+option (gogoproto.unmarshaler_all) = true;
+option (gogoproto.goproto_getters_all) = false;
+
+message Record {
+	optional int64 type  = 1 [(gogoproto.nullable) = false];
+	optional uint32 crc  = 2 [(gogoproto.nullable) = false];
+	optional bytes data  = 3;
+}
+
+message Snapshot {
+	optional uint64 index = 1 [(gogoproto.nullable) = false];
+	optional uint64 term  = 2 [(gogoproto.nullable) = false];
+}
diff --git a/vendor/golang.org/x/crypto/bcrypt/base64.go b/vendor/golang.org/x/crypto/bcrypt/base64.go
new file mode 100644
index 0000000..fc31160
--- /dev/null
+++ b/vendor/golang.org/x/crypto/bcrypt/base64.go
@@ -0,0 +1,35 @@
+// Copyright 2011 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package bcrypt
+
+import "encoding/base64"
+
+const alphabet = "./ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789"
+
+var bcEncoding = base64.NewEncoding(alphabet)
+
+func base64Encode(src []byte) []byte {
+	n := bcEncoding.EncodedLen(len(src))
+	dst := make([]byte, n)
+	bcEncoding.Encode(dst, src)
+	for dst[n-1] == '=' {
+		n--
+	}
+	return dst[:n]
+}
+
+func base64Decode(src []byte) ([]byte, error) {
+	numOfEquals := 4 - (len(src) % 4)
+	for i := 0; i < numOfEquals; i++ {
+		src = append(src, '=')
+	}
+
+	dst := make([]byte, bcEncoding.DecodedLen(len(src)))
+	n, err := bcEncoding.Decode(dst, src)
+	if err != nil {
+		return nil, err
+	}
+	return dst[:n], nil
+}
diff --git a/vendor/golang.org/x/crypto/bcrypt/bcrypt.go b/vendor/golang.org/x/crypto/bcrypt/bcrypt.go
new file mode 100644
index 0000000..aeb73f8
--- /dev/null
+++ b/vendor/golang.org/x/crypto/bcrypt/bcrypt.go
@@ -0,0 +1,295 @@
+// Copyright 2011 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// Package bcrypt implements Provos and Mazières's bcrypt adaptive hashing
+// algorithm. See http://www.usenix.org/event/usenix99/provos/provos.pdf
+package bcrypt // import "golang.org/x/crypto/bcrypt"
+
+// The code is a port of Provos and Mazières's C implementation.
+import (
+	"crypto/rand"
+	"crypto/subtle"
+	"errors"
+	"fmt"
+	"io"
+	"strconv"
+
+	"golang.org/x/crypto/blowfish"
+)
+
+const (
+	MinCost     int = 4  // the minimum allowable cost as passed in to GenerateFromPassword
+	MaxCost     int = 31 // the maximum allowable cost as passed in to GenerateFromPassword
+	DefaultCost int = 10 // the cost that will actually be set if a cost below MinCost is passed into GenerateFromPassword
+)
+
+// The error returned from CompareHashAndPassword when a password and hash do
+// not match.
+var ErrMismatchedHashAndPassword = errors.New("crypto/bcrypt: hashedPassword is not the hash of the given password")
+
+// The error returned from CompareHashAndPassword when a hash is too short to
+// be a bcrypt hash.
+var ErrHashTooShort = errors.New("crypto/bcrypt: hashedSecret too short to be a bcrypted password")
+
+// The error returned from CompareHashAndPassword when a hash was created with
+// a bcrypt algorithm newer than this implementation.
+type HashVersionTooNewError byte
+
+func (hv HashVersionTooNewError) Error() string {
+	return fmt.Sprintf("crypto/bcrypt: bcrypt algorithm version '%c' requested is newer than current version '%c'", byte(hv), majorVersion)
+}
+
+// The error returned from CompareHashAndPassword when a hash starts with something other than '$'
+type InvalidHashPrefixError byte
+
+func (ih InvalidHashPrefixError) Error() string {
+	return fmt.Sprintf("crypto/bcrypt: bcrypt hashes must start with '$', but hashedSecret started with '%c'", byte(ih))
+}
+
+type InvalidCostError int
+
+func (ic InvalidCostError) Error() string {
+	return fmt.Sprintf("crypto/bcrypt: cost %d is outside allowed range (%d,%d)", int(ic), int(MinCost), int(MaxCost))
+}
+
+const (
+	majorVersion       = '2'
+	minorVersion       = 'a'
+	maxSaltSize        = 16
+	maxCryptedHashSize = 23
+	encodedSaltSize    = 22
+	encodedHashSize    = 31
+	minHashSize        = 59
+)
+
+// magicCipherData is an IV for the 64 Blowfish encryption calls in
+// bcrypt(). It's the string "OrpheanBeholderScryDoubt" in big-endian bytes.
+var magicCipherData = []byte{
+	0x4f, 0x72, 0x70, 0x68,
+	0x65, 0x61, 0x6e, 0x42,
+	0x65, 0x68, 0x6f, 0x6c,
+	0x64, 0x65, 0x72, 0x53,
+	0x63, 0x72, 0x79, 0x44,
+	0x6f, 0x75, 0x62, 0x74,
+}
+
+type hashed struct {
+	hash  []byte
+	salt  []byte
+	cost  int // allowed range is MinCost to MaxCost
+	major byte
+	minor byte
+}
+
+// GenerateFromPassword returns the bcrypt hash of the password at the given
+// cost. If the cost given is less than MinCost, the cost will be set to
+// DefaultCost, instead. Use CompareHashAndPassword, as defined in this package,
+// to compare the returned hashed password with its cleartext version.
+func GenerateFromPassword(password []byte, cost int) ([]byte, error) {
+	p, err := newFromPassword(password, cost)
+	if err != nil {
+		return nil, err
+	}
+	return p.Hash(), nil
+}
+
+// CompareHashAndPassword compares a bcrypt hashed password with its possible
+// plaintext equivalent. Returns nil on success, or an error on failure.
+func CompareHashAndPassword(hashedPassword, password []byte) error {
+	p, err := newFromHash(hashedPassword)
+	if err != nil {
+		return err
+	}
+
+	otherHash, err := bcrypt(password, p.cost, p.salt)
+	if err != nil {
+		return err
+	}
+
+	otherP := &hashed{otherHash, p.salt, p.cost, p.major, p.minor}
+	if subtle.ConstantTimeCompare(p.Hash(), otherP.Hash()) == 1 {
+		return nil
+	}
+
+	return ErrMismatchedHashAndPassword
+}
+
+// Cost returns the hashing cost used to create the given hashed
+// password. When, in the future, the hashing cost of a password system needs
+// to be increased in order to adjust for greater computational power, this
+// function allows one to establish which passwords need to be updated.
+func Cost(hashedPassword []byte) (int, error) {
+	p, err := newFromHash(hashedPassword)
+	if err != nil {
+		return 0, err
+	}
+	return p.cost, nil
+}
+
+func newFromPassword(password []byte, cost int) (*hashed, error) {
+	if cost < MinCost {
+		cost = DefaultCost
+	}
+	p := new(hashed)
+	p.major = majorVersion
+	p.minor = minorVersion
+
+	err := checkCost(cost)
+	if err != nil {
+		return nil, err
+	}
+	p.cost = cost
+
+	unencodedSalt := make([]byte, maxSaltSize)
+	_, err = io.ReadFull(rand.Reader, unencodedSalt)
+	if err != nil {
+		return nil, err
+	}
+
+	p.salt = base64Encode(unencodedSalt)
+	hash, err := bcrypt(password, p.cost, p.salt)
+	if err != nil {
+		return nil, err
+	}
+	p.hash = hash
+	return p, err
+}
+
+func newFromHash(hashedSecret []byte) (*hashed, error) {
+	if len(hashedSecret) < minHashSize {
+		return nil, ErrHashTooShort
+	}
+	p := new(hashed)
+	n, err := p.decodeVersion(hashedSecret)
+	if err != nil {
+		return nil, err
+	}
+	hashedSecret = hashedSecret[n:]
+	n, err = p.decodeCost(hashedSecret)
+	if err != nil {
+		return nil, err
+	}
+	hashedSecret = hashedSecret[n:]
+
+	// The "+2" is here because we'll have to append at most 2 '=' to the salt
+	// when base64 decoding it in expensiveBlowfishSetup().
+	p.salt = make([]byte, encodedSaltSize, encodedSaltSize+2)
+	copy(p.salt, hashedSecret[:encodedSaltSize])
+
+	hashedSecret = hashedSecret[encodedSaltSize:]
+	p.hash = make([]byte, len(hashedSecret))
+	copy(p.hash, hashedSecret)
+
+	return p, nil
+}
+
+func bcrypt(password []byte, cost int, salt []byte) ([]byte, error) {
+	cipherData := make([]byte, len(magicCipherData))
+	copy(cipherData, magicCipherData)
+
+	c, err := expensiveBlowfishSetup(password, uint32(cost), salt)
+	if err != nil {
+		return nil, err
+	}
+
+	for i := 0; i < 24; i += 8 {
+		for j := 0; j < 64; j++ {
+			c.Encrypt(cipherData[i:i+8], cipherData[i:i+8])
+		}
+	}
+
+	// Bug compatibility with C bcrypt implementations. We only encode 23 of
+	// the 24 bytes encrypted.
+	hsh := base64Encode(cipherData[:maxCryptedHashSize])
+	return hsh, nil
+}
+
+func expensiveBlowfishSetup(key []byte, cost uint32, salt []byte) (*blowfish.Cipher, error) {
+	csalt, err := base64Decode(salt)
+	if err != nil {
+		return nil, err
+	}
+
+	// Bug compatibility with C bcrypt implementations. They use the trailing
+	// NULL in the key string during expansion.
+	// We copy the key to prevent changing the underlying array.
+	ckey := append(key[:len(key):len(key)], 0)
+
+	c, err := blowfish.NewSaltedCipher(ckey, csalt)
+	if err != nil {
+		return nil, err
+	}
+
+	var i, rounds uint64
+	rounds = 1 << cost
+	for i = 0; i < rounds; i++ {
+		blowfish.ExpandKey(ckey, c)
+		blowfish.ExpandKey(csalt, c)
+	}
+
+	return c, nil
+}
+
+func (p *hashed) Hash() []byte {
+	arr := make([]byte, 60)
+	arr[0] = '$'
+	arr[1] = p.major
+	n := 2
+	if p.minor != 0 {
+		arr[2] = p.minor
+		n = 3
+	}
+	arr[n] = '$'
+	n++
+	copy(arr[n:], []byte(fmt.Sprintf("%02d", p.cost)))
+	n += 2
+	arr[n] = '$'
+	n++
+	copy(arr[n:], p.salt)
+	n += encodedSaltSize
+	copy(arr[n:], p.hash)
+	n += encodedHashSize
+	return arr[:n]
+}
+
+func (p *hashed) decodeVersion(sbytes []byte) (int, error) {
+	if sbytes[0] != '$' {
+		return -1, InvalidHashPrefixError(sbytes[0])
+	}
+	if sbytes[1] > majorVersion {
+		return -1, HashVersionTooNewError(sbytes[1])
+	}
+	p.major = sbytes[1]
+	n := 3
+	if sbytes[2] != '$' {
+		p.minor = sbytes[2]
+		n++
+	}
+	return n, nil
+}
+
+// sbytes should begin where decodeVersion left off.
+func (p *hashed) decodeCost(sbytes []byte) (int, error) {
+	cost, err := strconv.Atoi(string(sbytes[0:2]))
+	if err != nil {
+		return -1, err
+	}
+	err = checkCost(cost)
+	if err != nil {
+		return -1, err
+	}
+	p.cost = cost
+	return 3, nil
+}
+
+func (p *hashed) String() string {
+	return fmt.Sprintf("&{hash: %#v, salt: %#v, cost: %d, major: %c, minor: %c}", string(p.hash), p.salt, p.cost, p.major, p.minor)
+}
+
+func checkCost(cost int) error {
+	if cost < MinCost || cost > MaxCost {
+		return InvalidCostError(cost)
+	}
+	return nil
+}
diff --git a/vendor/golang.org/x/crypto/blowfish/block.go b/vendor/golang.org/x/crypto/blowfish/block.go
new file mode 100644
index 0000000..9d80f19
--- /dev/null
+++ b/vendor/golang.org/x/crypto/blowfish/block.go
@@ -0,0 +1,159 @@
+// Copyright 2010 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package blowfish
+
+// getNextWord returns the next big-endian uint32 value from the byte slice
+// at the given position in a circular manner, updating the position.
+func getNextWord(b []byte, pos *int) uint32 {
+	var w uint32
+	j := *pos
+	for i := 0; i < 4; i++ {
+		w = w<<8 | uint32(b[j])
+		j++
+		if j >= len(b) {
+			j = 0
+		}
+	}
+	*pos = j
+	return w
+}
+
+// ExpandKey performs a key expansion on the given *Cipher. Specifically, it
+// performs the Blowfish algorithm's key schedule which sets up the *Cipher's
+// pi and substitution tables for calls to Encrypt. This is used, primarily,
+// by the bcrypt package to reuse the Blowfish key schedule during its
+// set up. It's unlikely that you need to use this directly.
+func ExpandKey(key []byte, c *Cipher) {
+	j := 0
+	for i := 0; i < 18; i++ {
+		// Using inlined getNextWord for performance.
+		var d uint32
+		for k := 0; k < 4; k++ {
+			d = d<<8 | uint32(key[j])
+			j++
+			if j >= len(key) {
+				j = 0
+			}
+		}
+		c.p[i] ^= d
+	}
+
+	var l, r uint32
+	for i := 0; i < 18; i += 2 {
+		l, r = encryptBlock(l, r, c)
+		c.p[i], c.p[i+1] = l, r
+	}
+
+	for i := 0; i < 256; i += 2 {
+		l, r = encryptBlock(l, r, c)
+		c.s0[i], c.s0[i+1] = l, r
+	}
+	for i := 0; i < 256; i += 2 {
+		l, r = encryptBlock(l, r, c)
+		c.s1[i], c.s1[i+1] = l, r
+	}
+	for i := 0; i < 256; i += 2 {
+		l, r = encryptBlock(l, r, c)
+		c.s2[i], c.s2[i+1] = l, r
+	}
+	for i := 0; i < 256; i += 2 {
+		l, r = encryptBlock(l, r, c)
+		c.s3[i], c.s3[i+1] = l, r
+	}
+}
+
+// This is similar to ExpandKey, but folds the salt during the key
+// schedule. While ExpandKey is essentially expandKeyWithSalt with an all-zero
+// salt passed in, reusing ExpandKey turns out to be a place of inefficiency
+// and specializing it here is useful.
+func expandKeyWithSalt(key []byte, salt []byte, c *Cipher) {
+	j := 0
+	for i := 0; i < 18; i++ {
+		c.p[i] ^= getNextWord(key, &j)
+	}
+
+	j = 0
+	var l, r uint32
+	for i := 0; i < 18; i += 2 {
+		l ^= getNextWord(salt, &j)
+		r ^= getNextWord(salt, &j)
+		l, r = encryptBlock(l, r, c)
+		c.p[i], c.p[i+1] = l, r
+	}
+
+	for i := 0; i < 256; i += 2 {
+		l ^= getNextWord(salt, &j)
+		r ^= getNextWord(salt, &j)
+		l, r = encryptBlock(l, r, c)
+		c.s0[i], c.s0[i+1] = l, r
+	}
+
+	for i := 0; i < 256; i += 2 {
+		l ^= getNextWord(salt, &j)
+		r ^= getNextWord(salt, &j)
+		l, r = encryptBlock(l, r, c)
+		c.s1[i], c.s1[i+1] = l, r
+	}
+
+	for i := 0; i < 256; i += 2 {
+		l ^= getNextWord(salt, &j)
+		r ^= getNextWord(salt, &j)
+		l, r = encryptBlock(l, r, c)
+		c.s2[i], c.s2[i+1] = l, r
+	}
+
+	for i := 0; i < 256; i += 2 {
+		l ^= getNextWord(salt, &j)
+		r ^= getNextWord(salt, &j)
+		l, r = encryptBlock(l, r, c)
+		c.s3[i], c.s3[i+1] = l, r
+	}
+}
+
+func encryptBlock(l, r uint32, c *Cipher) (uint32, uint32) {
+	xl, xr := l, r
+	xl ^= c.p[0]
+	xr ^= ((c.s0[byte(xl>>24)] + c.s1[byte(xl>>16)]) ^ c.s2[byte(xl>>8)]) + c.s3[byte(xl)] ^ c.p[1]
+	xl ^= ((c.s0[byte(xr>>24)] + c.s1[byte(xr>>16)]) ^ c.s2[byte(xr>>8)]) + c.s3[byte(xr)] ^ c.p[2]
+	xr ^= ((c.s0[byte(xl>>24)] + c.s1[byte(xl>>16)]) ^ c.s2[byte(xl>>8)]) + c.s3[byte(xl)] ^ c.p[3]
+	xl ^= ((c.s0[byte(xr>>24)] + c.s1[byte(xr>>16)]) ^ c.s2[byte(xr>>8)]) + c.s3[byte(xr)] ^ c.p[4]
+	xr ^= ((c.s0[byte(xl>>24)] + c.s1[byte(xl>>16)]) ^ c.s2[byte(xl>>8)]) + c.s3[byte(xl)] ^ c.p[5]
+	xl ^= ((c.s0[byte(xr>>24)] + c.s1[byte(xr>>16)]) ^ c.s2[byte(xr>>8)]) + c.s3[byte(xr)] ^ c.p[6]
+	xr ^= ((c.s0[byte(xl>>24)] + c.s1[byte(xl>>16)]) ^ c.s2[byte(xl>>8)]) + c.s3[byte(xl)] ^ c.p[7]
+	xl ^= ((c.s0[byte(xr>>24)] + c.s1[byte(xr>>16)]) ^ c.s2[byte(xr>>8)]) + c.s3[byte(xr)] ^ c.p[8]
+	xr ^= ((c.s0[byte(xl>>24)] + c.s1[byte(xl>>16)]) ^ c.s2[byte(xl>>8)]) + c.s3[byte(xl)] ^ c.p[9]
+	xl ^= ((c.s0[byte(xr>>24)] + c.s1[byte(xr>>16)]) ^ c.s2[byte(xr>>8)]) + c.s3[byte(xr)] ^ c.p[10]
+	xr ^= ((c.s0[byte(xl>>24)] + c.s1[byte(xl>>16)]) ^ c.s2[byte(xl>>8)]) + c.s3[byte(xl)] ^ c.p[11]
+	xl ^= ((c.s0[byte(xr>>24)] + c.s1[byte(xr>>16)]) ^ c.s2[byte(xr>>8)]) + c.s3[byte(xr)] ^ c.p[12]
+	xr ^= ((c.s0[byte(xl>>24)] + c.s1[byte(xl>>16)]) ^ c.s2[byte(xl>>8)]) + c.s3[byte(xl)] ^ c.p[13]
+	xl ^= ((c.s0[byte(xr>>24)] + c.s1[byte(xr>>16)]) ^ c.s2[byte(xr>>8)]) + c.s3[byte(xr)] ^ c.p[14]
+	xr ^= ((c.s0[byte(xl>>24)] + c.s1[byte(xl>>16)]) ^ c.s2[byte(xl>>8)]) + c.s3[byte(xl)] ^ c.p[15]
+	xl ^= ((c.s0[byte(xr>>24)] + c.s1[byte(xr>>16)]) ^ c.s2[byte(xr>>8)]) + c.s3[byte(xr)] ^ c.p[16]
+	xr ^= c.p[17]
+	return xr, xl
+}
+
+func decryptBlock(l, r uint32, c *Cipher) (uint32, uint32) {
+	xl, xr := l, r
+	xl ^= c.p[17]
+	xr ^= ((c.s0[byte(xl>>24)] + c.s1[byte(xl>>16)]) ^ c.s2[byte(xl>>8)]) + c.s3[byte(xl)] ^ c.p[16]
+	xl ^= ((c.s0[byte(xr>>24)] + c.s1[byte(xr>>16)]) ^ c.s2[byte(xr>>8)]) + c.s3[byte(xr)] ^ c.p[15]
+	xr ^= ((c.s0[byte(xl>>24)] + c.s1[byte(xl>>16)]) ^ c.s2[byte(xl>>8)]) + c.s3[byte(xl)] ^ c.p[14]
+	xl ^= ((c.s0[byte(xr>>24)] + c.s1[byte(xr>>16)]) ^ c.s2[byte(xr>>8)]) + c.s3[byte(xr)] ^ c.p[13]
+	xr ^= ((c.s0[byte(xl>>24)] + c.s1[byte(xl>>16)]) ^ c.s2[byte(xl>>8)]) + c.s3[byte(xl)] ^ c.p[12]
+	xl ^= ((c.s0[byte(xr>>24)] + c.s1[byte(xr>>16)]) ^ c.s2[byte(xr>>8)]) + c.s3[byte(xr)] ^ c.p[11]
+	xr ^= ((c.s0[byte(xl>>24)] + c.s1[byte(xl>>16)]) ^ c.s2[byte(xl>>8)]) + c.s3[byte(xl)] ^ c.p[10]
+	xl ^= ((c.s0[byte(xr>>24)] + c.s1[byte(xr>>16)]) ^ c.s2[byte(xr>>8)]) + c.s3[byte(xr)] ^ c.p[9]
+	xr ^= ((c.s0[byte(xl>>24)] + c.s1[byte(xl>>16)]) ^ c.s2[byte(xl>>8)]) + c.s3[byte(xl)] ^ c.p[8]
+	xl ^= ((c.s0[byte(xr>>24)] + c.s1[byte(xr>>16)]) ^ c.s2[byte(xr>>8)]) + c.s3[byte(xr)] ^ c.p[7]
+	xr ^= ((c.s0[byte(xl>>24)] + c.s1[byte(xl>>16)]) ^ c.s2[byte(xl>>8)]) + c.s3[byte(xl)] ^ c.p[6]
+	xl ^= ((c.s0[byte(xr>>24)] + c.s1[byte(xr>>16)]) ^ c.s2[byte(xr>>8)]) + c.s3[byte(xr)] ^ c.p[5]
+	xr ^= ((c.s0[byte(xl>>24)] + c.s1[byte(xl>>16)]) ^ c.s2[byte(xl>>8)]) + c.s3[byte(xl)] ^ c.p[4]
+	xl ^= ((c.s0[byte(xr>>24)] + c.s1[byte(xr>>16)]) ^ c.s2[byte(xr>>8)]) + c.s3[byte(xr)] ^ c.p[3]
+	xr ^= ((c.s0[byte(xl>>24)] + c.s1[byte(xl>>16)]) ^ c.s2[byte(xl>>8)]) + c.s3[byte(xl)] ^ c.p[2]
+	xl ^= ((c.s0[byte(xr>>24)] + c.s1[byte(xr>>16)]) ^ c.s2[byte(xr>>8)]) + c.s3[byte(xr)] ^ c.p[1]
+	xr ^= c.p[0]
+	return xr, xl
+}
diff --git a/vendor/golang.org/x/crypto/blowfish/cipher.go b/vendor/golang.org/x/crypto/blowfish/cipher.go
new file mode 100644
index 0000000..213bf20
--- /dev/null
+++ b/vendor/golang.org/x/crypto/blowfish/cipher.go
@@ -0,0 +1,99 @@
+// Copyright 2010 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// Package blowfish implements Bruce Schneier's Blowfish encryption algorithm.
+//
+// Blowfish is a legacy cipher and its short block size makes it vulnerable to
+// birthday bound attacks (see https://sweet32.info). It should only be used
+// where compatibility with legacy systems, not security, is the goal.
+//
+// Deprecated: any new system should use AES (from crypto/aes, if necessary in
+// an AEAD mode like crypto/cipher.NewGCM) or XChaCha20-Poly1305 (from
+// golang.org/x/crypto/chacha20poly1305).
+package blowfish // import "golang.org/x/crypto/blowfish"
+
+// The code is a port of Bruce Schneier's C implementation.
+// See https://www.schneier.com/blowfish.html.
+
+import "strconv"
+
+// The Blowfish block size in bytes.
+const BlockSize = 8
+
+// A Cipher is an instance of Blowfish encryption using a particular key.
+type Cipher struct {
+	p              [18]uint32
+	s0, s1, s2, s3 [256]uint32
+}
+
+type KeySizeError int
+
+func (k KeySizeError) Error() string {
+	return "crypto/blowfish: invalid key size " + strconv.Itoa(int(k))
+}
+
+// NewCipher creates and returns a Cipher.
+// The key argument should be the Blowfish key, from 1 to 56 bytes.
+func NewCipher(key []byte) (*Cipher, error) {
+	var result Cipher
+	if k := len(key); k < 1 || k > 56 {
+		return nil, KeySizeError(k)
+	}
+	initCipher(&result)
+	ExpandKey(key, &result)
+	return &result, nil
+}
+
+// NewSaltedCipher creates a returns a Cipher that folds a salt into its key
+// schedule. For most purposes, NewCipher, instead of NewSaltedCipher, is
+// sufficient and desirable. For bcrypt compatibility, the key can be over 56
+// bytes.
+func NewSaltedCipher(key, salt []byte) (*Cipher, error) {
+	if len(salt) == 0 {
+		return NewCipher(key)
+	}
+	var result Cipher
+	if k := len(key); k < 1 {
+		return nil, KeySizeError(k)
+	}
+	initCipher(&result)
+	expandKeyWithSalt(key, salt, &result)
+	return &result, nil
+}
+
+// BlockSize returns the Blowfish block size, 8 bytes.
+// It is necessary to satisfy the Block interface in the
+// package "crypto/cipher".
+func (c *Cipher) BlockSize() int { return BlockSize }
+
+// Encrypt encrypts the 8-byte buffer src using the key k
+// and stores the result in dst.
+// Note that for amounts of data larger than a block,
+// it is not safe to just call Encrypt on successive blocks;
+// instead, use an encryption mode like CBC (see crypto/cipher/cbc.go).
+func (c *Cipher) Encrypt(dst, src []byte) {
+	l := uint32(src[0])<<24 | uint32(src[1])<<16 | uint32(src[2])<<8 | uint32(src[3])
+	r := uint32(src[4])<<24 | uint32(src[5])<<16 | uint32(src[6])<<8 | uint32(src[7])
+	l, r = encryptBlock(l, r, c)
+	dst[0], dst[1], dst[2], dst[3] = byte(l>>24), byte(l>>16), byte(l>>8), byte(l)
+	dst[4], dst[5], dst[6], dst[7] = byte(r>>24), byte(r>>16), byte(r>>8), byte(r)
+}
+
+// Decrypt decrypts the 8-byte buffer src using the key k
+// and stores the result in dst.
+func (c *Cipher) Decrypt(dst, src []byte) {
+	l := uint32(src[0])<<24 | uint32(src[1])<<16 | uint32(src[2])<<8 | uint32(src[3])
+	r := uint32(src[4])<<24 | uint32(src[5])<<16 | uint32(src[6])<<8 | uint32(src[7])
+	l, r = decryptBlock(l, r, c)
+	dst[0], dst[1], dst[2], dst[3] = byte(l>>24), byte(l>>16), byte(l>>8), byte(l)
+	dst[4], dst[5], dst[6], dst[7] = byte(r>>24), byte(r>>16), byte(r>>8), byte(r)
+}
+
+func initCipher(c *Cipher) {
+	copy(c.p[0:], p[0:])
+	copy(c.s0[0:], s0[0:])
+	copy(c.s1[0:], s1[0:])
+	copy(c.s2[0:], s2[0:])
+	copy(c.s3[0:], s3[0:])
+}
diff --git a/vendor/golang.org/x/crypto/blowfish/const.go b/vendor/golang.org/x/crypto/blowfish/const.go
new file mode 100644
index 0000000..d040775
--- /dev/null
+++ b/vendor/golang.org/x/crypto/blowfish/const.go
@@ -0,0 +1,199 @@
+// Copyright 2010 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// The startup permutation array and substitution boxes.
+// They are the hexadecimal digits of PI; see:
+// https://www.schneier.com/code/constants.txt.
+
+package blowfish
+
+var s0 = [256]uint32{
+	0xd1310ba6, 0x98dfb5ac, 0x2ffd72db, 0xd01adfb7, 0xb8e1afed, 0x6a267e96,
+	0xba7c9045, 0xf12c7f99, 0x24a19947, 0xb3916cf7, 0x0801f2e2, 0x858efc16,
+	0x636920d8, 0x71574e69, 0xa458fea3, 0xf4933d7e, 0x0d95748f, 0x728eb658,
+	0x718bcd58, 0x82154aee, 0x7b54a41d, 0xc25a59b5, 0x9c30d539, 0x2af26013,
+	0xc5d1b023, 0x286085f0, 0xca417918, 0xb8db38ef, 0x8e79dcb0, 0x603a180e,
+	0x6c9e0e8b, 0xb01e8a3e, 0xd71577c1, 0xbd314b27, 0x78af2fda, 0x55605c60,
+	0xe65525f3, 0xaa55ab94, 0x57489862, 0x63e81440, 0x55ca396a, 0x2aab10b6,
+	0xb4cc5c34, 0x1141e8ce, 0xa15486af, 0x7c72e993, 0xb3ee1411, 0x636fbc2a,
+	0x2ba9c55d, 0x741831f6, 0xce5c3e16, 0x9b87931e, 0xafd6ba33, 0x6c24cf5c,
+	0x7a325381, 0x28958677, 0x3b8f4898, 0x6b4bb9af, 0xc4bfe81b, 0x66282193,
+	0x61d809cc, 0xfb21a991, 0x487cac60, 0x5dec8032, 0xef845d5d, 0xe98575b1,
+	0xdc262302, 0xeb651b88, 0x23893e81, 0xd396acc5, 0x0f6d6ff3, 0x83f44239,
+	0x2e0b4482, 0xa4842004, 0x69c8f04a, 0x9e1f9b5e, 0x21c66842, 0xf6e96c9a,
+	0x670c9c61, 0xabd388f0, 0x6a51a0d2, 0xd8542f68, 0x960fa728, 0xab5133a3,
+	0x6eef0b6c, 0x137a3be4, 0xba3bf050, 0x7efb2a98, 0xa1f1651d, 0x39af0176,
+	0x66ca593e, 0x82430e88, 0x8cee8619, 0x456f9fb4, 0x7d84a5c3, 0x3b8b5ebe,
+	0xe06f75d8, 0x85c12073, 0x401a449f, 0x56c16aa6, 0x4ed3aa62, 0x363f7706,
+	0x1bfedf72, 0x429b023d, 0x37d0d724, 0xd00a1248, 0xdb0fead3, 0x49f1c09b,
+	0x075372c9, 0x80991b7b, 0x25d479d8, 0xf6e8def7, 0xe3fe501a, 0xb6794c3b,
+	0x976ce0bd, 0x04c006ba, 0xc1a94fb6, 0x409f60c4, 0x5e5c9ec2, 0x196a2463,
+	0x68fb6faf, 0x3e6c53b5, 0x1339b2eb, 0x3b52ec6f, 0x6dfc511f, 0x9b30952c,
+	0xcc814544, 0xaf5ebd09, 0xbee3d004, 0xde334afd, 0x660f2807, 0x192e4bb3,
+	0xc0cba857, 0x45c8740f, 0xd20b5f39, 0xb9d3fbdb, 0x5579c0bd, 0x1a60320a,
+	0xd6a100c6, 0x402c7279, 0x679f25fe, 0xfb1fa3cc, 0x8ea5e9f8, 0xdb3222f8,
+	0x3c7516df, 0xfd616b15, 0x2f501ec8, 0xad0552ab, 0x323db5fa, 0xfd238760,
+	0x53317b48, 0x3e00df82, 0x9e5c57bb, 0xca6f8ca0, 0x1a87562e, 0xdf1769db,
+	0xd542a8f6, 0x287effc3, 0xac6732c6, 0x8c4f5573, 0x695b27b0, 0xbbca58c8,
+	0xe1ffa35d, 0xb8f011a0, 0x10fa3d98, 0xfd2183b8, 0x4afcb56c, 0x2dd1d35b,
+	0x9a53e479, 0xb6f84565, 0xd28e49bc, 0x4bfb9790, 0xe1ddf2da, 0xa4cb7e33,
+	0x62fb1341, 0xcee4c6e8, 0xef20cada, 0x36774c01, 0xd07e9efe, 0x2bf11fb4,
+	0x95dbda4d, 0xae909198, 0xeaad8e71, 0x6b93d5a0, 0xd08ed1d0, 0xafc725e0,
+	0x8e3c5b2f, 0x8e7594b7, 0x8ff6e2fb, 0xf2122b64, 0x8888b812, 0x900df01c,
+	0x4fad5ea0, 0x688fc31c, 0xd1cff191, 0xb3a8c1ad, 0x2f2f2218, 0xbe0e1777,
+	0xea752dfe, 0x8b021fa1, 0xe5a0cc0f, 0xb56f74e8, 0x18acf3d6, 0xce89e299,
+	0xb4a84fe0, 0xfd13e0b7, 0x7cc43b81, 0xd2ada8d9, 0x165fa266, 0x80957705,
+	0x93cc7314, 0x211a1477, 0xe6ad2065, 0x77b5fa86, 0xc75442f5, 0xfb9d35cf,
+	0xebcdaf0c, 0x7b3e89a0, 0xd6411bd3, 0xae1e7e49, 0x00250e2d, 0x2071b35e,
+	0x226800bb, 0x57b8e0af, 0x2464369b, 0xf009b91e, 0x5563911d, 0x59dfa6aa,
+	0x78c14389, 0xd95a537f, 0x207d5ba2, 0x02e5b9c5, 0x83260376, 0x6295cfa9,
+	0x11c81968, 0x4e734a41, 0xb3472dca, 0x7b14a94a, 0x1b510052, 0x9a532915,
+	0xd60f573f, 0xbc9bc6e4, 0x2b60a476, 0x81e67400, 0x08ba6fb5, 0x571be91f,
+	0xf296ec6b, 0x2a0dd915, 0xb6636521, 0xe7b9f9b6, 0xff34052e, 0xc5855664,
+	0x53b02d5d, 0xa99f8fa1, 0x08ba4799, 0x6e85076a,
+}
+
+var s1 = [256]uint32{
+	0x4b7a70e9, 0xb5b32944, 0xdb75092e, 0xc4192623, 0xad6ea6b0, 0x49a7df7d,
+	0x9cee60b8, 0x8fedb266, 0xecaa8c71, 0x699a17ff, 0x5664526c, 0xc2b19ee1,
+	0x193602a5, 0x75094c29, 0xa0591340, 0xe4183a3e, 0x3f54989a, 0x5b429d65,
+	0x6b8fe4d6, 0x99f73fd6, 0xa1d29c07, 0xefe830f5, 0x4d2d38e6, 0xf0255dc1,
+	0x4cdd2086, 0x8470eb26, 0x6382e9c6, 0x021ecc5e, 0x09686b3f, 0x3ebaefc9,
+	0x3c971814, 0x6b6a70a1, 0x687f3584, 0x52a0e286, 0xb79c5305, 0xaa500737,
+	0x3e07841c, 0x7fdeae5c, 0x8e7d44ec, 0x5716f2b8, 0xb03ada37, 0xf0500c0d,
+	0xf01c1f04, 0x0200b3ff, 0xae0cf51a, 0x3cb574b2, 0x25837a58, 0xdc0921bd,
+	0xd19113f9, 0x7ca92ff6, 0x94324773, 0x22f54701, 0x3ae5e581, 0x37c2dadc,
+	0xc8b57634, 0x9af3dda7, 0xa9446146, 0x0fd0030e, 0xecc8c73e, 0xa4751e41,
+	0xe238cd99, 0x3bea0e2f, 0x3280bba1, 0x183eb331, 0x4e548b38, 0x4f6db908,
+	0x6f420d03, 0xf60a04bf, 0x2cb81290, 0x24977c79, 0x5679b072, 0xbcaf89af,
+	0xde9a771f, 0xd9930810, 0xb38bae12, 0xdccf3f2e, 0x5512721f, 0x2e6b7124,
+	0x501adde6, 0x9f84cd87, 0x7a584718, 0x7408da17, 0xbc9f9abc, 0xe94b7d8c,
+	0xec7aec3a, 0xdb851dfa, 0x63094366, 0xc464c3d2, 0xef1c1847, 0x3215d908,
+	0xdd433b37, 0x24c2ba16, 0x12a14d43, 0x2a65c451, 0x50940002, 0x133ae4dd,
+	0x71dff89e, 0x10314e55, 0x81ac77d6, 0x5f11199b, 0x043556f1, 0xd7a3c76b,
+	0x3c11183b, 0x5924a509, 0xf28fe6ed, 0x97f1fbfa, 0x9ebabf2c, 0x1e153c6e,
+	0x86e34570, 0xeae96fb1, 0x860e5e0a, 0x5a3e2ab3, 0x771fe71c, 0x4e3d06fa,
+	0x2965dcb9, 0x99e71d0f, 0x803e89d6, 0x5266c825, 0x2e4cc978, 0x9c10b36a,
+	0xc6150eba, 0x94e2ea78, 0xa5fc3c53, 0x1e0a2df4, 0xf2f74ea7, 0x361d2b3d,
+	0x1939260f, 0x19c27960, 0x5223a708, 0xf71312b6, 0xebadfe6e, 0xeac31f66,
+	0xe3bc4595, 0xa67bc883, 0xb17f37d1, 0x018cff28, 0xc332ddef, 0xbe6c5aa5,
+	0x65582185, 0x68ab9802, 0xeecea50f, 0xdb2f953b, 0x2aef7dad, 0x5b6e2f84,
+	0x1521b628, 0x29076170, 0xecdd4775, 0x619f1510, 0x13cca830, 0xeb61bd96,
+	0x0334fe1e, 0xaa0363cf, 0xb5735c90, 0x4c70a239, 0xd59e9e0b, 0xcbaade14,
+	0xeecc86bc, 0x60622ca7, 0x9cab5cab, 0xb2f3846e, 0x648b1eaf, 0x19bdf0ca,
+	0xa02369b9, 0x655abb50, 0x40685a32, 0x3c2ab4b3, 0x319ee9d5, 0xc021b8f7,
+	0x9b540b19, 0x875fa099, 0x95f7997e, 0x623d7da8, 0xf837889a, 0x97e32d77,
+	0x11ed935f, 0x16681281, 0x0e358829, 0xc7e61fd6, 0x96dedfa1, 0x7858ba99,
+	0x57f584a5, 0x1b227263, 0x9b83c3ff, 0x1ac24696, 0xcdb30aeb, 0x532e3054,
+	0x8fd948e4, 0x6dbc3128, 0x58ebf2ef, 0x34c6ffea, 0xfe28ed61, 0xee7c3c73,
+	0x5d4a14d9, 0xe864b7e3, 0x42105d14, 0x203e13e0, 0x45eee2b6, 0xa3aaabea,
+	0xdb6c4f15, 0xfacb4fd0, 0xc742f442, 0xef6abbb5, 0x654f3b1d, 0x41cd2105,
+	0xd81e799e, 0x86854dc7, 0xe44b476a, 0x3d816250, 0xcf62a1f2, 0x5b8d2646,
+	0xfc8883a0, 0xc1c7b6a3, 0x7f1524c3, 0x69cb7492, 0x47848a0b, 0x5692b285,
+	0x095bbf00, 0xad19489d, 0x1462b174, 0x23820e00, 0x58428d2a, 0x0c55f5ea,
+	0x1dadf43e, 0x233f7061, 0x3372f092, 0x8d937e41, 0xd65fecf1, 0x6c223bdb,
+	0x7cde3759, 0xcbee7460, 0x4085f2a7, 0xce77326e, 0xa6078084, 0x19f8509e,
+	0xe8efd855, 0x61d99735, 0xa969a7aa, 0xc50c06c2, 0x5a04abfc, 0x800bcadc,
+	0x9e447a2e, 0xc3453484, 0xfdd56705, 0x0e1e9ec9, 0xdb73dbd3, 0x105588cd,
+	0x675fda79, 0xe3674340, 0xc5c43465, 0x713e38d8, 0x3d28f89e, 0xf16dff20,
+	0x153e21e7, 0x8fb03d4a, 0xe6e39f2b, 0xdb83adf7,
+}
+
+var s2 = [256]uint32{
+	0xe93d5a68, 0x948140f7, 0xf64c261c, 0x94692934, 0x411520f7, 0x7602d4f7,
+	0xbcf46b2e, 0xd4a20068, 0xd4082471, 0x3320f46a, 0x43b7d4b7, 0x500061af,
+	0x1e39f62e, 0x97244546, 0x14214f74, 0xbf8b8840, 0x4d95fc1d, 0x96b591af,
+	0x70f4ddd3, 0x66a02f45, 0xbfbc09ec, 0x03bd9785, 0x7fac6dd0, 0x31cb8504,
+	0x96eb27b3, 0x55fd3941, 0xda2547e6, 0xabca0a9a, 0x28507825, 0x530429f4,
+	0x0a2c86da, 0xe9b66dfb, 0x68dc1462, 0xd7486900, 0x680ec0a4, 0x27a18dee,
+	0x4f3ffea2, 0xe887ad8c, 0xb58ce006, 0x7af4d6b6, 0xaace1e7c, 0xd3375fec,
+	0xce78a399, 0x406b2a42, 0x20fe9e35, 0xd9f385b9, 0xee39d7ab, 0x3b124e8b,
+	0x1dc9faf7, 0x4b6d1856, 0x26a36631, 0xeae397b2, 0x3a6efa74, 0xdd5b4332,
+	0x6841e7f7, 0xca7820fb, 0xfb0af54e, 0xd8feb397, 0x454056ac, 0xba489527,
+	0x55533a3a, 0x20838d87, 0xfe6ba9b7, 0xd096954b, 0x55a867bc, 0xa1159a58,
+	0xcca92963, 0x99e1db33, 0xa62a4a56, 0x3f3125f9, 0x5ef47e1c, 0x9029317c,
+	0xfdf8e802, 0x04272f70, 0x80bb155c, 0x05282ce3, 0x95c11548, 0xe4c66d22,
+	0x48c1133f, 0xc70f86dc, 0x07f9c9ee, 0x41041f0f, 0x404779a4, 0x5d886e17,
+	0x325f51eb, 0xd59bc0d1, 0xf2bcc18f, 0x41113564, 0x257b7834, 0x602a9c60,
+	0xdff8e8a3, 0x1f636c1b, 0x0e12b4c2, 0x02e1329e, 0xaf664fd1, 0xcad18115,
+	0x6b2395e0, 0x333e92e1, 0x3b240b62, 0xeebeb922, 0x85b2a20e, 0xe6ba0d99,
+	0xde720c8c, 0x2da2f728, 0xd0127845, 0x95b794fd, 0x647d0862, 0xe7ccf5f0,
+	0x5449a36f, 0x877d48fa, 0xc39dfd27, 0xf33e8d1e, 0x0a476341, 0x992eff74,
+	0x3a6f6eab, 0xf4f8fd37, 0xa812dc60, 0xa1ebddf8, 0x991be14c, 0xdb6e6b0d,
+	0xc67b5510, 0x6d672c37, 0x2765d43b, 0xdcd0e804, 0xf1290dc7, 0xcc00ffa3,
+	0xb5390f92, 0x690fed0b, 0x667b9ffb, 0xcedb7d9c, 0xa091cf0b, 0xd9155ea3,
+	0xbb132f88, 0x515bad24, 0x7b9479bf, 0x763bd6eb, 0x37392eb3, 0xcc115979,
+	0x8026e297, 0xf42e312d, 0x6842ada7, 0xc66a2b3b, 0x12754ccc, 0x782ef11c,
+	0x6a124237, 0xb79251e7, 0x06a1bbe6, 0x4bfb6350, 0x1a6b1018, 0x11caedfa,
+	0x3d25bdd8, 0xe2e1c3c9, 0x44421659, 0x0a121386, 0xd90cec6e, 0xd5abea2a,
+	0x64af674e, 0xda86a85f, 0xbebfe988, 0x64e4c3fe, 0x9dbc8057, 0xf0f7c086,
+	0x60787bf8, 0x6003604d, 0xd1fd8346, 0xf6381fb0, 0x7745ae04, 0xd736fccc,
+	0x83426b33, 0xf01eab71, 0xb0804187, 0x3c005e5f, 0x77a057be, 0xbde8ae24,
+	0x55464299, 0xbf582e61, 0x4e58f48f, 0xf2ddfda2, 0xf474ef38, 0x8789bdc2,
+	0x5366f9c3, 0xc8b38e74, 0xb475f255, 0x46fcd9b9, 0x7aeb2661, 0x8b1ddf84,
+	0x846a0e79, 0x915f95e2, 0x466e598e, 0x20b45770, 0x8cd55591, 0xc902de4c,
+	0xb90bace1, 0xbb8205d0, 0x11a86248, 0x7574a99e, 0xb77f19b6, 0xe0a9dc09,
+	0x662d09a1, 0xc4324633, 0xe85a1f02, 0x09f0be8c, 0x4a99a025, 0x1d6efe10,
+	0x1ab93d1d, 0x0ba5a4df, 0xa186f20f, 0x2868f169, 0xdcb7da83, 0x573906fe,
+	0xa1e2ce9b, 0x4fcd7f52, 0x50115e01, 0xa70683fa, 0xa002b5c4, 0x0de6d027,
+	0x9af88c27, 0x773f8641, 0xc3604c06, 0x61a806b5, 0xf0177a28, 0xc0f586e0,
+	0x006058aa, 0x30dc7d62, 0x11e69ed7, 0x2338ea63, 0x53c2dd94, 0xc2c21634,
+	0xbbcbee56, 0x90bcb6de, 0xebfc7da1, 0xce591d76, 0x6f05e409, 0x4b7c0188,
+	0x39720a3d, 0x7c927c24, 0x86e3725f, 0x724d9db9, 0x1ac15bb4, 0xd39eb8fc,
+	0xed545578, 0x08fca5b5, 0xd83d7cd3, 0x4dad0fc4, 0x1e50ef5e, 0xb161e6f8,
+	0xa28514d9, 0x6c51133c, 0x6fd5c7e7, 0x56e14ec4, 0x362abfce, 0xddc6c837,
+	0xd79a3234, 0x92638212, 0x670efa8e, 0x406000e0,
+}
+
+var s3 = [256]uint32{
+	0x3a39ce37, 0xd3faf5cf, 0xabc27737, 0x5ac52d1b, 0x5cb0679e, 0x4fa33742,
+	0xd3822740, 0x99bc9bbe, 0xd5118e9d, 0xbf0f7315, 0xd62d1c7e, 0xc700c47b,
+	0xb78c1b6b, 0x21a19045, 0xb26eb1be, 0x6a366eb4, 0x5748ab2f, 0xbc946e79,
+	0xc6a376d2, 0x6549c2c8, 0x530ff8ee, 0x468dde7d, 0xd5730a1d, 0x4cd04dc6,
+	0x2939bbdb, 0xa9ba4650, 0xac9526e8, 0xbe5ee304, 0xa1fad5f0, 0x6a2d519a,
+	0x63ef8ce2, 0x9a86ee22, 0xc089c2b8, 0x43242ef6, 0xa51e03aa, 0x9cf2d0a4,
+	0x83c061ba, 0x9be96a4d, 0x8fe51550, 0xba645bd6, 0x2826a2f9, 0xa73a3ae1,
+	0x4ba99586, 0xef5562e9, 0xc72fefd3, 0xf752f7da, 0x3f046f69, 0x77fa0a59,
+	0x80e4a915, 0x87b08601, 0x9b09e6ad, 0x3b3ee593, 0xe990fd5a, 0x9e34d797,
+	0x2cf0b7d9, 0x022b8b51, 0x96d5ac3a, 0x017da67d, 0xd1cf3ed6, 0x7c7d2d28,
+	0x1f9f25cf, 0xadf2b89b, 0x5ad6b472, 0x5a88f54c, 0xe029ac71, 0xe019a5e6,
+	0x47b0acfd, 0xed93fa9b, 0xe8d3c48d, 0x283b57cc, 0xf8d56629, 0x79132e28,
+	0x785f0191, 0xed756055, 0xf7960e44, 0xe3d35e8c, 0x15056dd4, 0x88f46dba,
+	0x03a16125, 0x0564f0bd, 0xc3eb9e15, 0x3c9057a2, 0x97271aec, 0xa93a072a,
+	0x1b3f6d9b, 0x1e6321f5, 0xf59c66fb, 0x26dcf319, 0x7533d928, 0xb155fdf5,
+	0x03563482, 0x8aba3cbb, 0x28517711, 0xc20ad9f8, 0xabcc5167, 0xccad925f,
+	0x4de81751, 0x3830dc8e, 0x379d5862, 0x9320f991, 0xea7a90c2, 0xfb3e7bce,
+	0x5121ce64, 0x774fbe32, 0xa8b6e37e, 0xc3293d46, 0x48de5369, 0x6413e680,
+	0xa2ae0810, 0xdd6db224, 0x69852dfd, 0x09072166, 0xb39a460a, 0x6445c0dd,
+	0x586cdecf, 0x1c20c8ae, 0x5bbef7dd, 0x1b588d40, 0xccd2017f, 0x6bb4e3bb,
+	0xdda26a7e, 0x3a59ff45, 0x3e350a44, 0xbcb4cdd5, 0x72eacea8, 0xfa6484bb,
+	0x8d6612ae, 0xbf3c6f47, 0xd29be463, 0x542f5d9e, 0xaec2771b, 0xf64e6370,
+	0x740e0d8d, 0xe75b1357, 0xf8721671, 0xaf537d5d, 0x4040cb08, 0x4eb4e2cc,
+	0x34d2466a, 0x0115af84, 0xe1b00428, 0x95983a1d, 0x06b89fb4, 0xce6ea048,
+	0x6f3f3b82, 0x3520ab82, 0x011a1d4b, 0x277227f8, 0x611560b1, 0xe7933fdc,
+	0xbb3a792b, 0x344525bd, 0xa08839e1, 0x51ce794b, 0x2f32c9b7, 0xa01fbac9,
+	0xe01cc87e, 0xbcc7d1f6, 0xcf0111c3, 0xa1e8aac7, 0x1a908749, 0xd44fbd9a,
+	0xd0dadecb, 0xd50ada38, 0x0339c32a, 0xc6913667, 0x8df9317c, 0xe0b12b4f,
+	0xf79e59b7, 0x43f5bb3a, 0xf2d519ff, 0x27d9459c, 0xbf97222c, 0x15e6fc2a,
+	0x0f91fc71, 0x9b941525, 0xfae59361, 0xceb69ceb, 0xc2a86459, 0x12baa8d1,
+	0xb6c1075e, 0xe3056a0c, 0x10d25065, 0xcb03a442, 0xe0ec6e0e, 0x1698db3b,
+	0x4c98a0be, 0x3278e964, 0x9f1f9532, 0xe0d392df, 0xd3a0342b, 0x8971f21e,
+	0x1b0a7441, 0x4ba3348c, 0xc5be7120, 0xc37632d8, 0xdf359f8d, 0x9b992f2e,
+	0xe60b6f47, 0x0fe3f11d, 0xe54cda54, 0x1edad891, 0xce6279cf, 0xcd3e7e6f,
+	0x1618b166, 0xfd2c1d05, 0x848fd2c5, 0xf6fb2299, 0xf523f357, 0xa6327623,
+	0x93a83531, 0x56cccd02, 0xacf08162, 0x5a75ebb5, 0x6e163697, 0x88d273cc,
+	0xde966292, 0x81b949d0, 0x4c50901b, 0x71c65614, 0xe6c6c7bd, 0x327a140a,
+	0x45e1d006, 0xc3f27b9a, 0xc9aa53fd, 0x62a80f00, 0xbb25bfe2, 0x35bdd2f6,
+	0x71126905, 0xb2040222, 0xb6cbcf7c, 0xcd769c2b, 0x53113ec0, 0x1640e3d3,
+	0x38abbd60, 0x2547adf0, 0xba38209c, 0xf746ce76, 0x77afa1c5, 0x20756060,
+	0x85cbfe4e, 0x8ae88dd8, 0x7aaaf9b0, 0x4cf9aa7e, 0x1948c25c, 0x02fb8a8c,
+	0x01c36ae4, 0xd6ebe1f9, 0x90d4f869, 0xa65cdea0, 0x3f09252d, 0xc208e69f,
+	0xb74e6132, 0xce77e25b, 0x578fdfe3, 0x3ac372e6,
+}
+
+var p = [18]uint32{
+	0x243f6a88, 0x85a308d3, 0x13198a2e, 0x03707344, 0xa4093822, 0x299f31d0,
+	0x082efa98, 0xec4e6c89, 0x452821e6, 0x38d01377, 0xbe5466cf, 0x34e90c6c,
+	0xc0ac29b7, 0xc97c50dd, 0x3f84d5b5, 0xb5470917, 0x9216d5d9, 0x8979fb1b,
+}
diff --git a/vendor/golang.org/x/crypto/ssh/terminal/terminal.go b/vendor/golang.org/x/crypto/ssh/terminal/terminal.go
new file mode 100644
index 0000000..2f04ee5
--- /dev/null
+++ b/vendor/golang.org/x/crypto/ssh/terminal/terminal.go
@@ -0,0 +1,966 @@
+// Copyright 2011 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package terminal
+
+import (
+	"bytes"
+	"io"
+	"strconv"
+	"sync"
+	"unicode/utf8"
+)
+
+// EscapeCodes contains escape sequences that can be written to the terminal in
+// order to achieve different styles of text.
+type EscapeCodes struct {
+	// Foreground colors
+	Black, Red, Green, Yellow, Blue, Magenta, Cyan, White []byte
+
+	// Reset all attributes
+	Reset []byte
+}
+
+var vt100EscapeCodes = EscapeCodes{
+	Black:   []byte{keyEscape, '[', '3', '0', 'm'},
+	Red:     []byte{keyEscape, '[', '3', '1', 'm'},
+	Green:   []byte{keyEscape, '[', '3', '2', 'm'},
+	Yellow:  []byte{keyEscape, '[', '3', '3', 'm'},
+	Blue:    []byte{keyEscape, '[', '3', '4', 'm'},
+	Magenta: []byte{keyEscape, '[', '3', '5', 'm'},
+	Cyan:    []byte{keyEscape, '[', '3', '6', 'm'},
+	White:   []byte{keyEscape, '[', '3', '7', 'm'},
+
+	Reset: []byte{keyEscape, '[', '0', 'm'},
+}
+
+// Terminal contains the state for running a VT100 terminal that is capable of
+// reading lines of input.
+type Terminal struct {
+	// AutoCompleteCallback, if non-null, is called for each keypress with
+	// the full input line and the current position of the cursor (in
+	// bytes, as an index into |line|). If it returns ok=false, the key
+	// press is processed normally. Otherwise it returns a replacement line
+	// and the new cursor position.
+	AutoCompleteCallback func(line string, pos int, key rune) (newLine string, newPos int, ok bool)
+
+	// Escape contains a pointer to the escape codes for this terminal.
+	// It's always a valid pointer, although the escape codes themselves
+	// may be empty if the terminal doesn't support them.
+	Escape *EscapeCodes
+
+	// lock protects the terminal and the state in this object from
+	// concurrent processing of a key press and a Write() call.
+	lock sync.Mutex
+
+	c      io.ReadWriter
+	prompt []rune
+
+	// line is the current line being entered.
+	line []rune
+	// pos is the logical position of the cursor in line
+	pos int
+	// echo is true if local echo is enabled
+	echo bool
+	// pasteActive is true iff there is a bracketed paste operation in
+	// progress.
+	pasteActive bool
+
+	// cursorX contains the current X value of the cursor where the left
+	// edge is 0. cursorY contains the row number where the first row of
+	// the current line is 0.
+	cursorX, cursorY int
+	// maxLine is the greatest value of cursorY so far.
+	maxLine int
+
+	termWidth, termHeight int
+
+	// outBuf contains the terminal data to be sent.
+	outBuf []byte
+	// remainder contains the remainder of any partial key sequences after
+	// a read. It aliases into inBuf.
+	remainder []byte
+	inBuf     [256]byte
+
+	// history contains previously entered commands so that they can be
+	// accessed with the up and down keys.
+	history stRingBuffer
+	// historyIndex stores the currently accessed history entry, where zero
+	// means the immediately previous entry.
+	historyIndex int
+	// When navigating up and down the history it's possible to return to
+	// the incomplete, initial line. That value is stored in
+	// historyPending.
+	historyPending string
+}
+
+// NewTerminal runs a VT100 terminal on the given ReadWriter. If the ReadWriter is
+// a local terminal, that terminal must first have been put into raw mode.
+// prompt is a string that is written at the start of each input line (i.e.
+// "> ").
+func NewTerminal(c io.ReadWriter, prompt string) *Terminal {
+	return &Terminal{
+		Escape:       &vt100EscapeCodes,
+		c:            c,
+		prompt:       []rune(prompt),
+		termWidth:    80,
+		termHeight:   24,
+		echo:         true,
+		historyIndex: -1,
+	}
+}
+
+const (
+	keyCtrlD     = 4
+	keyCtrlU     = 21
+	keyEnter     = '\r'
+	keyEscape    = 27
+	keyBackspace = 127
+	keyUnknown   = 0xd800 /* UTF-16 surrogate area */ + iota
+	keyUp
+	keyDown
+	keyLeft
+	keyRight
+	keyAltLeft
+	keyAltRight
+	keyHome
+	keyEnd
+	keyDeleteWord
+	keyDeleteLine
+	keyClearScreen
+	keyPasteStart
+	keyPasteEnd
+)
+
+var (
+	crlf       = []byte{'\r', '\n'}
+	pasteStart = []byte{keyEscape, '[', '2', '0', '0', '~'}
+	pasteEnd   = []byte{keyEscape, '[', '2', '0', '1', '~'}
+)
+
+// bytesToKey tries to parse a key sequence from b. If successful, it returns
+// the key and the remainder of the input. Otherwise it returns utf8.RuneError.
+func bytesToKey(b []byte, pasteActive bool) (rune, []byte) {
+	if len(b) == 0 {
+		return utf8.RuneError, nil
+	}
+
+	if !pasteActive {
+		switch b[0] {
+		case 1: // ^A
+			return keyHome, b[1:]
+		case 5: // ^E
+			return keyEnd, b[1:]
+		case 8: // ^H
+			return keyBackspace, b[1:]
+		case 11: // ^K
+			return keyDeleteLine, b[1:]
+		case 12: // ^L
+			return keyClearScreen, b[1:]
+		case 23: // ^W
+			return keyDeleteWord, b[1:]
+		case 14: // ^N
+			return keyDown, b[1:]
+		case 16: // ^P
+			return keyUp, b[1:]
+		}
+	}
+
+	if b[0] != keyEscape {
+		if !utf8.FullRune(b) {
+			return utf8.RuneError, b
+		}
+		r, l := utf8.DecodeRune(b)
+		return r, b[l:]
+	}
+
+	if !pasteActive && len(b) >= 3 && b[0] == keyEscape && b[1] == '[' {
+		switch b[2] {
+		case 'A':
+			return keyUp, b[3:]
+		case 'B':
+			return keyDown, b[3:]
+		case 'C':
+			return keyRight, b[3:]
+		case 'D':
+			return keyLeft, b[3:]
+		case 'H':
+			return keyHome, b[3:]
+		case 'F':
+			return keyEnd, b[3:]
+		}
+	}
+
+	if !pasteActive && len(b) >= 6 && b[0] == keyEscape && b[1] == '[' && b[2] == '1' && b[3] == ';' && b[4] == '3' {
+		switch b[5] {
+		case 'C':
+			return keyAltRight, b[6:]
+		case 'D':
+			return keyAltLeft, b[6:]
+		}
+	}
+
+	if !pasteActive && len(b) >= 6 && bytes.Equal(b[:6], pasteStart) {
+		return keyPasteStart, b[6:]
+	}
+
+	if pasteActive && len(b) >= 6 && bytes.Equal(b[:6], pasteEnd) {
+		return keyPasteEnd, b[6:]
+	}
+
+	// If we get here then we have a key that we don't recognise, or a
+	// partial sequence. It's not clear how one should find the end of a
+	// sequence without knowing them all, but it seems that [a-zA-Z~] only
+	// appears at the end of a sequence.
+	for i, c := range b[0:] {
+		if c >= 'a' && c <= 'z' || c >= 'A' && c <= 'Z' || c == '~' {
+			return keyUnknown, b[i+1:]
+		}
+	}
+
+	return utf8.RuneError, b
+}
+
+// queue appends data to the end of t.outBuf
+func (t *Terminal) queue(data []rune) {
+	t.outBuf = append(t.outBuf, []byte(string(data))...)
+}
+
+var eraseUnderCursor = []rune{' ', keyEscape, '[', 'D'}
+var space = []rune{' '}
+
+func isPrintable(key rune) bool {
+	isInSurrogateArea := key >= 0xd800 && key <= 0xdbff
+	return key >= 32 && !isInSurrogateArea
+}
+
+// moveCursorToPos appends data to t.outBuf which will move the cursor to the
+// given, logical position in the text.
+func (t *Terminal) moveCursorToPos(pos int) {
+	if !t.echo {
+		return
+	}
+
+	x := visualLength(t.prompt) + pos
+	y := x / t.termWidth
+	x = x % t.termWidth
+
+	up := 0
+	if y < t.cursorY {
+		up = t.cursorY - y
+	}
+
+	down := 0
+	if y > t.cursorY {
+		down = y - t.cursorY
+	}
+
+	left := 0
+	if x < t.cursorX {
+		left = t.cursorX - x
+	}
+
+	right := 0
+	if x > t.cursorX {
+		right = x - t.cursorX
+	}
+
+	t.cursorX = x
+	t.cursorY = y
+	t.move(up, down, left, right)
+}
+
+func (t *Terminal) move(up, down, left, right int) {
+	m := []rune{}
+
+	// 1 unit up can be expressed as ^[[A or ^[A
+	// 5 units up can be expressed as ^[[5A
+
+	if up == 1 {
+		m = append(m, keyEscape, '[', 'A')
+	} else if up > 1 {
+		m = append(m, keyEscape, '[')
+		m = append(m, []rune(strconv.Itoa(up))...)
+		m = append(m, 'A')
+	}
+
+	if down == 1 {
+		m = append(m, keyEscape, '[', 'B')
+	} else if down > 1 {
+		m = append(m, keyEscape, '[')
+		m = append(m, []rune(strconv.Itoa(down))...)
+		m = append(m, 'B')
+	}
+
+	if right == 1 {
+		m = append(m, keyEscape, '[', 'C')
+	} else if right > 1 {
+		m = append(m, keyEscape, '[')
+		m = append(m, []rune(strconv.Itoa(right))...)
+		m = append(m, 'C')
+	}
+
+	if left == 1 {
+		m = append(m, keyEscape, '[', 'D')
+	} else if left > 1 {
+		m = append(m, keyEscape, '[')
+		m = append(m, []rune(strconv.Itoa(left))...)
+		m = append(m, 'D')
+	}
+
+	t.queue(m)
+}
+
+func (t *Terminal) clearLineToRight() {
+	op := []rune{keyEscape, '[', 'K'}
+	t.queue(op)
+}
+
+const maxLineLength = 4096
+
+func (t *Terminal) setLine(newLine []rune, newPos int) {
+	if t.echo {
+		t.moveCursorToPos(0)
+		t.writeLine(newLine)
+		for i := len(newLine); i < len(t.line); i++ {
+			t.writeLine(space)
+		}
+		t.moveCursorToPos(newPos)
+	}
+	t.line = newLine
+	t.pos = newPos
+}
+
+func (t *Terminal) advanceCursor(places int) {
+	t.cursorX += places
+	t.cursorY += t.cursorX / t.termWidth
+	if t.cursorY > t.maxLine {
+		t.maxLine = t.cursorY
+	}
+	t.cursorX = t.cursorX % t.termWidth
+
+	if places > 0 && t.cursorX == 0 {
+		// Normally terminals will advance the current position
+		// when writing a character. But that doesn't happen
+		// for the last character in a line. However, when
+		// writing a character (except a new line) that causes
+		// a line wrap, the position will be advanced two
+		// places.
+		//
+		// So, if we are stopping at the end of a line, we
+		// need to write a newline so that our cursor can be
+		// advanced to the next line.
+		t.outBuf = append(t.outBuf, '\r', '\n')
+	}
+}
+
+func (t *Terminal) eraseNPreviousChars(n int) {
+	if n == 0 {
+		return
+	}
+
+	if t.pos < n {
+		n = t.pos
+	}
+	t.pos -= n
+	t.moveCursorToPos(t.pos)
+
+	copy(t.line[t.pos:], t.line[n+t.pos:])
+	t.line = t.line[:len(t.line)-n]
+	if t.echo {
+		t.writeLine(t.line[t.pos:])
+		for i := 0; i < n; i++ {
+			t.queue(space)
+		}
+		t.advanceCursor(n)
+		t.moveCursorToPos(t.pos)
+	}
+}
+
+// countToLeftWord returns then number of characters from the cursor to the
+// start of the previous word.
+func (t *Terminal) countToLeftWord() int {
+	if t.pos == 0 {
+		return 0
+	}
+
+	pos := t.pos - 1
+	for pos > 0 {
+		if t.line[pos] != ' ' {
+			break
+		}
+		pos--
+	}
+	for pos > 0 {
+		if t.line[pos] == ' ' {
+			pos++
+			break
+		}
+		pos--
+	}
+
+	return t.pos - pos
+}
+
+// countToRightWord returns then number of characters from the cursor to the
+// start of the next word.
+func (t *Terminal) countToRightWord() int {
+	pos := t.pos
+	for pos < len(t.line) {
+		if t.line[pos] == ' ' {
+			break
+		}
+		pos++
+	}
+	for pos < len(t.line) {
+		if t.line[pos] != ' ' {
+			break
+		}
+		pos++
+	}
+	return pos - t.pos
+}
+
+// visualLength returns the number of visible glyphs in s.
+func visualLength(runes []rune) int {
+	inEscapeSeq := false
+	length := 0
+
+	for _, r := range runes {
+		switch {
+		case inEscapeSeq:
+			if (r >= 'a' && r <= 'z') || (r >= 'A' && r <= 'Z') {
+				inEscapeSeq = false
+			}
+		case r == '\x1b':
+			inEscapeSeq = true
+		default:
+			length++
+		}
+	}
+
+	return length
+}
+
+// handleKey processes the given key and, optionally, returns a line of text
+// that the user has entered.
+func (t *Terminal) handleKey(key rune) (line string, ok bool) {
+	if t.pasteActive && key != keyEnter {
+		t.addKeyToLine(key)
+		return
+	}
+
+	switch key {
+	case keyBackspace:
+		if t.pos == 0 {
+			return
+		}
+		t.eraseNPreviousChars(1)
+	case keyAltLeft:
+		// move left by a word.
+		t.pos -= t.countToLeftWord()
+		t.moveCursorToPos(t.pos)
+	case keyAltRight:
+		// move right by a word.
+		t.pos += t.countToRightWord()
+		t.moveCursorToPos(t.pos)
+	case keyLeft:
+		if t.pos == 0 {
+			return
+		}
+		t.pos--
+		t.moveCursorToPos(t.pos)
+	case keyRight:
+		if t.pos == len(t.line) {
+			return
+		}
+		t.pos++
+		t.moveCursorToPos(t.pos)
+	case keyHome:
+		if t.pos == 0 {
+			return
+		}
+		t.pos = 0
+		t.moveCursorToPos(t.pos)
+	case keyEnd:
+		if t.pos == len(t.line) {
+			return
+		}
+		t.pos = len(t.line)
+		t.moveCursorToPos(t.pos)
+	case keyUp:
+		entry, ok := t.history.NthPreviousEntry(t.historyIndex + 1)
+		if !ok {
+			return "", false
+		}
+		if t.historyIndex == -1 {
+			t.historyPending = string(t.line)
+		}
+		t.historyIndex++
+		runes := []rune(entry)
+		t.setLine(runes, len(runes))
+	case keyDown:
+		switch t.historyIndex {
+		case -1:
+			return
+		case 0:
+			runes := []rune(t.historyPending)
+			t.setLine(runes, len(runes))
+			t.historyIndex--
+		default:
+			entry, ok := t.history.NthPreviousEntry(t.historyIndex - 1)
+			if ok {
+				t.historyIndex--
+				runes := []rune(entry)
+				t.setLine(runes, len(runes))
+			}
+		}
+	case keyEnter:
+		t.moveCursorToPos(len(t.line))
+		t.queue([]rune("\r\n"))
+		line = string(t.line)
+		ok = true
+		t.line = t.line[:0]
+		t.pos = 0
+		t.cursorX = 0
+		t.cursorY = 0
+		t.maxLine = 0
+	case keyDeleteWord:
+		// Delete zero or more spaces and then one or more characters.
+		t.eraseNPreviousChars(t.countToLeftWord())
+	case keyDeleteLine:
+		// Delete everything from the current cursor position to the
+		// end of line.
+		for i := t.pos; i < len(t.line); i++ {
+			t.queue(space)
+			t.advanceCursor(1)
+		}
+		t.line = t.line[:t.pos]
+		t.moveCursorToPos(t.pos)
+	case keyCtrlD:
+		// Erase the character under the current position.
+		// The EOF case when the line is empty is handled in
+		// readLine().
+		if t.pos < len(t.line) {
+			t.pos++
+			t.eraseNPreviousChars(1)
+		}
+	case keyCtrlU:
+		t.eraseNPreviousChars(t.pos)
+	case keyClearScreen:
+		// Erases the screen and moves the cursor to the home position.
+		t.queue([]rune("\x1b[2J\x1b[H"))
+		t.queue(t.prompt)
+		t.cursorX, t.cursorY = 0, 0
+		t.advanceCursor(visualLength(t.prompt))
+		t.setLine(t.line, t.pos)
+	default:
+		if t.AutoCompleteCallback != nil {
+			prefix := string(t.line[:t.pos])
+			suffix := string(t.line[t.pos:])
+
+			t.lock.Unlock()
+			newLine, newPos, completeOk := t.AutoCompleteCallback(prefix+suffix, len(prefix), key)
+			t.lock.Lock()
+
+			if completeOk {
+				t.setLine([]rune(newLine), utf8.RuneCount([]byte(newLine)[:newPos]))
+				return
+			}
+		}
+		if !isPrintable(key) {
+			return
+		}
+		if len(t.line) == maxLineLength {
+			return
+		}
+		t.addKeyToLine(key)
+	}
+	return
+}
+
+// addKeyToLine inserts the given key at the current position in the current
+// line.
+func (t *Terminal) addKeyToLine(key rune) {
+	if len(t.line) == cap(t.line) {
+		newLine := make([]rune, len(t.line), 2*(1+len(t.line)))
+		copy(newLine, t.line)
+		t.line = newLine
+	}
+	t.line = t.line[:len(t.line)+1]
+	copy(t.line[t.pos+1:], t.line[t.pos:])
+	t.line[t.pos] = key
+	if t.echo {
+		t.writeLine(t.line[t.pos:])
+	}
+	t.pos++
+	t.moveCursorToPos(t.pos)
+}
+
+func (t *Terminal) writeLine(line []rune) {
+	for len(line) != 0 {
+		remainingOnLine := t.termWidth - t.cursorX
+		todo := len(line)
+		if todo > remainingOnLine {
+			todo = remainingOnLine
+		}
+		t.queue(line[:todo])
+		t.advanceCursor(visualLength(line[:todo]))
+		line = line[todo:]
+	}
+}
+
+// writeWithCRLF writes buf to w but replaces all occurrences of \n with \r\n.
+func writeWithCRLF(w io.Writer, buf []byte) (n int, err error) {
+	for len(buf) > 0 {
+		i := bytes.IndexByte(buf, '\n')
+		todo := len(buf)
+		if i >= 0 {
+			todo = i
+		}
+
+		var nn int
+		nn, err = w.Write(buf[:todo])
+		n += nn
+		if err != nil {
+			return n, err
+		}
+		buf = buf[todo:]
+
+		if i >= 0 {
+			if _, err = w.Write(crlf); err != nil {
+				return n, err
+			}
+			n++
+			buf = buf[1:]
+		}
+	}
+
+	return n, nil
+}
+
+func (t *Terminal) Write(buf []byte) (n int, err error) {
+	t.lock.Lock()
+	defer t.lock.Unlock()
+
+	if t.cursorX == 0 && t.cursorY == 0 {
+		// This is the easy case: there's nothing on the screen that we
+		// have to move out of the way.
+		return writeWithCRLF(t.c, buf)
+	}
+
+	// We have a prompt and possibly user input on the screen. We
+	// have to clear it first.
+	t.move(0 /* up */, 0 /* down */, t.cursorX /* left */, 0 /* right */)
+	t.cursorX = 0
+	t.clearLineToRight()
+
+	for t.cursorY > 0 {
+		t.move(1 /* up */, 0, 0, 0)
+		t.cursorY--
+		t.clearLineToRight()
+	}
+
+	if _, err = t.c.Write(t.outBuf); err != nil {
+		return
+	}
+	t.outBuf = t.outBuf[:0]
+
+	if n, err = writeWithCRLF(t.c, buf); err != nil {
+		return
+	}
+
+	t.writeLine(t.prompt)
+	if t.echo {
+		t.writeLine(t.line)
+	}
+
+	t.moveCursorToPos(t.pos)
+
+	if _, err = t.c.Write(t.outBuf); err != nil {
+		return
+	}
+	t.outBuf = t.outBuf[:0]
+	return
+}
+
+// ReadPassword temporarily changes the prompt and reads a password, without
+// echo, from the terminal.
+func (t *Terminal) ReadPassword(prompt string) (line string, err error) {
+	t.lock.Lock()
+	defer t.lock.Unlock()
+
+	oldPrompt := t.prompt
+	t.prompt = []rune(prompt)
+	t.echo = false
+
+	line, err = t.readLine()
+
+	t.prompt = oldPrompt
+	t.echo = true
+
+	return
+}
+
+// ReadLine returns a line of input from the terminal.
+func (t *Terminal) ReadLine() (line string, err error) {
+	t.lock.Lock()
+	defer t.lock.Unlock()
+
+	return t.readLine()
+}
+
+func (t *Terminal) readLine() (line string, err error) {
+	// t.lock must be held at this point
+
+	if t.cursorX == 0 && t.cursorY == 0 {
+		t.writeLine(t.prompt)
+		t.c.Write(t.outBuf)
+		t.outBuf = t.outBuf[:0]
+	}
+
+	lineIsPasted := t.pasteActive
+
+	for {
+		rest := t.remainder
+		lineOk := false
+		for !lineOk {
+			var key rune
+			key, rest = bytesToKey(rest, t.pasteActive)
+			if key == utf8.RuneError {
+				break
+			}
+			if !t.pasteActive {
+				if key == keyCtrlD {
+					if len(t.line) == 0 {
+						return "", io.EOF
+					}
+				}
+				if key == keyPasteStart {
+					t.pasteActive = true
+					if len(t.line) == 0 {
+						lineIsPasted = true
+					}
+					continue
+				}
+			} else if key == keyPasteEnd {
+				t.pasteActive = false
+				continue
+			}
+			if !t.pasteActive {
+				lineIsPasted = false
+			}
+			line, lineOk = t.handleKey(key)
+		}
+		if len(rest) > 0 {
+			n := copy(t.inBuf[:], rest)
+			t.remainder = t.inBuf[:n]
+		} else {
+			t.remainder = nil
+		}
+		t.c.Write(t.outBuf)
+		t.outBuf = t.outBuf[:0]
+		if lineOk {
+			if t.echo {
+				t.historyIndex = -1
+				t.history.Add(line)
+			}
+			if lineIsPasted {
+				err = ErrPasteIndicator
+			}
+			return
+		}
+
+		// t.remainder is a slice at the beginning of t.inBuf
+		// containing a partial key sequence
+		readBuf := t.inBuf[len(t.remainder):]
+		var n int
+
+		t.lock.Unlock()
+		n, err = t.c.Read(readBuf)
+		t.lock.Lock()
+
+		if err != nil {
+			return
+		}
+
+		t.remainder = t.inBuf[:n+len(t.remainder)]
+	}
+}
+
+// SetPrompt sets the prompt to be used when reading subsequent lines.
+func (t *Terminal) SetPrompt(prompt string) {
+	t.lock.Lock()
+	defer t.lock.Unlock()
+
+	t.prompt = []rune(prompt)
+}
+
+func (t *Terminal) clearAndRepaintLinePlusNPrevious(numPrevLines int) {
+	// Move cursor to column zero at the start of the line.
+	t.move(t.cursorY, 0, t.cursorX, 0)
+	t.cursorX, t.cursorY = 0, 0
+	t.clearLineToRight()
+	for t.cursorY < numPrevLines {
+		// Move down a line
+		t.move(0, 1, 0, 0)
+		t.cursorY++
+		t.clearLineToRight()
+	}
+	// Move back to beginning.
+	t.move(t.cursorY, 0, 0, 0)
+	t.cursorX, t.cursorY = 0, 0
+
+	t.queue(t.prompt)
+	t.advanceCursor(visualLength(t.prompt))
+	t.writeLine(t.line)
+	t.moveCursorToPos(t.pos)
+}
+
+func (t *Terminal) SetSize(width, height int) error {
+	t.lock.Lock()
+	defer t.lock.Unlock()
+
+	if width == 0 {
+		width = 1
+	}
+
+	oldWidth := t.termWidth
+	t.termWidth, t.termHeight = width, height
+
+	switch {
+	case width == oldWidth:
+		// If the width didn't change then nothing else needs to be
+		// done.
+		return nil
+	case len(t.line) == 0 && t.cursorX == 0 && t.cursorY == 0:
+		// If there is nothing on current line and no prompt printed,
+		// just do nothing
+		return nil
+	case width < oldWidth:
+		// Some terminals (e.g. xterm) will truncate lines that were
+		// too long when shinking. Others, (e.g. gnome-terminal) will
+		// attempt to wrap them. For the former, repainting t.maxLine
+		// works great, but that behaviour goes badly wrong in the case
+		// of the latter because they have doubled every full line.
+
+		// We assume that we are working on a terminal that wraps lines
+		// and adjust the cursor position based on every previous line
+		// wrapping and turning into two. This causes the prompt on
+		// xterms to move upwards, which isn't great, but it avoids a
+		// huge mess with gnome-terminal.
+		if t.cursorX >= t.termWidth {
+			t.cursorX = t.termWidth - 1
+		}
+		t.cursorY *= 2
+		t.clearAndRepaintLinePlusNPrevious(t.maxLine * 2)
+	case width > oldWidth:
+		// If the terminal expands then our position calculations will
+		// be wrong in the future because we think the cursor is
+		// |t.pos| chars into the string, but there will be a gap at
+		// the end of any wrapped line.
+		//
+		// But the position will actually be correct until we move, so
+		// we can move back to the beginning and repaint everything.
+		t.clearAndRepaintLinePlusNPrevious(t.maxLine)
+	}
+
+	_, err := t.c.Write(t.outBuf)
+	t.outBuf = t.outBuf[:0]
+	return err
+}
+
+type pasteIndicatorError struct{}
+
+func (pasteIndicatorError) Error() string {
+	return "terminal: ErrPasteIndicator not correctly handled"
+}
+
+// ErrPasteIndicator may be returned from ReadLine as the error, in addition
+// to valid line data. It indicates that bracketed paste mode is enabled and
+// that the returned line consists only of pasted data. Programs may wish to
+// interpret pasted data more literally than typed data.
+var ErrPasteIndicator = pasteIndicatorError{}
+
+// SetBracketedPasteMode requests that the terminal bracket paste operations
+// with markers. Not all terminals support this but, if it is supported, then
+// enabling this mode will stop any autocomplete callback from running due to
+// pastes. Additionally, any lines that are completely pasted will be returned
+// from ReadLine with the error set to ErrPasteIndicator.
+func (t *Terminal) SetBracketedPasteMode(on bool) {
+	if on {
+		io.WriteString(t.c, "\x1b[?2004h")
+	} else {
+		io.WriteString(t.c, "\x1b[?2004l")
+	}
+}
+
+// stRingBuffer is a ring buffer of strings.
+type stRingBuffer struct {
+	// entries contains max elements.
+	entries []string
+	max     int
+	// head contains the index of the element most recently added to the ring.
+	head int
+	// size contains the number of elements in the ring.
+	size int
+}
+
+func (s *stRingBuffer) Add(a string) {
+	if s.entries == nil {
+		const defaultNumEntries = 100
+		s.entries = make([]string, defaultNumEntries)
+		s.max = defaultNumEntries
+	}
+
+	s.head = (s.head + 1) % s.max
+	s.entries[s.head] = a
+	if s.size < s.max {
+		s.size++
+	}
+}
+
+// NthPreviousEntry returns the value passed to the nth previous call to Add.
+// If n is zero then the immediately prior value is returned, if one, then the
+// next most recent, and so on. If such an element doesn't exist then ok is
+// false.
+func (s *stRingBuffer) NthPreviousEntry(n int) (value string, ok bool) {
+	if n >= s.size {
+		return "", false
+	}
+	index := s.head - n
+	if index < 0 {
+		index += s.max
+	}
+	return s.entries[index], true
+}
+
+// readPasswordLine reads from reader until it finds \n or io.EOF.
+// The slice returned does not include the \n.
+// readPasswordLine also ignores any \r it finds.
+func readPasswordLine(reader io.Reader) ([]byte, error) {
+	var buf [1]byte
+	var ret []byte
+
+	for {
+		n, err := reader.Read(buf[:])
+		if n > 0 {
+			switch buf[0] {
+			case '\n':
+				return ret, nil
+			case '\r':
+				// remove \r from passwords on Windows
+			default:
+				ret = append(ret, buf[0])
+			}
+			continue
+		}
+		if err != nil {
+			if err == io.EOF && len(ret) > 0 {
+				return ret, nil
+			}
+			return ret, err
+		}
+	}
+}
diff --git a/vendor/golang.org/x/crypto/ssh/terminal/util.go b/vendor/golang.org/x/crypto/ssh/terminal/util.go
new file mode 100644
index 0000000..3911040
--- /dev/null
+++ b/vendor/golang.org/x/crypto/ssh/terminal/util.go
@@ -0,0 +1,114 @@
+// Copyright 2011 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build aix darwin dragonfly freebsd linux,!appengine netbsd openbsd
+
+// Package terminal provides support functions for dealing with terminals, as
+// commonly found on UNIX systems.
+//
+// Putting a terminal into raw mode is the most common requirement:
+//
+// 	oldState, err := terminal.MakeRaw(0)
+// 	if err != nil {
+// 	        panic(err)
+// 	}
+// 	defer terminal.Restore(0, oldState)
+package terminal // import "golang.org/x/crypto/ssh/terminal"
+
+import (
+	"golang.org/x/sys/unix"
+)
+
+// State contains the state of a terminal.
+type State struct {
+	termios unix.Termios
+}
+
+// IsTerminal returns whether the given file descriptor is a terminal.
+func IsTerminal(fd int) bool {
+	_, err := unix.IoctlGetTermios(fd, ioctlReadTermios)
+	return err == nil
+}
+
+// MakeRaw put the terminal connected to the given file descriptor into raw
+// mode and returns the previous state of the terminal so that it can be
+// restored.
+func MakeRaw(fd int) (*State, error) {
+	termios, err := unix.IoctlGetTermios(fd, ioctlReadTermios)
+	if err != nil {
+		return nil, err
+	}
+
+	oldState := State{termios: *termios}
+
+	// This attempts to replicate the behaviour documented for cfmakeraw in
+	// the termios(3) manpage.
+	termios.Iflag &^= unix.IGNBRK | unix.BRKINT | unix.PARMRK | unix.ISTRIP | unix.INLCR | unix.IGNCR | unix.ICRNL | unix.IXON
+	termios.Oflag &^= unix.OPOST
+	termios.Lflag &^= unix.ECHO | unix.ECHONL | unix.ICANON | unix.ISIG | unix.IEXTEN
+	termios.Cflag &^= unix.CSIZE | unix.PARENB
+	termios.Cflag |= unix.CS8
+	termios.Cc[unix.VMIN] = 1
+	termios.Cc[unix.VTIME] = 0
+	if err := unix.IoctlSetTermios(fd, ioctlWriteTermios, termios); err != nil {
+		return nil, err
+	}
+
+	return &oldState, nil
+}
+
+// GetState returns the current state of a terminal which may be useful to
+// restore the terminal after a signal.
+func GetState(fd int) (*State, error) {
+	termios, err := unix.IoctlGetTermios(fd, ioctlReadTermios)
+	if err != nil {
+		return nil, err
+	}
+
+	return &State{termios: *termios}, nil
+}
+
+// Restore restores the terminal connected to the given file descriptor to a
+// previous state.
+func Restore(fd int, state *State) error {
+	return unix.IoctlSetTermios(fd, ioctlWriteTermios, &state.termios)
+}
+
+// GetSize returns the dimensions of the given terminal.
+func GetSize(fd int) (width, height int, err error) {
+	ws, err := unix.IoctlGetWinsize(fd, unix.TIOCGWINSZ)
+	if err != nil {
+		return -1, -1, err
+	}
+	return int(ws.Col), int(ws.Row), nil
+}
+
+// passwordReader is an io.Reader that reads from a specific file descriptor.
+type passwordReader int
+
+func (r passwordReader) Read(buf []byte) (int, error) {
+	return unix.Read(int(r), buf)
+}
+
+// ReadPassword reads a line of input from a terminal without local echo.  This
+// is commonly used for inputting passwords and other sensitive data. The slice
+// returned does not include the \n.
+func ReadPassword(fd int) ([]byte, error) {
+	termios, err := unix.IoctlGetTermios(fd, ioctlReadTermios)
+	if err != nil {
+		return nil, err
+	}
+
+	newState := *termios
+	newState.Lflag &^= unix.ECHO
+	newState.Lflag |= unix.ICANON | unix.ISIG
+	newState.Iflag |= unix.ICRNL
+	if err := unix.IoctlSetTermios(fd, ioctlWriteTermios, &newState); err != nil {
+		return nil, err
+	}
+
+	defer unix.IoctlSetTermios(fd, ioctlWriteTermios, termios)
+
+	return readPasswordLine(passwordReader(fd))
+}
diff --git a/vendor/golang.org/x/crypto/ssh/terminal/util_aix.go b/vendor/golang.org/x/crypto/ssh/terminal/util_aix.go
new file mode 100644
index 0000000..dfcd627
--- /dev/null
+++ b/vendor/golang.org/x/crypto/ssh/terminal/util_aix.go
@@ -0,0 +1,12 @@
+// Copyright 2018 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build aix
+
+package terminal
+
+import "golang.org/x/sys/unix"
+
+const ioctlReadTermios = unix.TCGETS
+const ioctlWriteTermios = unix.TCSETS
diff --git a/vendor/golang.org/x/crypto/ssh/terminal/util_bsd.go b/vendor/golang.org/x/crypto/ssh/terminal/util_bsd.go
new file mode 100644
index 0000000..cb23a59
--- /dev/null
+++ b/vendor/golang.org/x/crypto/ssh/terminal/util_bsd.go
@@ -0,0 +1,12 @@
+// Copyright 2013 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build darwin dragonfly freebsd netbsd openbsd
+
+package terminal
+
+import "golang.org/x/sys/unix"
+
+const ioctlReadTermios = unix.TIOCGETA
+const ioctlWriteTermios = unix.TIOCSETA
diff --git a/vendor/golang.org/x/crypto/ssh/terminal/util_linux.go b/vendor/golang.org/x/crypto/ssh/terminal/util_linux.go
new file mode 100644
index 0000000..5fadfe8
--- /dev/null
+++ b/vendor/golang.org/x/crypto/ssh/terminal/util_linux.go
@@ -0,0 +1,10 @@
+// Copyright 2013 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package terminal
+
+import "golang.org/x/sys/unix"
+
+const ioctlReadTermios = unix.TCGETS
+const ioctlWriteTermios = unix.TCSETS
diff --git a/vendor/golang.org/x/crypto/ssh/terminal/util_plan9.go b/vendor/golang.org/x/crypto/ssh/terminal/util_plan9.go
new file mode 100644
index 0000000..9317ac7
--- /dev/null
+++ b/vendor/golang.org/x/crypto/ssh/terminal/util_plan9.go
@@ -0,0 +1,58 @@
+// Copyright 2016 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// Package terminal provides support functions for dealing with terminals, as
+// commonly found on UNIX systems.
+//
+// Putting a terminal into raw mode is the most common requirement:
+//
+// 	oldState, err := terminal.MakeRaw(0)
+// 	if err != nil {
+// 	        panic(err)
+// 	}
+// 	defer terminal.Restore(0, oldState)
+package terminal
+
+import (
+	"fmt"
+	"runtime"
+)
+
+type State struct{}
+
+// IsTerminal returns whether the given file descriptor is a terminal.
+func IsTerminal(fd int) bool {
+	return false
+}
+
+// MakeRaw put the terminal connected to the given file descriptor into raw
+// mode and returns the previous state of the terminal so that it can be
+// restored.
+func MakeRaw(fd int) (*State, error) {
+	return nil, fmt.Errorf("terminal: MakeRaw not implemented on %s/%s", runtime.GOOS, runtime.GOARCH)
+}
+
+// GetState returns the current state of a terminal which may be useful to
+// restore the terminal after a signal.
+func GetState(fd int) (*State, error) {
+	return nil, fmt.Errorf("terminal: GetState not implemented on %s/%s", runtime.GOOS, runtime.GOARCH)
+}
+
+// Restore restores the terminal connected to the given file descriptor to a
+// previous state.
+func Restore(fd int, state *State) error {
+	return fmt.Errorf("terminal: Restore not implemented on %s/%s", runtime.GOOS, runtime.GOARCH)
+}
+
+// GetSize returns the dimensions of the given terminal.
+func GetSize(fd int) (width, height int, err error) {
+	return 0, 0, fmt.Errorf("terminal: GetSize not implemented on %s/%s", runtime.GOOS, runtime.GOARCH)
+}
+
+// ReadPassword reads a line of input from a terminal without local echo.  This
+// is commonly used for inputting passwords and other sensitive data. The slice
+// returned does not include the \n.
+func ReadPassword(fd int) ([]byte, error) {
+	return nil, fmt.Errorf("terminal: ReadPassword not implemented on %s/%s", runtime.GOOS, runtime.GOARCH)
+}
diff --git a/vendor/golang.org/x/crypto/ssh/terminal/util_solaris.go b/vendor/golang.org/x/crypto/ssh/terminal/util_solaris.go
new file mode 100644
index 0000000..3d5f06a
--- /dev/null
+++ b/vendor/golang.org/x/crypto/ssh/terminal/util_solaris.go
@@ -0,0 +1,124 @@
+// Copyright 2015 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build solaris
+
+package terminal // import "golang.org/x/crypto/ssh/terminal"
+
+import (
+	"golang.org/x/sys/unix"
+	"io"
+	"syscall"
+)
+
+// State contains the state of a terminal.
+type State struct {
+	termios unix.Termios
+}
+
+// IsTerminal returns whether the given file descriptor is a terminal.
+func IsTerminal(fd int) bool {
+	_, err := unix.IoctlGetTermio(fd, unix.TCGETA)
+	return err == nil
+}
+
+// ReadPassword reads a line of input from a terminal without local echo.  This
+// is commonly used for inputting passwords and other sensitive data. The slice
+// returned does not include the \n.
+func ReadPassword(fd int) ([]byte, error) {
+	// see also: http://src.illumos.org/source/xref/illumos-gate/usr/src/lib/libast/common/uwin/getpass.c
+	val, err := unix.IoctlGetTermios(fd, unix.TCGETS)
+	if err != nil {
+		return nil, err
+	}
+	oldState := *val
+
+	newState := oldState
+	newState.Lflag &^= syscall.ECHO
+	newState.Lflag |= syscall.ICANON | syscall.ISIG
+	newState.Iflag |= syscall.ICRNL
+	err = unix.IoctlSetTermios(fd, unix.TCSETS, &newState)
+	if err != nil {
+		return nil, err
+	}
+
+	defer unix.IoctlSetTermios(fd, unix.TCSETS, &oldState)
+
+	var buf [16]byte
+	var ret []byte
+	for {
+		n, err := syscall.Read(fd, buf[:])
+		if err != nil {
+			return nil, err
+		}
+		if n == 0 {
+			if len(ret) == 0 {
+				return nil, io.EOF
+			}
+			break
+		}
+		if buf[n-1] == '\n' {
+			n--
+		}
+		ret = append(ret, buf[:n]...)
+		if n < len(buf) {
+			break
+		}
+	}
+
+	return ret, nil
+}
+
+// MakeRaw puts the terminal connected to the given file descriptor into raw
+// mode and returns the previous state of the terminal so that it can be
+// restored.
+// see http://cr.illumos.org/~webrev/andy_js/1060/
+func MakeRaw(fd int) (*State, error) {
+	termios, err := unix.IoctlGetTermios(fd, unix.TCGETS)
+	if err != nil {
+		return nil, err
+	}
+
+	oldState := State{termios: *termios}
+
+	termios.Iflag &^= unix.IGNBRK | unix.BRKINT | unix.PARMRK | unix.ISTRIP | unix.INLCR | unix.IGNCR | unix.ICRNL | unix.IXON
+	termios.Oflag &^= unix.OPOST
+	termios.Lflag &^= unix.ECHO | unix.ECHONL | unix.ICANON | unix.ISIG | unix.IEXTEN
+	termios.Cflag &^= unix.CSIZE | unix.PARENB
+	termios.Cflag |= unix.CS8
+	termios.Cc[unix.VMIN] = 1
+	termios.Cc[unix.VTIME] = 0
+
+	if err := unix.IoctlSetTermios(fd, unix.TCSETS, termios); err != nil {
+		return nil, err
+	}
+
+	return &oldState, nil
+}
+
+// Restore restores the terminal connected to the given file descriptor to a
+// previous state.
+func Restore(fd int, oldState *State) error {
+	return unix.IoctlSetTermios(fd, unix.TCSETS, &oldState.termios)
+}
+
+// GetState returns the current state of a terminal which may be useful to
+// restore the terminal after a signal.
+func GetState(fd int) (*State, error) {
+	termios, err := unix.IoctlGetTermios(fd, unix.TCGETS)
+	if err != nil {
+		return nil, err
+	}
+
+	return &State{termios: *termios}, nil
+}
+
+// GetSize returns the dimensions of the given terminal.
+func GetSize(fd int) (width, height int, err error) {
+	ws, err := unix.IoctlGetWinsize(fd, unix.TIOCGWINSZ)
+	if err != nil {
+		return 0, 0, err
+	}
+	return int(ws.Col), int(ws.Row), nil
+}
diff --git a/vendor/golang.org/x/crypto/ssh/terminal/util_windows.go b/vendor/golang.org/x/crypto/ssh/terminal/util_windows.go
new file mode 100644
index 0000000..5cfdf8f
--- /dev/null
+++ b/vendor/golang.org/x/crypto/ssh/terminal/util_windows.go
@@ -0,0 +1,105 @@
+// Copyright 2011 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build windows
+
+// Package terminal provides support functions for dealing with terminals, as
+// commonly found on UNIX systems.
+//
+// Putting a terminal into raw mode is the most common requirement:
+//
+// 	oldState, err := terminal.MakeRaw(0)
+// 	if err != nil {
+// 	        panic(err)
+// 	}
+// 	defer terminal.Restore(0, oldState)
+package terminal
+
+import (
+	"os"
+
+	"golang.org/x/sys/windows"
+)
+
+type State struct {
+	mode uint32
+}
+
+// IsTerminal returns whether the given file descriptor is a terminal.
+func IsTerminal(fd int) bool {
+	var st uint32
+	err := windows.GetConsoleMode(windows.Handle(fd), &st)
+	return err == nil
+}
+
+// MakeRaw put the terminal connected to the given file descriptor into raw
+// mode and returns the previous state of the terminal so that it can be
+// restored.
+func MakeRaw(fd int) (*State, error) {
+	var st uint32
+	if err := windows.GetConsoleMode(windows.Handle(fd), &st); err != nil {
+		return nil, err
+	}
+	raw := st &^ (windows.ENABLE_ECHO_INPUT | windows.ENABLE_PROCESSED_INPUT | windows.ENABLE_LINE_INPUT | windows.ENABLE_PROCESSED_OUTPUT)
+	if err := windows.SetConsoleMode(windows.Handle(fd), raw); err != nil {
+		return nil, err
+	}
+	return &State{st}, nil
+}
+
+// GetState returns the current state of a terminal which may be useful to
+// restore the terminal after a signal.
+func GetState(fd int) (*State, error) {
+	var st uint32
+	if err := windows.GetConsoleMode(windows.Handle(fd), &st); err != nil {
+		return nil, err
+	}
+	return &State{st}, nil
+}
+
+// Restore restores the terminal connected to the given file descriptor to a
+// previous state.
+func Restore(fd int, state *State) error {
+	return windows.SetConsoleMode(windows.Handle(fd), state.mode)
+}
+
+// GetSize returns the visible dimensions of the given terminal.
+//
+// These dimensions don't include any scrollback buffer height.
+func GetSize(fd int) (width, height int, err error) {
+	var info windows.ConsoleScreenBufferInfo
+	if err := windows.GetConsoleScreenBufferInfo(windows.Handle(fd), &info); err != nil {
+		return 0, 0, err
+	}
+	return int(info.Window.Right - info.Window.Left + 1), int(info.Window.Bottom - info.Window.Top + 1), nil
+}
+
+// ReadPassword reads a line of input from a terminal without local echo.  This
+// is commonly used for inputting passwords and other sensitive data. The slice
+// returned does not include the \n.
+func ReadPassword(fd int) ([]byte, error) {
+	var st uint32
+	if err := windows.GetConsoleMode(windows.Handle(fd), &st); err != nil {
+		return nil, err
+	}
+	old := st
+
+	st &^= (windows.ENABLE_ECHO_INPUT)
+	st |= (windows.ENABLE_PROCESSED_INPUT | windows.ENABLE_LINE_INPUT | windows.ENABLE_PROCESSED_OUTPUT)
+	if err := windows.SetConsoleMode(windows.Handle(fd), st); err != nil {
+		return nil, err
+	}
+
+	defer windows.SetConsoleMode(windows.Handle(fd), old)
+
+	var h windows.Handle
+	p, _ := windows.GetCurrentProcess()
+	if err := windows.DuplicateHandle(p, windows.Handle(fd), p, &h, 0, false, windows.DUPLICATE_SAME_ACCESS); err != nil {
+		return nil, err
+	}
+
+	f := os.NewFile(uintptr(h), "stdin")
+	defer f.Close()
+	return readPasswordLine(f)
+}
diff --git a/vendor/golang.org/x/sys/windows/aliases.go b/vendor/golang.org/x/sys/windows/aliases.go
new file mode 100644
index 0000000..af3af60
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/aliases.go
@@ -0,0 +1,13 @@
+// Copyright 2018 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build windows
+// +build go1.9
+
+package windows
+
+import "syscall"
+
+type Errno = syscall.Errno
+type SysProcAttr = syscall.SysProcAttr
diff --git a/vendor/golang.org/x/sys/windows/asm_windows_386.s b/vendor/golang.org/x/sys/windows/asm_windows_386.s
new file mode 100644
index 0000000..21d994d
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/asm_windows_386.s
@@ -0,0 +1,13 @@
+// Copyright 2009 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+//
+// System calls for 386, Windows are implemented in runtime/syscall_windows.goc
+//
+
+TEXT ·getprocaddress(SB), 7, $0-16
+	JMP	syscall·getprocaddress(SB)
+
+TEXT ·loadlibrary(SB), 7, $0-12
+	JMP	syscall·loadlibrary(SB)
diff --git a/vendor/golang.org/x/sys/windows/asm_windows_amd64.s b/vendor/golang.org/x/sys/windows/asm_windows_amd64.s
new file mode 100644
index 0000000..5bfdf79
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/asm_windows_amd64.s
@@ -0,0 +1,13 @@
+// Copyright 2009 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+//
+// System calls for amd64, Windows are implemented in runtime/syscall_windows.goc
+//
+
+TEXT ·getprocaddress(SB), 7, $0-32
+	JMP	syscall·getprocaddress(SB)
+
+TEXT ·loadlibrary(SB), 7, $0-24
+	JMP	syscall·loadlibrary(SB)
diff --git a/vendor/golang.org/x/sys/windows/asm_windows_arm.s b/vendor/golang.org/x/sys/windows/asm_windows_arm.s
new file mode 100644
index 0000000..55d8b91
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/asm_windows_arm.s
@@ -0,0 +1,11 @@
+// Copyright 2018 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+#include "textflag.h"
+
+TEXT ·getprocaddress(SB),NOSPLIT,$0
+	B	syscall·getprocaddress(SB)
+
+TEXT ·loadlibrary(SB),NOSPLIT,$0
+	B	syscall·loadlibrary(SB)
diff --git a/vendor/golang.org/x/sys/windows/dll_windows.go b/vendor/golang.org/x/sys/windows/dll_windows.go
new file mode 100644
index 0000000..ba67658
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/dll_windows.go
@@ -0,0 +1,378 @@
+// Copyright 2011 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package windows
+
+import (
+	"sync"
+	"sync/atomic"
+	"syscall"
+	"unsafe"
+)
+
+// DLLError describes reasons for DLL load failures.
+type DLLError struct {
+	Err     error
+	ObjName string
+	Msg     string
+}
+
+func (e *DLLError) Error() string { return e.Msg }
+
+// Implemented in runtime/syscall_windows.goc; we provide jumps to them in our assembly file.
+func loadlibrary(filename *uint16) (handle uintptr, err syscall.Errno)
+func getprocaddress(handle uintptr, procname *uint8) (proc uintptr, err syscall.Errno)
+
+// A DLL implements access to a single DLL.
+type DLL struct {
+	Name   string
+	Handle Handle
+}
+
+// LoadDLL loads DLL file into memory.
+//
+// Warning: using LoadDLL without an absolute path name is subject to
+// DLL preloading attacks. To safely load a system DLL, use LazyDLL
+// with System set to true, or use LoadLibraryEx directly.
+func LoadDLL(name string) (dll *DLL, err error) {
+	namep, err := UTF16PtrFromString(name)
+	if err != nil {
+		return nil, err
+	}
+	h, e := loadlibrary(namep)
+	if e != 0 {
+		return nil, &DLLError{
+			Err:     e,
+			ObjName: name,
+			Msg:     "Failed to load " + name + ": " + e.Error(),
+		}
+	}
+	d := &DLL{
+		Name:   name,
+		Handle: Handle(h),
+	}
+	return d, nil
+}
+
+// MustLoadDLL is like LoadDLL but panics if load operation failes.
+func MustLoadDLL(name string) *DLL {
+	d, e := LoadDLL(name)
+	if e != nil {
+		panic(e)
+	}
+	return d
+}
+
+// FindProc searches DLL d for procedure named name and returns *Proc
+// if found. It returns an error if search fails.
+func (d *DLL) FindProc(name string) (proc *Proc, err error) {
+	namep, err := BytePtrFromString(name)
+	if err != nil {
+		return nil, err
+	}
+	a, e := getprocaddress(uintptr(d.Handle), namep)
+	if e != 0 {
+		return nil, &DLLError{
+			Err:     e,
+			ObjName: name,
+			Msg:     "Failed to find " + name + " procedure in " + d.Name + ": " + e.Error(),
+		}
+	}
+	p := &Proc{
+		Dll:  d,
+		Name: name,
+		addr: a,
+	}
+	return p, nil
+}
+
+// MustFindProc is like FindProc but panics if search fails.
+func (d *DLL) MustFindProc(name string) *Proc {
+	p, e := d.FindProc(name)
+	if e != nil {
+		panic(e)
+	}
+	return p
+}
+
+// Release unloads DLL d from memory.
+func (d *DLL) Release() (err error) {
+	return FreeLibrary(d.Handle)
+}
+
+// A Proc implements access to a procedure inside a DLL.
+type Proc struct {
+	Dll  *DLL
+	Name string
+	addr uintptr
+}
+
+// Addr returns the address of the procedure represented by p.
+// The return value can be passed to Syscall to run the procedure.
+func (p *Proc) Addr() uintptr {
+	return p.addr
+}
+
+//go:uintptrescapes
+
+// Call executes procedure p with arguments a. It will panic, if more than 15 arguments
+// are supplied.
+//
+// The returned error is always non-nil, constructed from the result of GetLastError.
+// Callers must inspect the primary return value to decide whether an error occurred
+// (according to the semantics of the specific function being called) before consulting
+// the error. The error will be guaranteed to contain windows.Errno.
+func (p *Proc) Call(a ...uintptr) (r1, r2 uintptr, lastErr error) {
+	switch len(a) {
+	case 0:
+		return syscall.Syscall(p.Addr(), uintptr(len(a)), 0, 0, 0)
+	case 1:
+		return syscall.Syscall(p.Addr(), uintptr(len(a)), a[0], 0, 0)
+	case 2:
+		return syscall.Syscall(p.Addr(), uintptr(len(a)), a[0], a[1], 0)
+	case 3:
+		return syscall.Syscall(p.Addr(), uintptr(len(a)), a[0], a[1], a[2])
+	case 4:
+		return syscall.Syscall6(p.Addr(), uintptr(len(a)), a[0], a[1], a[2], a[3], 0, 0)
+	case 5:
+		return syscall.Syscall6(p.Addr(), uintptr(len(a)), a[0], a[1], a[2], a[3], a[4], 0)
+	case 6:
+		return syscall.Syscall6(p.Addr(), uintptr(len(a)), a[0], a[1], a[2], a[3], a[4], a[5])
+	case 7:
+		return syscall.Syscall9(p.Addr(), uintptr(len(a)), a[0], a[1], a[2], a[3], a[4], a[5], a[6], 0, 0)
+	case 8:
+		return syscall.Syscall9(p.Addr(), uintptr(len(a)), a[0], a[1], a[2], a[3], a[4], a[5], a[6], a[7], 0)
+	case 9:
+		return syscall.Syscall9(p.Addr(), uintptr(len(a)), a[0], a[1], a[2], a[3], a[4], a[5], a[6], a[7], a[8])
+	case 10:
+		return syscall.Syscall12(p.Addr(), uintptr(len(a)), a[0], a[1], a[2], a[3], a[4], a[5], a[6], a[7], a[8], a[9], 0, 0)
+	case 11:
+		return syscall.Syscall12(p.Addr(), uintptr(len(a)), a[0], a[1], a[2], a[3], a[4], a[5], a[6], a[7], a[8], a[9], a[10], 0)
+	case 12:
+		return syscall.Syscall12(p.Addr(), uintptr(len(a)), a[0], a[1], a[2], a[3], a[4], a[5], a[6], a[7], a[8], a[9], a[10], a[11])
+	case 13:
+		return syscall.Syscall15(p.Addr(), uintptr(len(a)), a[0], a[1], a[2], a[3], a[4], a[5], a[6], a[7], a[8], a[9], a[10], a[11], a[12], 0, 0)
+	case 14:
+		return syscall.Syscall15(p.Addr(), uintptr(len(a)), a[0], a[1], a[2], a[3], a[4], a[5], a[6], a[7], a[8], a[9], a[10], a[11], a[12], a[13], 0)
+	case 15:
+		return syscall.Syscall15(p.Addr(), uintptr(len(a)), a[0], a[1], a[2], a[3], a[4], a[5], a[6], a[7], a[8], a[9], a[10], a[11], a[12], a[13], a[14])
+	default:
+		panic("Call " + p.Name + " with too many arguments " + itoa(len(a)) + ".")
+	}
+}
+
+// A LazyDLL implements access to a single DLL.
+// It will delay the load of the DLL until the first
+// call to its Handle method or to one of its
+// LazyProc's Addr method.
+type LazyDLL struct {
+	Name string
+
+	// System determines whether the DLL must be loaded from the
+	// Windows System directory, bypassing the normal DLL search
+	// path.
+	System bool
+
+	mu  sync.Mutex
+	dll *DLL // non nil once DLL is loaded
+}
+
+// Load loads DLL file d.Name into memory. It returns an error if fails.
+// Load will not try to load DLL, if it is already loaded into memory.
+func (d *LazyDLL) Load() error {
+	// Non-racy version of:
+	// if d.dll != nil {
+	if atomic.LoadPointer((*unsafe.Pointer)(unsafe.Pointer(&d.dll))) != nil {
+		return nil
+	}
+	d.mu.Lock()
+	defer d.mu.Unlock()
+	if d.dll != nil {
+		return nil
+	}
+
+	// kernel32.dll is special, since it's where LoadLibraryEx comes from.
+	// The kernel already special-cases its name, so it's always
+	// loaded from system32.
+	var dll *DLL
+	var err error
+	if d.Name == "kernel32.dll" {
+		dll, err = LoadDLL(d.Name)
+	} else {
+		dll, err = loadLibraryEx(d.Name, d.System)
+	}
+	if err != nil {
+		return err
+	}
+
+	// Non-racy version of:
+	// d.dll = dll
+	atomic.StorePointer((*unsafe.Pointer)(unsafe.Pointer(&d.dll)), unsafe.Pointer(dll))
+	return nil
+}
+
+// mustLoad is like Load but panics if search fails.
+func (d *LazyDLL) mustLoad() {
+	e := d.Load()
+	if e != nil {
+		panic(e)
+	}
+}
+
+// Handle returns d's module handle.
+func (d *LazyDLL) Handle() uintptr {
+	d.mustLoad()
+	return uintptr(d.dll.Handle)
+}
+
+// NewProc returns a LazyProc for accessing the named procedure in the DLL d.
+func (d *LazyDLL) NewProc(name string) *LazyProc {
+	return &LazyProc{l: d, Name: name}
+}
+
+// NewLazyDLL creates new LazyDLL associated with DLL file.
+func NewLazyDLL(name string) *LazyDLL {
+	return &LazyDLL{Name: name}
+}
+
+// NewLazySystemDLL is like NewLazyDLL, but will only
+// search Windows System directory for the DLL if name is
+// a base name (like "advapi32.dll").
+func NewLazySystemDLL(name string) *LazyDLL {
+	return &LazyDLL{Name: name, System: true}
+}
+
+// A LazyProc implements access to a procedure inside a LazyDLL.
+// It delays the lookup until the Addr method is called.
+type LazyProc struct {
+	Name string
+
+	mu   sync.Mutex
+	l    *LazyDLL
+	proc *Proc
+}
+
+// Find searches DLL for procedure named p.Name. It returns
+// an error if search fails. Find will not search procedure,
+// if it is already found and loaded into memory.
+func (p *LazyProc) Find() error {
+	// Non-racy version of:
+	// if p.proc == nil {
+	if atomic.LoadPointer((*unsafe.Pointer)(unsafe.Pointer(&p.proc))) == nil {
+		p.mu.Lock()
+		defer p.mu.Unlock()
+		if p.proc == nil {
+			e := p.l.Load()
+			if e != nil {
+				return e
+			}
+			proc, e := p.l.dll.FindProc(p.Name)
+			if e != nil {
+				return e
+			}
+			// Non-racy version of:
+			// p.proc = proc
+			atomic.StorePointer((*unsafe.Pointer)(unsafe.Pointer(&p.proc)), unsafe.Pointer(proc))
+		}
+	}
+	return nil
+}
+
+// mustFind is like Find but panics if search fails.
+func (p *LazyProc) mustFind() {
+	e := p.Find()
+	if e != nil {
+		panic(e)
+	}
+}
+
+// Addr returns the address of the procedure represented by p.
+// The return value can be passed to Syscall to run the procedure.
+// It will panic if the procedure cannot be found.
+func (p *LazyProc) Addr() uintptr {
+	p.mustFind()
+	return p.proc.Addr()
+}
+
+//go:uintptrescapes
+
+// Call executes procedure p with arguments a. It will panic, if more than 15 arguments
+// are supplied. It will also panic if the procedure cannot be found.
+//
+// The returned error is always non-nil, constructed from the result of GetLastError.
+// Callers must inspect the primary return value to decide whether an error occurred
+// (according to the semantics of the specific function being called) before consulting
+// the error. The error will be guaranteed to contain windows.Errno.
+func (p *LazyProc) Call(a ...uintptr) (r1, r2 uintptr, lastErr error) {
+	p.mustFind()
+	return p.proc.Call(a...)
+}
+
+var canDoSearchSystem32Once struct {
+	sync.Once
+	v bool
+}
+
+func initCanDoSearchSystem32() {
+	// https://msdn.microsoft.com/en-us/library/ms684179(v=vs.85).aspx says:
+	// "Windows 7, Windows Server 2008 R2, Windows Vista, and Windows
+	// Server 2008: The LOAD_LIBRARY_SEARCH_* flags are available on
+	// systems that have KB2533623 installed. To determine whether the
+	// flags are available, use GetProcAddress to get the address of the
+	// AddDllDirectory, RemoveDllDirectory, or SetDefaultDllDirectories
+	// function. If GetProcAddress succeeds, the LOAD_LIBRARY_SEARCH_*
+	// flags can be used with LoadLibraryEx."
+	canDoSearchSystem32Once.v = (modkernel32.NewProc("AddDllDirectory").Find() == nil)
+}
+
+func canDoSearchSystem32() bool {
+	canDoSearchSystem32Once.Do(initCanDoSearchSystem32)
+	return canDoSearchSystem32Once.v
+}
+
+func isBaseName(name string) bool {
+	for _, c := range name {
+		if c == ':' || c == '/' || c == '\\' {
+			return false
+		}
+	}
+	return true
+}
+
+// loadLibraryEx wraps the Windows LoadLibraryEx function.
+//
+// See https://msdn.microsoft.com/en-us/library/windows/desktop/ms684179(v=vs.85).aspx
+//
+// If name is not an absolute path, LoadLibraryEx searches for the DLL
+// in a variety of automatic locations unless constrained by flags.
+// See: https://msdn.microsoft.com/en-us/library/ff919712%28VS.85%29.aspx
+func loadLibraryEx(name string, system bool) (*DLL, error) {
+	loadDLL := name
+	var flags uintptr
+	if system {
+		if canDoSearchSystem32() {
+			const LOAD_LIBRARY_SEARCH_SYSTEM32 = 0x00000800
+			flags = LOAD_LIBRARY_SEARCH_SYSTEM32
+		} else if isBaseName(name) {
+			// WindowsXP or unpatched Windows machine
+			// trying to load "foo.dll" out of the system
+			// folder, but LoadLibraryEx doesn't support
+			// that yet on their system, so emulate it.
+			systemdir, err := GetSystemDirectory()
+			if err != nil {
+				return nil, err
+			}
+			loadDLL = systemdir + "\\" + name
+		}
+	}
+	h, err := LoadLibraryEx(loadDLL, 0, flags)
+	if err != nil {
+		return nil, err
+	}
+	return &DLL{Name: name, Handle: h}, nil
+}
+
+type errString string
+
+func (s errString) Error() string { return string(s) }
diff --git a/vendor/golang.org/x/sys/windows/env_windows.go b/vendor/golang.org/x/sys/windows/env_windows.go
new file mode 100644
index 0000000..f482a9f
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/env_windows.go
@@ -0,0 +1,61 @@
+// Copyright 2010 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// Windows environment variables.
+
+package windows
+
+import (
+	"syscall"
+	"unicode/utf16"
+	"unsafe"
+)
+
+func Getenv(key string) (value string, found bool) {
+	return syscall.Getenv(key)
+}
+
+func Setenv(key, value string) error {
+	return syscall.Setenv(key, value)
+}
+
+func Clearenv() {
+	syscall.Clearenv()
+}
+
+func Environ() []string {
+	return syscall.Environ()
+}
+
+// Returns a default environment associated with the token, rather than the current
+// process. If inheritExisting is true, then this environment also inherits the
+// environment of the current process.
+func (token Token) Environ(inheritExisting bool) (env []string, err error) {
+	var block *uint16
+	err = CreateEnvironmentBlock(&block, token, inheritExisting)
+	if err != nil {
+		return nil, err
+	}
+	defer DestroyEnvironmentBlock(block)
+	blockp := uintptr(unsafe.Pointer(block))
+	for {
+		entry := (*[(1 << 30) - 1]uint16)(unsafe.Pointer(blockp))[:]
+		for i, v := range entry {
+			if v == 0 {
+				entry = entry[:i]
+				break
+			}
+		}
+		if len(entry) == 0 {
+			break
+		}
+		env = append(env, string(utf16.Decode(entry)))
+		blockp += 2 * (uintptr(len(entry)) + 1)
+	}
+	return env, nil
+}
+
+func Unsetenv(key string) error {
+	return syscall.Unsetenv(key)
+}
diff --git a/vendor/golang.org/x/sys/windows/eventlog.go b/vendor/golang.org/x/sys/windows/eventlog.go
new file mode 100644
index 0000000..40af946
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/eventlog.go
@@ -0,0 +1,20 @@
+// Copyright 2012 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build windows
+
+package windows
+
+const (
+	EVENTLOG_SUCCESS          = 0
+	EVENTLOG_ERROR_TYPE       = 1
+	EVENTLOG_WARNING_TYPE     = 2
+	EVENTLOG_INFORMATION_TYPE = 4
+	EVENTLOG_AUDIT_SUCCESS    = 8
+	EVENTLOG_AUDIT_FAILURE    = 16
+)
+
+//sys	RegisterEventSource(uncServerName *uint16, sourceName *uint16) (handle Handle, err error) [failretval==0] = advapi32.RegisterEventSourceW
+//sys	DeregisterEventSource(handle Handle) (err error) = advapi32.DeregisterEventSource
+//sys	ReportEvent(log Handle, etype uint16, category uint16, eventId uint32, usrSId uintptr, numStrings uint16, dataSize uint32, strings **uint16, rawData *byte) (err error) = advapi32.ReportEventW
diff --git a/vendor/golang.org/x/sys/windows/exec_windows.go b/vendor/golang.org/x/sys/windows/exec_windows.go
new file mode 100644
index 0000000..3606c3a
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/exec_windows.go
@@ -0,0 +1,97 @@
+// Copyright 2009 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// Fork, exec, wait, etc.
+
+package windows
+
+// EscapeArg rewrites command line argument s as prescribed
+// in http://msdn.microsoft.com/en-us/library/ms880421.
+// This function returns "" (2 double quotes) if s is empty.
+// Alternatively, these transformations are done:
+// - every back slash (\) is doubled, but only if immediately
+//   followed by double quote (");
+// - every double quote (") is escaped by back slash (\);
+// - finally, s is wrapped with double quotes (arg -> "arg"),
+//   but only if there is space or tab inside s.
+func EscapeArg(s string) string {
+	if len(s) == 0 {
+		return "\"\""
+	}
+	n := len(s)
+	hasSpace := false
+	for i := 0; i < len(s); i++ {
+		switch s[i] {
+		case '"', '\\':
+			n++
+		case ' ', '\t':
+			hasSpace = true
+		}
+	}
+	if hasSpace {
+		n += 2
+	}
+	if n == len(s) {
+		return s
+	}
+
+	qs := make([]byte, n)
+	j := 0
+	if hasSpace {
+		qs[j] = '"'
+		j++
+	}
+	slashes := 0
+	for i := 0; i < len(s); i++ {
+		switch s[i] {
+		default:
+			slashes = 0
+			qs[j] = s[i]
+		case '\\':
+			slashes++
+			qs[j] = s[i]
+		case '"':
+			for ; slashes > 0; slashes-- {
+				qs[j] = '\\'
+				j++
+			}
+			qs[j] = '\\'
+			j++
+			qs[j] = s[i]
+		}
+		j++
+	}
+	if hasSpace {
+		for ; slashes > 0; slashes-- {
+			qs[j] = '\\'
+			j++
+		}
+		qs[j] = '"'
+		j++
+	}
+	return string(qs[:j])
+}
+
+func CloseOnExec(fd Handle) {
+	SetHandleInformation(Handle(fd), HANDLE_FLAG_INHERIT, 0)
+}
+
+// FullPath retrieves the full path of the specified file.
+func FullPath(name string) (path string, err error) {
+	p, err := UTF16PtrFromString(name)
+	if err != nil {
+		return "", err
+	}
+	n := uint32(100)
+	for {
+		buf := make([]uint16, n)
+		n, err = GetFullPathName(p, uint32(len(buf)), &buf[0], nil)
+		if err != nil {
+			return "", err
+		}
+		if n <= uint32(len(buf)) {
+			return UTF16ToString(buf[:n]), nil
+		}
+	}
+}
diff --git a/vendor/golang.org/x/sys/windows/memory_windows.go b/vendor/golang.org/x/sys/windows/memory_windows.go
new file mode 100644
index 0000000..f80a420
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/memory_windows.go
@@ -0,0 +1,26 @@
+// Copyright 2017 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package windows
+
+const (
+	MEM_COMMIT      = 0x00001000
+	MEM_RESERVE     = 0x00002000
+	MEM_DECOMMIT    = 0x00004000
+	MEM_RELEASE     = 0x00008000
+	MEM_RESET       = 0x00080000
+	MEM_TOP_DOWN    = 0x00100000
+	MEM_WRITE_WATCH = 0x00200000
+	MEM_PHYSICAL    = 0x00400000
+	MEM_RESET_UNDO  = 0x01000000
+	MEM_LARGE_PAGES = 0x20000000
+
+	PAGE_NOACCESS          = 0x01
+	PAGE_READONLY          = 0x02
+	PAGE_READWRITE         = 0x04
+	PAGE_WRITECOPY         = 0x08
+	PAGE_EXECUTE_READ      = 0x20
+	PAGE_EXECUTE_READWRITE = 0x40
+	PAGE_EXECUTE_WRITECOPY = 0x80
+)
diff --git a/vendor/golang.org/x/sys/windows/mkerrors.bash b/vendor/golang.org/x/sys/windows/mkerrors.bash
new file mode 100644
index 0000000..2163843
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/mkerrors.bash
@@ -0,0 +1,63 @@
+#!/bin/bash
+
+# Copyright 2019 The Go Authors. All rights reserved.
+# Use of this source code is governed by a BSD-style
+# license that can be found in the LICENSE file.
+
+set -e
+shopt -s nullglob
+
+winerror="$(printf '%s\n' "/mnt/c/Program Files (x86)/Windows Kits/"/*/Include/*/shared/winerror.h | sort -Vr | head -n 1)"
+[[ -n $winerror ]] || { echo "Unable to find winerror.h" >&2; exit 1; }
+
+declare -A errors
+
+{
+	echo "// Code generated by 'mkerrors.bash'; DO NOT EDIT."
+	echo
+	echo "package windows"
+	echo "import \"syscall\""
+	echo "const ("
+
+	while read -r line; do
+		unset vtype
+		if [[ $line =~ ^#define\ +([A-Z0-9_]+k?)\ +([A-Z0-9_]+\()?([A-Z][A-Z0-9_]+k?)\)? ]]; then
+			key="${BASH_REMATCH[1]}"
+			value="${BASH_REMATCH[3]}"
+		elif [[ $line =~ ^#define\ +([A-Z0-9_]+k?)\ +([A-Z0-9_]+\()?((0x)?[0-9A-Fa-f]+)L?\)? ]]; then
+			key="${BASH_REMATCH[1]}"
+			value="${BASH_REMATCH[3]}"
+			vtype="${BASH_REMATCH[2]}"
+		elif [[ $line =~ ^#define\ +([A-Z0-9_]+k?)\ +\(\(([A-Z]+)\)((0x)?[0-9A-Fa-f]+)L?\) ]]; then
+			key="${BASH_REMATCH[1]}"
+			value="${BASH_REMATCH[3]}"
+			vtype="${BASH_REMATCH[2]}"
+		else
+			continue
+		fi
+		[[ -n $key && -n $value ]] || continue
+		[[ -z ${errors["$key"]} ]] || continue
+		errors["$key"]="$value"
+		if [[ -v vtype ]]; then
+			if [[ $key == FACILITY_* || $key == NO_ERROR ]]; then
+				vtype=""
+			elif [[ $vtype == *HANDLE* || $vtype == *HRESULT* ]]; then
+				vtype="Handle"
+			else
+				vtype="syscall.Errno"
+			fi
+			last_vtype="$vtype"
+		else
+			vtype=""
+			if [[ $last_vtype == Handle && $value == NO_ERROR ]]; then
+				value="S_OK"
+			elif [[ $last_vtype == syscall.Errno && $value == NO_ERROR ]]; then
+				value="ERROR_SUCCESS"
+			fi
+		fi
+
+		echo "$key $vtype = $value"
+	done < "$winerror"
+
+	echo ")"
+} | gofmt > "zerrors_windows.go"
diff --git a/vendor/golang.org/x/sys/windows/mkknownfolderids.bash b/vendor/golang.org/x/sys/windows/mkknownfolderids.bash
new file mode 100644
index 0000000..ab8924e
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/mkknownfolderids.bash
@@ -0,0 +1,27 @@
+#!/bin/bash
+
+# Copyright 2019 The Go Authors. All rights reserved.
+# Use of this source code is governed by a BSD-style
+# license that can be found in the LICENSE file.
+
+set -e
+shopt -s nullglob
+
+knownfolders="$(printf '%s\n' "/mnt/c/Program Files (x86)/Windows Kits/"/*/Include/*/um/KnownFolders.h | sort -Vr | head -n 1)"
+[[ -n $knownfolders ]] || { echo "Unable to find KnownFolders.h" >&2; exit 1; }
+
+{
+	echo "// Code generated by 'mkknownfolderids.bash'; DO NOT EDIT."
+	echo
+	echo "package windows"
+	echo "type KNOWNFOLDERID GUID"
+	echo "var ("
+	while read -r line; do
+		[[ $line =~ DEFINE_KNOWN_FOLDER\((FOLDERID_[^,]+),[\t\ ]*(0x[^,]+),[\t\ ]*(0x[^,]+),[\t\ ]*(0x[^,]+),[\t\ ]*(0x[^,]+),[\t\ ]*(0x[^,]+),[\t\ ]*(0x[^,]+),[\t\ ]*(0x[^,]+),[\t\ ]*(0x[^,]+),[\t\ ]*(0x[^,]+),[\t\ ]*(0x[^,]+),[\t\ ]*(0x[^,]+)\) ]] || continue
+		printf "%s = &KNOWNFOLDERID{0x%08x, 0x%04x, 0x%04x, [8]byte{0x%02x, 0x%02x, 0x%02x, 0x%02x, 0x%02x, 0x%02x, 0x%02x, 0x%02x}}\n" \
+			"${BASH_REMATCH[1]}" $(( "${BASH_REMATCH[2]}" )) $(( "${BASH_REMATCH[3]}" )) $(( "${BASH_REMATCH[4]}" )) \
+			$(( "${BASH_REMATCH[5]}" )) $(( "${BASH_REMATCH[6]}" )) $(( "${BASH_REMATCH[7]}" )) $(( "${BASH_REMATCH[8]}" )) \
+			$(( "${BASH_REMATCH[9]}" )) $(( "${BASH_REMATCH[10]}" )) $(( "${BASH_REMATCH[11]}" )) $(( "${BASH_REMATCH[12]}" ))
+	done < "$knownfolders"
+	echo ")"
+} | gofmt > "zknownfolderids_windows.go"
diff --git a/vendor/golang.org/x/sys/windows/mksyscall.go b/vendor/golang.org/x/sys/windows/mksyscall.go
new file mode 100644
index 0000000..6277057
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/mksyscall.go
@@ -0,0 +1,9 @@
+// Copyright 2009 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build generate
+
+package windows
+
+//go:generate go run $GOROOT/src/syscall/mksyscall_windows.go -output zsyscall_windows.go eventlog.go service.go syscall_windows.go security_windows.go
diff --git a/vendor/golang.org/x/sys/windows/race.go b/vendor/golang.org/x/sys/windows/race.go
new file mode 100644
index 0000000..a74e3e2
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/race.go
@@ -0,0 +1,30 @@
+// Copyright 2012 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build windows,race
+
+package windows
+
+import (
+	"runtime"
+	"unsafe"
+)
+
+const raceenabled = true
+
+func raceAcquire(addr unsafe.Pointer) {
+	runtime.RaceAcquire(addr)
+}
+
+func raceReleaseMerge(addr unsafe.Pointer) {
+	runtime.RaceReleaseMerge(addr)
+}
+
+func raceReadRange(addr unsafe.Pointer, len int) {
+	runtime.RaceReadRange(addr, len)
+}
+
+func raceWriteRange(addr unsafe.Pointer, len int) {
+	runtime.RaceWriteRange(addr, len)
+}
diff --git a/vendor/golang.org/x/sys/windows/race0.go b/vendor/golang.org/x/sys/windows/race0.go
new file mode 100644
index 0000000..e44a3cb
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/race0.go
@@ -0,0 +1,25 @@
+// Copyright 2012 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build windows,!race
+
+package windows
+
+import (
+	"unsafe"
+)
+
+const raceenabled = false
+
+func raceAcquire(addr unsafe.Pointer) {
+}
+
+func raceReleaseMerge(addr unsafe.Pointer) {
+}
+
+func raceReadRange(addr unsafe.Pointer, len int) {
+}
+
+func raceWriteRange(addr unsafe.Pointer, len int) {
+}
diff --git a/vendor/golang.org/x/sys/windows/security_windows.go b/vendor/golang.org/x/sys/windows/security_windows.go
new file mode 100644
index 0000000..c605ee6
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/security_windows.go
@@ -0,0 +1,1396 @@
+// Copyright 2012 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package windows
+
+import (
+	"syscall"
+	"unsafe"
+)
+
+const (
+	NameUnknown          = 0
+	NameFullyQualifiedDN = 1
+	NameSamCompatible    = 2
+	NameDisplay          = 3
+	NameUniqueId         = 6
+	NameCanonical        = 7
+	NameUserPrincipal    = 8
+	NameCanonicalEx      = 9
+	NameServicePrincipal = 10
+	NameDnsDomain        = 12
+)
+
+// This function returns 1 byte BOOLEAN rather than the 4 byte BOOL.
+// http://blogs.msdn.com/b/drnick/archive/2007/12/19/windows-and-upn-format-credentials.aspx
+//sys	TranslateName(accName *uint16, accNameFormat uint32, desiredNameFormat uint32, translatedName *uint16, nSize *uint32) (err error) [failretval&0xff==0] = secur32.TranslateNameW
+//sys	GetUserNameEx(nameFormat uint32, nameBuffre *uint16, nSize *uint32) (err error) [failretval&0xff==0] = secur32.GetUserNameExW
+
+// TranslateAccountName converts a directory service
+// object name from one format to another.
+func TranslateAccountName(username string, from, to uint32, initSize int) (string, error) {
+	u, e := UTF16PtrFromString(username)
+	if e != nil {
+		return "", e
+	}
+	n := uint32(50)
+	for {
+		b := make([]uint16, n)
+		e = TranslateName(u, from, to, &b[0], &n)
+		if e == nil {
+			return UTF16ToString(b[:n]), nil
+		}
+		if e != ERROR_INSUFFICIENT_BUFFER {
+			return "", e
+		}
+		if n <= uint32(len(b)) {
+			return "", e
+		}
+	}
+}
+
+const (
+	// do not reorder
+	NetSetupUnknownStatus = iota
+	NetSetupUnjoined
+	NetSetupWorkgroupName
+	NetSetupDomainName
+)
+
+type UserInfo10 struct {
+	Name       *uint16
+	Comment    *uint16
+	UsrComment *uint16
+	FullName   *uint16
+}
+
+//sys	NetUserGetInfo(serverName *uint16, userName *uint16, level uint32, buf **byte) (neterr error) = netapi32.NetUserGetInfo
+//sys	NetGetJoinInformation(server *uint16, name **uint16, bufType *uint32) (neterr error) = netapi32.NetGetJoinInformation
+//sys	NetApiBufferFree(buf *byte) (neterr error) = netapi32.NetApiBufferFree
+
+const (
+	// do not reorder
+	SidTypeUser = 1 + iota
+	SidTypeGroup
+	SidTypeDomain
+	SidTypeAlias
+	SidTypeWellKnownGroup
+	SidTypeDeletedAccount
+	SidTypeInvalid
+	SidTypeUnknown
+	SidTypeComputer
+	SidTypeLabel
+)
+
+type SidIdentifierAuthority struct {
+	Value [6]byte
+}
+
+var (
+	SECURITY_NULL_SID_AUTHORITY        = SidIdentifierAuthority{[6]byte{0, 0, 0, 0, 0, 0}}
+	SECURITY_WORLD_SID_AUTHORITY       = SidIdentifierAuthority{[6]byte{0, 0, 0, 0, 0, 1}}
+	SECURITY_LOCAL_SID_AUTHORITY       = SidIdentifierAuthority{[6]byte{0, 0, 0, 0, 0, 2}}
+	SECURITY_CREATOR_SID_AUTHORITY     = SidIdentifierAuthority{[6]byte{0, 0, 0, 0, 0, 3}}
+	SECURITY_NON_UNIQUE_AUTHORITY      = SidIdentifierAuthority{[6]byte{0, 0, 0, 0, 0, 4}}
+	SECURITY_NT_AUTHORITY              = SidIdentifierAuthority{[6]byte{0, 0, 0, 0, 0, 5}}
+	SECURITY_MANDATORY_LABEL_AUTHORITY = SidIdentifierAuthority{[6]byte{0, 0, 0, 0, 0, 16}}
+)
+
+const (
+	SECURITY_NULL_RID                   = 0
+	SECURITY_WORLD_RID                  = 0
+	SECURITY_LOCAL_RID                  = 0
+	SECURITY_CREATOR_OWNER_RID          = 0
+	SECURITY_CREATOR_GROUP_RID          = 1
+	SECURITY_DIALUP_RID                 = 1
+	SECURITY_NETWORK_RID                = 2
+	SECURITY_BATCH_RID                  = 3
+	SECURITY_INTERACTIVE_RID            = 4
+	SECURITY_LOGON_IDS_RID              = 5
+	SECURITY_SERVICE_RID                = 6
+	SECURITY_LOCAL_SYSTEM_RID           = 18
+	SECURITY_BUILTIN_DOMAIN_RID         = 32
+	SECURITY_PRINCIPAL_SELF_RID         = 10
+	SECURITY_CREATOR_OWNER_SERVER_RID   = 0x2
+	SECURITY_CREATOR_GROUP_SERVER_RID   = 0x3
+	SECURITY_LOGON_IDS_RID_COUNT        = 0x3
+	SECURITY_ANONYMOUS_LOGON_RID        = 0x7
+	SECURITY_PROXY_RID                  = 0x8
+	SECURITY_ENTERPRISE_CONTROLLERS_RID = 0x9
+	SECURITY_SERVER_LOGON_RID           = SECURITY_ENTERPRISE_CONTROLLERS_RID
+	SECURITY_AUTHENTICATED_USER_RID     = 0xb
+	SECURITY_RESTRICTED_CODE_RID        = 0xc
+	SECURITY_NT_NON_UNIQUE_RID          = 0x15
+)
+
+// Predefined domain-relative RIDs for local groups.
+// See https://msdn.microsoft.com/en-us/library/windows/desktop/aa379649(v=vs.85).aspx
+const (
+	DOMAIN_ALIAS_RID_ADMINS                         = 0x220
+	DOMAIN_ALIAS_RID_USERS                          = 0x221
+	DOMAIN_ALIAS_RID_GUESTS                         = 0x222
+	DOMAIN_ALIAS_RID_POWER_USERS                    = 0x223
+	DOMAIN_ALIAS_RID_ACCOUNT_OPS                    = 0x224
+	DOMAIN_ALIAS_RID_SYSTEM_OPS                     = 0x225
+	DOMAIN_ALIAS_RID_PRINT_OPS                      = 0x226
+	DOMAIN_ALIAS_RID_BACKUP_OPS                     = 0x227
+	DOMAIN_ALIAS_RID_REPLICATOR                     = 0x228
+	DOMAIN_ALIAS_RID_RAS_SERVERS                    = 0x229
+	DOMAIN_ALIAS_RID_PREW2KCOMPACCESS               = 0x22a
+	DOMAIN_ALIAS_RID_REMOTE_DESKTOP_USERS           = 0x22b
+	DOMAIN_ALIAS_RID_NETWORK_CONFIGURATION_OPS      = 0x22c
+	DOMAIN_ALIAS_RID_INCOMING_FOREST_TRUST_BUILDERS = 0x22d
+	DOMAIN_ALIAS_RID_MONITORING_USERS               = 0x22e
+	DOMAIN_ALIAS_RID_LOGGING_USERS                  = 0x22f
+	DOMAIN_ALIAS_RID_AUTHORIZATIONACCESS            = 0x230
+	DOMAIN_ALIAS_RID_TS_LICENSE_SERVERS             = 0x231
+	DOMAIN_ALIAS_RID_DCOM_USERS                     = 0x232
+	DOMAIN_ALIAS_RID_IUSERS                         = 0x238
+	DOMAIN_ALIAS_RID_CRYPTO_OPERATORS               = 0x239
+	DOMAIN_ALIAS_RID_CACHEABLE_PRINCIPALS_GROUP     = 0x23b
+	DOMAIN_ALIAS_RID_NON_CACHEABLE_PRINCIPALS_GROUP = 0x23c
+	DOMAIN_ALIAS_RID_EVENT_LOG_READERS_GROUP        = 0x23d
+	DOMAIN_ALIAS_RID_CERTSVC_DCOM_ACCESS_GROUP      = 0x23e
+)
+
+//sys	LookupAccountSid(systemName *uint16, sid *SID, name *uint16, nameLen *uint32, refdDomainName *uint16, refdDomainNameLen *uint32, use *uint32) (err error) = advapi32.LookupAccountSidW
+//sys	LookupAccountName(systemName *uint16, accountName *uint16, sid *SID, sidLen *uint32, refdDomainName *uint16, refdDomainNameLen *uint32, use *uint32) (err error) = advapi32.LookupAccountNameW
+//sys	ConvertSidToStringSid(sid *SID, stringSid **uint16) (err error) = advapi32.ConvertSidToStringSidW
+//sys	ConvertStringSidToSid(stringSid *uint16, sid **SID) (err error) = advapi32.ConvertStringSidToSidW
+//sys	GetLengthSid(sid *SID) (len uint32) = advapi32.GetLengthSid
+//sys	CopySid(destSidLen uint32, destSid *SID, srcSid *SID) (err error) = advapi32.CopySid
+//sys	AllocateAndInitializeSid(identAuth *SidIdentifierAuthority, subAuth byte, subAuth0 uint32, subAuth1 uint32, subAuth2 uint32, subAuth3 uint32, subAuth4 uint32, subAuth5 uint32, subAuth6 uint32, subAuth7 uint32, sid **SID) (err error) = advapi32.AllocateAndInitializeSid
+//sys	createWellKnownSid(sidType WELL_KNOWN_SID_TYPE, domainSid *SID, sid *SID, sizeSid *uint32) (err error) = advapi32.CreateWellKnownSid
+//sys	isWellKnownSid(sid *SID, sidType WELL_KNOWN_SID_TYPE) (isWellKnown bool) = advapi32.IsWellKnownSid
+//sys	FreeSid(sid *SID) (err error) [failretval!=0] = advapi32.FreeSid
+//sys	EqualSid(sid1 *SID, sid2 *SID) (isEqual bool) = advapi32.EqualSid
+//sys	getSidIdentifierAuthority(sid *SID) (authority *SidIdentifierAuthority) = advapi32.GetSidIdentifierAuthority
+//sys	getSidSubAuthorityCount(sid *SID) (count *uint8) = advapi32.GetSidSubAuthorityCount
+//sys	getSidSubAuthority(sid *SID, index uint32) (subAuthority *uint32) = advapi32.GetSidSubAuthority
+//sys	isValidSid(sid *SID) (isValid bool) = advapi32.IsValidSid
+
+// The security identifier (SID) structure is a variable-length
+// structure used to uniquely identify users or groups.
+type SID struct{}
+
+// StringToSid converts a string-format security identifier
+// SID into a valid, functional SID.
+func StringToSid(s string) (*SID, error) {
+	var sid *SID
+	p, e := UTF16PtrFromString(s)
+	if e != nil {
+		return nil, e
+	}
+	e = ConvertStringSidToSid(p, &sid)
+	if e != nil {
+		return nil, e
+	}
+	defer LocalFree((Handle)(unsafe.Pointer(sid)))
+	return sid.Copy()
+}
+
+// LookupSID retrieves a security identifier SID for the account
+// and the name of the domain on which the account was found.
+// System specify target computer to search.
+func LookupSID(system, account string) (sid *SID, domain string, accType uint32, err error) {
+	if len(account) == 0 {
+		return nil, "", 0, syscall.EINVAL
+	}
+	acc, e := UTF16PtrFromString(account)
+	if e != nil {
+		return nil, "", 0, e
+	}
+	var sys *uint16
+	if len(system) > 0 {
+		sys, e = UTF16PtrFromString(system)
+		if e != nil {
+			return nil, "", 0, e
+		}
+	}
+	n := uint32(50)
+	dn := uint32(50)
+	for {
+		b := make([]byte, n)
+		db := make([]uint16, dn)
+		sid = (*SID)(unsafe.Pointer(&b[0]))
+		e = LookupAccountName(sys, acc, sid, &n, &db[0], &dn, &accType)
+		if e == nil {
+			return sid, UTF16ToString(db), accType, nil
+		}
+		if e != ERROR_INSUFFICIENT_BUFFER {
+			return nil, "", 0, e
+		}
+		if n <= uint32(len(b)) {
+			return nil, "", 0, e
+		}
+	}
+}
+
+// String converts SID to a string format suitable for display, storage, or transmission.
+func (sid *SID) String() string {
+	var s *uint16
+	e := ConvertSidToStringSid(sid, &s)
+	if e != nil {
+		return ""
+	}
+	defer LocalFree((Handle)(unsafe.Pointer(s)))
+	return UTF16ToString((*[(1 << 30) - 1]uint16)(unsafe.Pointer(s))[:])
+}
+
+// Len returns the length, in bytes, of a valid security identifier SID.
+func (sid *SID) Len() int {
+	return int(GetLengthSid(sid))
+}
+
+// Copy creates a duplicate of security identifier SID.
+func (sid *SID) Copy() (*SID, error) {
+	b := make([]byte, sid.Len())
+	sid2 := (*SID)(unsafe.Pointer(&b[0]))
+	e := CopySid(uint32(len(b)), sid2, sid)
+	if e != nil {
+		return nil, e
+	}
+	return sid2, nil
+}
+
+// IdentifierAuthority returns the identifier authority of the SID.
+func (sid *SID) IdentifierAuthority() SidIdentifierAuthority {
+	return *getSidIdentifierAuthority(sid)
+}
+
+// SubAuthorityCount returns the number of sub-authorities in the SID.
+func (sid *SID) SubAuthorityCount() uint8 {
+	return *getSidSubAuthorityCount(sid)
+}
+
+// SubAuthority returns the sub-authority of the SID as specified by
+// the index, which must be less than sid.SubAuthorityCount().
+func (sid *SID) SubAuthority(idx uint32) uint32 {
+	if idx >= uint32(sid.SubAuthorityCount()) {
+		panic("sub-authority index out of range")
+	}
+	return *getSidSubAuthority(sid, idx)
+}
+
+// IsValid returns whether the SID has a valid revision and length.
+func (sid *SID) IsValid() bool {
+	return isValidSid(sid)
+}
+
+// Equals compares two SIDs for equality.
+func (sid *SID) Equals(sid2 *SID) bool {
+	return EqualSid(sid, sid2)
+}
+
+// IsWellKnown determines whether the SID matches the well-known sidType.
+func (sid *SID) IsWellKnown(sidType WELL_KNOWN_SID_TYPE) bool {
+	return isWellKnownSid(sid, sidType)
+}
+
+// LookupAccount retrieves the name of the account for this SID
+// and the name of the first domain on which this SID is found.
+// System specify target computer to search for.
+func (sid *SID) LookupAccount(system string) (account, domain string, accType uint32, err error) {
+	var sys *uint16
+	if len(system) > 0 {
+		sys, err = UTF16PtrFromString(system)
+		if err != nil {
+			return "", "", 0, err
+		}
+	}
+	n := uint32(50)
+	dn := uint32(50)
+	for {
+		b := make([]uint16, n)
+		db := make([]uint16, dn)
+		e := LookupAccountSid(sys, sid, &b[0], &n, &db[0], &dn, &accType)
+		if e == nil {
+			return UTF16ToString(b), UTF16ToString(db), accType, nil
+		}
+		if e != ERROR_INSUFFICIENT_BUFFER {
+			return "", "", 0, e
+		}
+		if n <= uint32(len(b)) {
+			return "", "", 0, e
+		}
+	}
+}
+
+// Various types of pre-specified SIDs that can be synthesized and compared at runtime.
+type WELL_KNOWN_SID_TYPE uint32
+
+const (
+	WinNullSid                                    = 0
+	WinWorldSid                                   = 1
+	WinLocalSid                                   = 2
+	WinCreatorOwnerSid                            = 3
+	WinCreatorGroupSid                            = 4
+	WinCreatorOwnerServerSid                      = 5
+	WinCreatorGroupServerSid                      = 6
+	WinNtAuthoritySid                             = 7
+	WinDialupSid                                  = 8
+	WinNetworkSid                                 = 9
+	WinBatchSid                                   = 10
+	WinInteractiveSid                             = 11
+	WinServiceSid                                 = 12
+	WinAnonymousSid                               = 13
+	WinProxySid                                   = 14
+	WinEnterpriseControllersSid                   = 15
+	WinSelfSid                                    = 16
+	WinAuthenticatedUserSid                       = 17
+	WinRestrictedCodeSid                          = 18
+	WinTerminalServerSid                          = 19
+	WinRemoteLogonIdSid                           = 20
+	WinLogonIdsSid                                = 21
+	WinLocalSystemSid                             = 22
+	WinLocalServiceSid                            = 23
+	WinNetworkServiceSid                          = 24
+	WinBuiltinDomainSid                           = 25
+	WinBuiltinAdministratorsSid                   = 26
+	WinBuiltinUsersSid                            = 27
+	WinBuiltinGuestsSid                           = 28
+	WinBuiltinPowerUsersSid                       = 29
+	WinBuiltinAccountOperatorsSid                 = 30
+	WinBuiltinSystemOperatorsSid                  = 31
+	WinBuiltinPrintOperatorsSid                   = 32
+	WinBuiltinBackupOperatorsSid                  = 33
+	WinBuiltinReplicatorSid                       = 34
+	WinBuiltinPreWindows2000CompatibleAccessSid   = 35
+	WinBuiltinRemoteDesktopUsersSid               = 36
+	WinBuiltinNetworkConfigurationOperatorsSid    = 37
+	WinAccountAdministratorSid                    = 38
+	WinAccountGuestSid                            = 39
+	WinAccountKrbtgtSid                           = 40
+	WinAccountDomainAdminsSid                     = 41
+	WinAccountDomainUsersSid                      = 42
+	WinAccountDomainGuestsSid                     = 43
+	WinAccountComputersSid                        = 44
+	WinAccountControllersSid                      = 45
+	WinAccountCertAdminsSid                       = 46
+	WinAccountSchemaAdminsSid                     = 47
+	WinAccountEnterpriseAdminsSid                 = 48
+	WinAccountPolicyAdminsSid                     = 49
+	WinAccountRasAndIasServersSid                 = 50
+	WinNTLMAuthenticationSid                      = 51
+	WinDigestAuthenticationSid                    = 52
+	WinSChannelAuthenticationSid                  = 53
+	WinThisOrganizationSid                        = 54
+	WinOtherOrganizationSid                       = 55
+	WinBuiltinIncomingForestTrustBuildersSid      = 56
+	WinBuiltinPerfMonitoringUsersSid              = 57
+	WinBuiltinPerfLoggingUsersSid                 = 58
+	WinBuiltinAuthorizationAccessSid              = 59
+	WinBuiltinTerminalServerLicenseServersSid     = 60
+	WinBuiltinDCOMUsersSid                        = 61
+	WinBuiltinIUsersSid                           = 62
+	WinIUserSid                                   = 63
+	WinBuiltinCryptoOperatorsSid                  = 64
+	WinUntrustedLabelSid                          = 65
+	WinLowLabelSid                                = 66
+	WinMediumLabelSid                             = 67
+	WinHighLabelSid                               = 68
+	WinSystemLabelSid                             = 69
+	WinWriteRestrictedCodeSid                     = 70
+	WinCreatorOwnerRightsSid                      = 71
+	WinCacheablePrincipalsGroupSid                = 72
+	WinNonCacheablePrincipalsGroupSid             = 73
+	WinEnterpriseReadonlyControllersSid           = 74
+	WinAccountReadonlyControllersSid              = 75
+	WinBuiltinEventLogReadersGroup                = 76
+	WinNewEnterpriseReadonlyControllersSid        = 77
+	WinBuiltinCertSvcDComAccessGroup              = 78
+	WinMediumPlusLabelSid                         = 79
+	WinLocalLogonSid                              = 80
+	WinConsoleLogonSid                            = 81
+	WinThisOrganizationCertificateSid             = 82
+	WinApplicationPackageAuthoritySid             = 83
+	WinBuiltinAnyPackageSid                       = 84
+	WinCapabilityInternetClientSid                = 85
+	WinCapabilityInternetClientServerSid          = 86
+	WinCapabilityPrivateNetworkClientServerSid    = 87
+	WinCapabilityPicturesLibrarySid               = 88
+	WinCapabilityVideosLibrarySid                 = 89
+	WinCapabilityMusicLibrarySid                  = 90
+	WinCapabilityDocumentsLibrarySid              = 91
+	WinCapabilitySharedUserCertificatesSid        = 92
+	WinCapabilityEnterpriseAuthenticationSid      = 93
+	WinCapabilityRemovableStorageSid              = 94
+	WinBuiltinRDSRemoteAccessServersSid           = 95
+	WinBuiltinRDSEndpointServersSid               = 96
+	WinBuiltinRDSManagementServersSid             = 97
+	WinUserModeDriversSid                         = 98
+	WinBuiltinHyperVAdminsSid                     = 99
+	WinAccountCloneableControllersSid             = 100
+	WinBuiltinAccessControlAssistanceOperatorsSid = 101
+	WinBuiltinRemoteManagementUsersSid            = 102
+	WinAuthenticationAuthorityAssertedSid         = 103
+	WinAuthenticationServiceAssertedSid           = 104
+	WinLocalAccountSid                            = 105
+	WinLocalAccountAndAdministratorSid            = 106
+	WinAccountProtectedUsersSid                   = 107
+	WinCapabilityAppointmentsSid                  = 108
+	WinCapabilityContactsSid                      = 109
+	WinAccountDefaultSystemManagedSid             = 110
+	WinBuiltinDefaultSystemManagedGroupSid        = 111
+	WinBuiltinStorageReplicaAdminsSid             = 112
+	WinAccountKeyAdminsSid                        = 113
+	WinAccountEnterpriseKeyAdminsSid              = 114
+	WinAuthenticationKeyTrustSid                  = 115
+	WinAuthenticationKeyPropertyMFASid            = 116
+	WinAuthenticationKeyPropertyAttestationSid    = 117
+	WinAuthenticationFreshKeyAuthSid              = 118
+	WinBuiltinDeviceOwnersSid                     = 119
+)
+
+// Creates a SID for a well-known predefined alias, generally using the constants of the form
+// Win*Sid, for the local machine.
+func CreateWellKnownSid(sidType WELL_KNOWN_SID_TYPE) (*SID, error) {
+	return CreateWellKnownDomainSid(sidType, nil)
+}
+
+// Creates a SID for a well-known predefined alias, generally using the constants of the form
+// Win*Sid, for the domain specified by the domainSid parameter.
+func CreateWellKnownDomainSid(sidType WELL_KNOWN_SID_TYPE, domainSid *SID) (*SID, error) {
+	n := uint32(50)
+	for {
+		b := make([]byte, n)
+		sid := (*SID)(unsafe.Pointer(&b[0]))
+		err := createWellKnownSid(sidType, domainSid, sid, &n)
+		if err == nil {
+			return sid, nil
+		}
+		if err != ERROR_INSUFFICIENT_BUFFER {
+			return nil, err
+		}
+		if n <= uint32(len(b)) {
+			return nil, err
+		}
+	}
+}
+
+const (
+	// do not reorder
+	TOKEN_ASSIGN_PRIMARY = 1 << iota
+	TOKEN_DUPLICATE
+	TOKEN_IMPERSONATE
+	TOKEN_QUERY
+	TOKEN_QUERY_SOURCE
+	TOKEN_ADJUST_PRIVILEGES
+	TOKEN_ADJUST_GROUPS
+	TOKEN_ADJUST_DEFAULT
+	TOKEN_ADJUST_SESSIONID
+
+	TOKEN_ALL_ACCESS = STANDARD_RIGHTS_REQUIRED |
+		TOKEN_ASSIGN_PRIMARY |
+		TOKEN_DUPLICATE |
+		TOKEN_IMPERSONATE |
+		TOKEN_QUERY |
+		TOKEN_QUERY_SOURCE |
+		TOKEN_ADJUST_PRIVILEGES |
+		TOKEN_ADJUST_GROUPS |
+		TOKEN_ADJUST_DEFAULT |
+		TOKEN_ADJUST_SESSIONID
+	TOKEN_READ  = STANDARD_RIGHTS_READ | TOKEN_QUERY
+	TOKEN_WRITE = STANDARD_RIGHTS_WRITE |
+		TOKEN_ADJUST_PRIVILEGES |
+		TOKEN_ADJUST_GROUPS |
+		TOKEN_ADJUST_DEFAULT
+	TOKEN_EXECUTE = STANDARD_RIGHTS_EXECUTE
+)
+
+const (
+	// do not reorder
+	TokenUser = 1 + iota
+	TokenGroups
+	TokenPrivileges
+	TokenOwner
+	TokenPrimaryGroup
+	TokenDefaultDacl
+	TokenSource
+	TokenType
+	TokenImpersonationLevel
+	TokenStatistics
+	TokenRestrictedSids
+	TokenSessionId
+	TokenGroupsAndPrivileges
+	TokenSessionReference
+	TokenSandBoxInert
+	TokenAuditPolicy
+	TokenOrigin
+	TokenElevationType
+	TokenLinkedToken
+	TokenElevation
+	TokenHasRestrictions
+	TokenAccessInformation
+	TokenVirtualizationAllowed
+	TokenVirtualizationEnabled
+	TokenIntegrityLevel
+	TokenUIAccess
+	TokenMandatoryPolicy
+	TokenLogonSid
+	MaxTokenInfoClass
+)
+
+// Group attributes inside of Tokengroups.Groups[i].Attributes
+const (
+	SE_GROUP_MANDATORY          = 0x00000001
+	SE_GROUP_ENABLED_BY_DEFAULT = 0x00000002
+	SE_GROUP_ENABLED            = 0x00000004
+	SE_GROUP_OWNER              = 0x00000008
+	SE_GROUP_USE_FOR_DENY_ONLY  = 0x00000010
+	SE_GROUP_INTEGRITY          = 0x00000020
+	SE_GROUP_INTEGRITY_ENABLED  = 0x00000040
+	SE_GROUP_LOGON_ID           = 0xC0000000
+	SE_GROUP_RESOURCE           = 0x20000000
+	SE_GROUP_VALID_ATTRIBUTES   = SE_GROUP_MANDATORY | SE_GROUP_ENABLED_BY_DEFAULT | SE_GROUP_ENABLED | SE_GROUP_OWNER | SE_GROUP_USE_FOR_DENY_ONLY | SE_GROUP_LOGON_ID | SE_GROUP_RESOURCE | SE_GROUP_INTEGRITY | SE_GROUP_INTEGRITY_ENABLED
+)
+
+// Privilege attributes
+const (
+	SE_PRIVILEGE_ENABLED_BY_DEFAULT = 0x00000001
+	SE_PRIVILEGE_ENABLED            = 0x00000002
+	SE_PRIVILEGE_REMOVED            = 0x00000004
+	SE_PRIVILEGE_USED_FOR_ACCESS    = 0x80000000
+	SE_PRIVILEGE_VALID_ATTRIBUTES   = SE_PRIVILEGE_ENABLED_BY_DEFAULT | SE_PRIVILEGE_ENABLED | SE_PRIVILEGE_REMOVED | SE_PRIVILEGE_USED_FOR_ACCESS
+)
+
+// Token types
+const (
+	TokenPrimary       = 1
+	TokenImpersonation = 2
+)
+
+// Impersonation levels
+const (
+	SecurityAnonymous      = 0
+	SecurityIdentification = 1
+	SecurityImpersonation  = 2
+	SecurityDelegation     = 3
+)
+
+type LUID struct {
+	LowPart  uint32
+	HighPart int32
+}
+
+type LUIDAndAttributes struct {
+	Luid       LUID
+	Attributes uint32
+}
+
+type SIDAndAttributes struct {
+	Sid        *SID
+	Attributes uint32
+}
+
+type Tokenuser struct {
+	User SIDAndAttributes
+}
+
+type Tokenprimarygroup struct {
+	PrimaryGroup *SID
+}
+
+type Tokengroups struct {
+	GroupCount uint32
+	Groups     [1]SIDAndAttributes // Use AllGroups() for iterating.
+}
+
+// AllGroups returns a slice that can be used to iterate over the groups in g.
+func (g *Tokengroups) AllGroups() []SIDAndAttributes {
+	return (*[(1 << 28) - 1]SIDAndAttributes)(unsafe.Pointer(&g.Groups[0]))[:g.GroupCount:g.GroupCount]
+}
+
+type Tokenprivileges struct {
+	PrivilegeCount uint32
+	Privileges     [1]LUIDAndAttributes // Use AllPrivileges() for iterating.
+}
+
+// AllPrivileges returns a slice that can be used to iterate over the privileges in p.
+func (p *Tokenprivileges) AllPrivileges() []LUIDAndAttributes {
+	return (*[(1 << 27) - 1]LUIDAndAttributes)(unsafe.Pointer(&p.Privileges[0]))[:p.PrivilegeCount:p.PrivilegeCount]
+}
+
+type Tokenmandatorylabel struct {
+	Label SIDAndAttributes
+}
+
+func (tml *Tokenmandatorylabel) Size() uint32 {
+	return uint32(unsafe.Sizeof(Tokenmandatorylabel{})) + GetLengthSid(tml.Label.Sid)
+}
+
+// Authorization Functions
+//sys	checkTokenMembership(tokenHandle Token, sidToCheck *SID, isMember *int32) (err error) = advapi32.CheckTokenMembership
+//sys	OpenProcessToken(process Handle, access uint32, token *Token) (err error) = advapi32.OpenProcessToken
+//sys	OpenThreadToken(thread Handle, access uint32, openAsSelf bool, token *Token) (err error) = advapi32.OpenThreadToken
+//sys	ImpersonateSelf(impersonationlevel uint32) (err error) = advapi32.ImpersonateSelf
+//sys	RevertToSelf() (err error) = advapi32.RevertToSelf
+//sys	SetThreadToken(thread *Handle, token Token) (err error) = advapi32.SetThreadToken
+//sys	LookupPrivilegeValue(systemname *uint16, name *uint16, luid *LUID) (err error) = advapi32.LookupPrivilegeValueW
+//sys	AdjustTokenPrivileges(token Token, disableAllPrivileges bool, newstate *Tokenprivileges, buflen uint32, prevstate *Tokenprivileges, returnlen *uint32) (err error) = advapi32.AdjustTokenPrivileges
+//sys	AdjustTokenGroups(token Token, resetToDefault bool, newstate *Tokengroups, buflen uint32, prevstate *Tokengroups, returnlen *uint32) (err error) = advapi32.AdjustTokenGroups
+//sys	GetTokenInformation(token Token, infoClass uint32, info *byte, infoLen uint32, returnedLen *uint32) (err error) = advapi32.GetTokenInformation
+//sys	SetTokenInformation(token Token, infoClass uint32, info *byte, infoLen uint32) (err error) = advapi32.SetTokenInformation
+//sys	DuplicateTokenEx(existingToken Token, desiredAccess uint32, tokenAttributes *SecurityAttributes, impersonationLevel uint32, tokenType uint32, newToken *Token) (err error) = advapi32.DuplicateTokenEx
+//sys	GetUserProfileDirectory(t Token, dir *uint16, dirLen *uint32) (err error) = userenv.GetUserProfileDirectoryW
+//sys	getSystemDirectory(dir *uint16, dirLen uint32) (len uint32, err error) = kernel32.GetSystemDirectoryW
+//sys	getWindowsDirectory(dir *uint16, dirLen uint32) (len uint32, err error) = kernel32.GetWindowsDirectoryW
+//sys	getSystemWindowsDirectory(dir *uint16, dirLen uint32) (len uint32, err error) = kernel32.GetSystemWindowsDirectoryW
+
+// An access token contains the security information for a logon session.
+// The system creates an access token when a user logs on, and every
+// process executed on behalf of the user has a copy of the token.
+// The token identifies the user, the user's groups, and the user's
+// privileges. The system uses the token to control access to securable
+// objects and to control the ability of the user to perform various
+// system-related operations on the local computer.
+type Token Handle
+
+// OpenCurrentProcessToken opens an access token associated with current
+// process with TOKEN_QUERY access. It is a real token that needs to be closed.
+//
+// Deprecated: Explicitly call OpenProcessToken(CurrentProcess(), ...)
+// with the desired access instead, or use GetCurrentProcessToken for a
+// TOKEN_QUERY token.
+func OpenCurrentProcessToken() (Token, error) {
+	var token Token
+	err := OpenProcessToken(CurrentProcess(), TOKEN_QUERY, &token)
+	return token, err
+}
+
+// GetCurrentProcessToken returns the access token associated with
+// the current process. It is a pseudo token that does not need
+// to be closed.
+func GetCurrentProcessToken() Token {
+	return Token(^uintptr(4 - 1))
+}
+
+// GetCurrentThreadToken return the access token associated with
+// the current thread. It is a pseudo token that does not need
+// to be closed.
+func GetCurrentThreadToken() Token {
+	return Token(^uintptr(5 - 1))
+}
+
+// GetCurrentThreadEffectiveToken returns the effective access token
+// associated with the current thread. It is a pseudo token that does
+// not need to be closed.
+func GetCurrentThreadEffectiveToken() Token {
+	return Token(^uintptr(6 - 1))
+}
+
+// Close releases access to access token.
+func (t Token) Close() error {
+	return CloseHandle(Handle(t))
+}
+
+// getInfo retrieves a specified type of information about an access token.
+func (t Token) getInfo(class uint32, initSize int) (unsafe.Pointer, error) {
+	n := uint32(initSize)
+	for {
+		b := make([]byte, n)
+		e := GetTokenInformation(t, class, &b[0], uint32(len(b)), &n)
+		if e == nil {
+			return unsafe.Pointer(&b[0]), nil
+		}
+		if e != ERROR_INSUFFICIENT_BUFFER {
+			return nil, e
+		}
+		if n <= uint32(len(b)) {
+			return nil, e
+		}
+	}
+}
+
+// GetTokenUser retrieves access token t user account information.
+func (t Token) GetTokenUser() (*Tokenuser, error) {
+	i, e := t.getInfo(TokenUser, 50)
+	if e != nil {
+		return nil, e
+	}
+	return (*Tokenuser)(i), nil
+}
+
+// GetTokenGroups retrieves group accounts associated with access token t.
+func (t Token) GetTokenGroups() (*Tokengroups, error) {
+	i, e := t.getInfo(TokenGroups, 50)
+	if e != nil {
+		return nil, e
+	}
+	return (*Tokengroups)(i), nil
+}
+
+// GetTokenPrimaryGroup retrieves access token t primary group information.
+// A pointer to a SID structure representing a group that will become
+// the primary group of any objects created by a process using this access token.
+func (t Token) GetTokenPrimaryGroup() (*Tokenprimarygroup, error) {
+	i, e := t.getInfo(TokenPrimaryGroup, 50)
+	if e != nil {
+		return nil, e
+	}
+	return (*Tokenprimarygroup)(i), nil
+}
+
+// GetUserProfileDirectory retrieves path to the
+// root directory of the access token t user's profile.
+func (t Token) GetUserProfileDirectory() (string, error) {
+	n := uint32(100)
+	for {
+		b := make([]uint16, n)
+		e := GetUserProfileDirectory(t, &b[0], &n)
+		if e == nil {
+			return UTF16ToString(b), nil
+		}
+		if e != ERROR_INSUFFICIENT_BUFFER {
+			return "", e
+		}
+		if n <= uint32(len(b)) {
+			return "", e
+		}
+	}
+}
+
+// IsElevated returns whether the current token is elevated from a UAC perspective.
+func (token Token) IsElevated() bool {
+	var isElevated uint32
+	var outLen uint32
+	err := GetTokenInformation(token, TokenElevation, (*byte)(unsafe.Pointer(&isElevated)), uint32(unsafe.Sizeof(isElevated)), &outLen)
+	if err != nil {
+		return false
+	}
+	return outLen == uint32(unsafe.Sizeof(isElevated)) && isElevated != 0
+}
+
+// GetLinkedToken returns the linked token, which may be an elevated UAC token.
+func (token Token) GetLinkedToken() (Token, error) {
+	var linkedToken Token
+	var outLen uint32
+	err := GetTokenInformation(token, TokenLinkedToken, (*byte)(unsafe.Pointer(&linkedToken)), uint32(unsafe.Sizeof(linkedToken)), &outLen)
+	if err != nil {
+		return Token(0), err
+	}
+	return linkedToken, nil
+}
+
+// GetSystemDirectory retrieves the path to current location of the system
+// directory, which is typically, though not always, `C:\Windows\System32`.
+func GetSystemDirectory() (string, error) {
+	n := uint32(MAX_PATH)
+	for {
+		b := make([]uint16, n)
+		l, e := getSystemDirectory(&b[0], n)
+		if e != nil {
+			return "", e
+		}
+		if l <= n {
+			return UTF16ToString(b[:l]), nil
+		}
+		n = l
+	}
+}
+
+// GetWindowsDirectory retrieves the path to current location of the Windows
+// directory, which is typically, though not always, `C:\Windows`. This may
+// be a private user directory in the case that the application is running
+// under a terminal server.
+func GetWindowsDirectory() (string, error) {
+	n := uint32(MAX_PATH)
+	for {
+		b := make([]uint16, n)
+		l, e := getWindowsDirectory(&b[0], n)
+		if e != nil {
+			return "", e
+		}
+		if l <= n {
+			return UTF16ToString(b[:l]), nil
+		}
+		n = l
+	}
+}
+
+// GetSystemWindowsDirectory retrieves the path to current location of the
+// Windows directory, which is typically, though not always, `C:\Windows`.
+func GetSystemWindowsDirectory() (string, error) {
+	n := uint32(MAX_PATH)
+	for {
+		b := make([]uint16, n)
+		l, e := getSystemWindowsDirectory(&b[0], n)
+		if e != nil {
+			return "", e
+		}
+		if l <= n {
+			return UTF16ToString(b[:l]), nil
+		}
+		n = l
+	}
+}
+
+// IsMember reports whether the access token t is a member of the provided SID.
+func (t Token) IsMember(sid *SID) (bool, error) {
+	var b int32
+	if e := checkTokenMembership(t, sid, &b); e != nil {
+		return false, e
+	}
+	return b != 0, nil
+}
+
+const (
+	WTS_CONSOLE_CONNECT        = 0x1
+	WTS_CONSOLE_DISCONNECT     = 0x2
+	WTS_REMOTE_CONNECT         = 0x3
+	WTS_REMOTE_DISCONNECT      = 0x4
+	WTS_SESSION_LOGON          = 0x5
+	WTS_SESSION_LOGOFF         = 0x6
+	WTS_SESSION_LOCK           = 0x7
+	WTS_SESSION_UNLOCK         = 0x8
+	WTS_SESSION_REMOTE_CONTROL = 0x9
+	WTS_SESSION_CREATE         = 0xa
+	WTS_SESSION_TERMINATE      = 0xb
+)
+
+const (
+	WTSActive       = 0
+	WTSConnected    = 1
+	WTSConnectQuery = 2
+	WTSShadow       = 3
+	WTSDisconnected = 4
+	WTSIdle         = 5
+	WTSListen       = 6
+	WTSReset        = 7
+	WTSDown         = 8
+	WTSInit         = 9
+)
+
+type WTSSESSION_NOTIFICATION struct {
+	Size      uint32
+	SessionID uint32
+}
+
+type WTS_SESSION_INFO struct {
+	SessionID         uint32
+	WindowStationName *uint16
+	State             uint32
+}
+
+//sys WTSQueryUserToken(session uint32, token *Token) (err error) = wtsapi32.WTSQueryUserToken
+//sys WTSEnumerateSessions(handle Handle, reserved uint32, version uint32, sessions **WTS_SESSION_INFO, count *uint32) (err error) = wtsapi32.WTSEnumerateSessionsW
+//sys WTSFreeMemory(ptr uintptr) = wtsapi32.WTSFreeMemory
+
+type ACL struct {
+	aclRevision byte
+	sbz1        byte
+	aclSize     uint16
+	aceCount    uint16
+	sbz2        uint16
+}
+
+type SECURITY_DESCRIPTOR struct {
+	revision byte
+	sbz1     byte
+	control  SECURITY_DESCRIPTOR_CONTROL
+	owner    *SID
+	group    *SID
+	sacl     *ACL
+	dacl     *ACL
+}
+
+type SecurityAttributes struct {
+	Length             uint32
+	SecurityDescriptor *SECURITY_DESCRIPTOR
+	InheritHandle      uint32
+}
+
+type SE_OBJECT_TYPE uint32
+
+// Constants for type SE_OBJECT_TYPE
+const (
+	SE_UNKNOWN_OBJECT_TYPE     = 0
+	SE_FILE_OBJECT             = 1
+	SE_SERVICE                 = 2
+	SE_PRINTER                 = 3
+	SE_REGISTRY_KEY            = 4
+	SE_LMSHARE                 = 5
+	SE_KERNEL_OBJECT           = 6
+	SE_WINDOW_OBJECT           = 7
+	SE_DS_OBJECT               = 8
+	SE_DS_OBJECT_ALL           = 9
+	SE_PROVIDER_DEFINED_OBJECT = 10
+	SE_WMIGUID_OBJECT          = 11
+	SE_REGISTRY_WOW64_32KEY    = 12
+	SE_REGISTRY_WOW64_64KEY    = 13
+)
+
+type SECURITY_INFORMATION uint32
+
+// Constants for type SECURITY_INFORMATION
+const (
+	OWNER_SECURITY_INFORMATION            = 0x00000001
+	GROUP_SECURITY_INFORMATION            = 0x00000002
+	DACL_SECURITY_INFORMATION             = 0x00000004
+	SACL_SECURITY_INFORMATION             = 0x00000008
+	LABEL_SECURITY_INFORMATION            = 0x00000010
+	ATTRIBUTE_SECURITY_INFORMATION        = 0x00000020
+	SCOPE_SECURITY_INFORMATION            = 0x00000040
+	BACKUP_SECURITY_INFORMATION           = 0x00010000
+	PROTECTED_DACL_SECURITY_INFORMATION   = 0x80000000
+	PROTECTED_SACL_SECURITY_INFORMATION   = 0x40000000
+	UNPROTECTED_DACL_SECURITY_INFORMATION = 0x20000000
+	UNPROTECTED_SACL_SECURITY_INFORMATION = 0x10000000
+)
+
+type SECURITY_DESCRIPTOR_CONTROL uint16
+
+// Constants for type SECURITY_DESCRIPTOR_CONTROL
+const (
+	SE_OWNER_DEFAULTED       = 0x0001
+	SE_GROUP_DEFAULTED       = 0x0002
+	SE_DACL_PRESENT          = 0x0004
+	SE_DACL_DEFAULTED        = 0x0008
+	SE_SACL_PRESENT          = 0x0010
+	SE_SACL_DEFAULTED        = 0x0020
+	SE_DACL_AUTO_INHERIT_REQ = 0x0100
+	SE_SACL_AUTO_INHERIT_REQ = 0x0200
+	SE_DACL_AUTO_INHERITED   = 0x0400
+	SE_SACL_AUTO_INHERITED   = 0x0800
+	SE_DACL_PROTECTED        = 0x1000
+	SE_SACL_PROTECTED        = 0x2000
+	SE_RM_CONTROL_VALID      = 0x4000
+	SE_SELF_RELATIVE         = 0x8000
+)
+
+type ACCESS_MASK uint32
+
+// Constants for type ACCESS_MASK
+const (
+	DELETE                   = 0x00010000
+	READ_CONTROL             = 0x00020000
+	WRITE_DAC                = 0x00040000
+	WRITE_OWNER              = 0x00080000
+	SYNCHRONIZE              = 0x00100000
+	STANDARD_RIGHTS_REQUIRED = 0x000F0000
+	STANDARD_RIGHTS_READ     = READ_CONTROL
+	STANDARD_RIGHTS_WRITE    = READ_CONTROL
+	STANDARD_RIGHTS_EXECUTE  = READ_CONTROL
+	STANDARD_RIGHTS_ALL      = 0x001F0000
+	SPECIFIC_RIGHTS_ALL      = 0x0000FFFF
+	ACCESS_SYSTEM_SECURITY   = 0x01000000
+	MAXIMUM_ALLOWED          = 0x02000000
+	GENERIC_READ             = 0x80000000
+	GENERIC_WRITE            = 0x40000000
+	GENERIC_EXECUTE          = 0x20000000
+	GENERIC_ALL              = 0x10000000
+)
+
+type ACCESS_MODE uint32
+
+// Constants for type ACCESS_MODE
+const (
+	NOT_USED_ACCESS   = 0
+	GRANT_ACCESS      = 1
+	SET_ACCESS        = 2
+	DENY_ACCESS       = 3
+	REVOKE_ACCESS     = 4
+	SET_AUDIT_SUCCESS = 5
+	SET_AUDIT_FAILURE = 6
+)
+
+// Constants for AceFlags and Inheritance fields
+const (
+	NO_INHERITANCE                     = 0x0
+	SUB_OBJECTS_ONLY_INHERIT           = 0x1
+	SUB_CONTAINERS_ONLY_INHERIT        = 0x2
+	SUB_CONTAINERS_AND_OBJECTS_INHERIT = 0x3
+	INHERIT_NO_PROPAGATE               = 0x4
+	INHERIT_ONLY                       = 0x8
+	INHERITED_ACCESS_ENTRY             = 0x10
+	INHERITED_PARENT                   = 0x10000000
+	INHERITED_GRANDPARENT              = 0x20000000
+	OBJECT_INHERIT_ACE                 = 0x1
+	CONTAINER_INHERIT_ACE              = 0x2
+	NO_PROPAGATE_INHERIT_ACE           = 0x4
+	INHERIT_ONLY_ACE                   = 0x8
+	INHERITED_ACE                      = 0x10
+	VALID_INHERIT_FLAGS                = 0x1F
+)
+
+type MULTIPLE_TRUSTEE_OPERATION uint32
+
+// Constants for MULTIPLE_TRUSTEE_OPERATION
+const (
+	NO_MULTIPLE_TRUSTEE    = 0
+	TRUSTEE_IS_IMPERSONATE = 1
+)
+
+type TRUSTEE_FORM uint32
+
+// Constants for TRUSTEE_FORM
+const (
+	TRUSTEE_IS_SID              = 0
+	TRUSTEE_IS_NAME             = 1
+	TRUSTEE_BAD_FORM            = 2
+	TRUSTEE_IS_OBJECTS_AND_SID  = 3
+	TRUSTEE_IS_OBJECTS_AND_NAME = 4
+)
+
+type TRUSTEE_TYPE uint32
+
+// Constants for TRUSTEE_TYPE
+const (
+	TRUSTEE_IS_UNKNOWN          = 0
+	TRUSTEE_IS_USER             = 1
+	TRUSTEE_IS_GROUP            = 2
+	TRUSTEE_IS_DOMAIN           = 3
+	TRUSTEE_IS_ALIAS            = 4
+	TRUSTEE_IS_WELL_KNOWN_GROUP = 5
+	TRUSTEE_IS_DELETED          = 6
+	TRUSTEE_IS_INVALID          = 7
+	TRUSTEE_IS_COMPUTER         = 8
+)
+
+// Constants for ObjectsPresent field
+const (
+	ACE_OBJECT_TYPE_PRESENT           = 0x1
+	ACE_INHERITED_OBJECT_TYPE_PRESENT = 0x2
+)
+
+type EXPLICIT_ACCESS struct {
+	AccessPermissions ACCESS_MASK
+	AccessMode        ACCESS_MODE
+	Inheritance       uint32
+	Trustee           TRUSTEE
+}
+
+// This type is the union inside of TRUSTEE and must be created using one of the TrusteeValueFrom* functions.
+type TrusteeValue uintptr
+
+func TrusteeValueFromString(str string) TrusteeValue {
+	return TrusteeValue(unsafe.Pointer(StringToUTF16Ptr(str)))
+}
+func TrusteeValueFromSID(sid *SID) TrusteeValue {
+	return TrusteeValue(unsafe.Pointer(sid))
+}
+func TrusteeValueFromObjectsAndSid(objectsAndSid *OBJECTS_AND_SID) TrusteeValue {
+	return TrusteeValue(unsafe.Pointer(objectsAndSid))
+}
+func TrusteeValueFromObjectsAndName(objectsAndName *OBJECTS_AND_NAME) TrusteeValue {
+	return TrusteeValue(unsafe.Pointer(objectsAndName))
+}
+
+type TRUSTEE struct {
+	MultipleTrustee          *TRUSTEE
+	MultipleTrusteeOperation MULTIPLE_TRUSTEE_OPERATION
+	TrusteeForm              TRUSTEE_FORM
+	TrusteeType              TRUSTEE_TYPE
+	TrusteeValue             TrusteeValue
+}
+
+type OBJECTS_AND_SID struct {
+	ObjectsPresent          uint32
+	ObjectTypeGuid          GUID
+	InheritedObjectTypeGuid GUID
+	Sid                     *SID
+}
+
+type OBJECTS_AND_NAME struct {
+	ObjectsPresent          uint32
+	ObjectType              SE_OBJECT_TYPE
+	ObjectTypeName          *uint16
+	InheritedObjectTypeName *uint16
+	Name                    *uint16
+}
+
+//sys	getSecurityInfo(handle Handle, objectType SE_OBJECT_TYPE, securityInformation SECURITY_INFORMATION, owner **SID, group **SID, dacl **ACL, sacl **ACL, sd **SECURITY_DESCRIPTOR) (ret error) = advapi32.GetSecurityInfo
+//sys	SetSecurityInfo(handle Handle, objectType SE_OBJECT_TYPE, securityInformation SECURITY_INFORMATION, owner *SID, group *SID, dacl *ACL, sacl *ACL) = advapi32.SetSecurityInfo
+//sys	getNamedSecurityInfo(objectName string, objectType SE_OBJECT_TYPE, securityInformation SECURITY_INFORMATION, owner **SID, group **SID, dacl **ACL, sacl **ACL, sd **SECURITY_DESCRIPTOR) (ret error) = advapi32.GetNamedSecurityInfoW
+//sys	SetNamedSecurityInfo(objectName string, objectType SE_OBJECT_TYPE, securityInformation SECURITY_INFORMATION, owner *SID, group *SID, dacl *ACL, sacl *ACL) (ret error) = advapi32.SetNamedSecurityInfoW
+
+//sys	buildSecurityDescriptor(owner *TRUSTEE, group *TRUSTEE, countAccessEntries uint32, accessEntries *EXPLICIT_ACCESS, countAuditEntries uint32, auditEntries *EXPLICIT_ACCESS, oldSecurityDescriptor *SECURITY_DESCRIPTOR, sizeNewSecurityDescriptor *uint32, newSecurityDescriptor **SECURITY_DESCRIPTOR) (ret error) = advapi32.BuildSecurityDescriptorW
+//sys	initializeSecurityDescriptor(absoluteSD *SECURITY_DESCRIPTOR, revision uint32) (err error) = advapi32.InitializeSecurityDescriptor
+
+//sys	getSecurityDescriptorControl(sd *SECURITY_DESCRIPTOR, control *SECURITY_DESCRIPTOR_CONTROL, revision *uint32) (err error) = advapi32.GetSecurityDescriptorControl
+//sys	getSecurityDescriptorDacl(sd *SECURITY_DESCRIPTOR, daclPresent *bool, dacl **ACL, daclDefaulted *bool) (err error) = advapi32.GetSecurityDescriptorDacl
+//sys	getSecurityDescriptorSacl(sd *SECURITY_DESCRIPTOR, saclPresent *bool, sacl **ACL, saclDefaulted *bool) (err error) = advapi32.GetSecurityDescriptorSacl
+//sys	getSecurityDescriptorOwner(sd *SECURITY_DESCRIPTOR, owner **SID, ownerDefaulted *bool) (err error) = advapi32.GetSecurityDescriptorOwner
+//sys	getSecurityDescriptorGroup(sd *SECURITY_DESCRIPTOR, group **SID, groupDefaulted *bool) (err error) = advapi32.GetSecurityDescriptorGroup
+//sys	getSecurityDescriptorLength(sd *SECURITY_DESCRIPTOR) (len uint32) = advapi32.GetSecurityDescriptorLength
+//sys	getSecurityDescriptorRMControl(sd *SECURITY_DESCRIPTOR, rmControl *uint8) (ret error) [failretval!=0] = advapi32.GetSecurityDescriptorRMControl
+//sys	isValidSecurityDescriptor(sd *SECURITY_DESCRIPTOR) (isValid bool) = advapi32.IsValidSecurityDescriptor
+
+//sys	setSecurityDescriptorControl(sd *SECURITY_DESCRIPTOR, controlBitsOfInterest SECURITY_DESCRIPTOR_CONTROL, controlBitsToSet SECURITY_DESCRIPTOR_CONTROL) (err error) = advapi32.SetSecurityDescriptorControl
+//sys	setSecurityDescriptorDacl(sd *SECURITY_DESCRIPTOR, daclPresent bool, dacl *ACL, daclDefaulted bool) (err error) = advapi32.SetSecurityDescriptorDacl
+//sys	setSecurityDescriptorSacl(sd *SECURITY_DESCRIPTOR, saclPresent bool, sacl *ACL, saclDefaulted bool) (err error) = advapi32.SetSecurityDescriptorSacl
+//sys	setSecurityDescriptorOwner(sd *SECURITY_DESCRIPTOR, owner *SID, ownerDefaulted bool) (err error) = advapi32.SetSecurityDescriptorOwner
+//sys	setSecurityDescriptorGroup(sd *SECURITY_DESCRIPTOR, group *SID, groupDefaulted bool) (err error) = advapi32.SetSecurityDescriptorGroup
+//sys	setSecurityDescriptorRMControl(sd *SECURITY_DESCRIPTOR, rmControl *uint8) = advapi32.SetSecurityDescriptorRMControl
+
+//sys	convertStringSecurityDescriptorToSecurityDescriptor(str string, revision uint32, sd **SECURITY_DESCRIPTOR, size *uint32) (err error) = advapi32.ConvertStringSecurityDescriptorToSecurityDescriptorW
+//sys	convertSecurityDescriptorToStringSecurityDescriptor(sd *SECURITY_DESCRIPTOR, revision uint32, securityInformation SECURITY_INFORMATION, str **uint16, strLen *uint32) (err error) = advapi32.ConvertSecurityDescriptorToStringSecurityDescriptorW
+
+//sys	makeAbsoluteSD(selfRelativeSD *SECURITY_DESCRIPTOR, absoluteSD *SECURITY_DESCRIPTOR, absoluteSDSize *uint32, dacl *ACL, daclSize *uint32, sacl *ACL, saclSize *uint32, owner *SID, ownerSize *uint32, group *SID, groupSize *uint32) (err error) = advapi32.MakeAbsoluteSD
+//sys	makeSelfRelativeSD(absoluteSD *SECURITY_DESCRIPTOR, selfRelativeSD *SECURITY_DESCRIPTOR, selfRelativeSDSize *uint32) (err error) = advapi32.MakeSelfRelativeSD
+
+//sys	setEntriesInAcl(countExplicitEntries uint32, explicitEntries *EXPLICIT_ACCESS, oldACL *ACL, newACL **ACL) (ret error) = advapi32.SetEntriesInAclW
+
+// Control returns the security descriptor control bits.
+func (sd *SECURITY_DESCRIPTOR) Control() (control SECURITY_DESCRIPTOR_CONTROL, revision uint32, err error) {
+	err = getSecurityDescriptorControl(sd, &control, &revision)
+	return
+}
+
+// SetControl sets the security descriptor control bits.
+func (sd *SECURITY_DESCRIPTOR) SetControl(controlBitsOfInterest SECURITY_DESCRIPTOR_CONTROL, controlBitsToSet SECURITY_DESCRIPTOR_CONTROL) error {
+	return setSecurityDescriptorControl(sd, controlBitsOfInterest, controlBitsToSet)
+}
+
+// RMControl returns the security descriptor resource manager control bits.
+func (sd *SECURITY_DESCRIPTOR) RMControl() (control uint8, err error) {
+	err = getSecurityDescriptorRMControl(sd, &control)
+	return
+}
+
+// SetRMControl sets the security descriptor resource manager control bits.
+func (sd *SECURITY_DESCRIPTOR) SetRMControl(rmControl uint8) {
+	setSecurityDescriptorRMControl(sd, &rmControl)
+}
+
+// DACL returns the security descriptor DACL and whether it was defaulted. The dacl return value may be nil
+// if a DACL exists but is an "empty DACL", meaning fully permissive. If the DACL does not exist, err returns
+// ERROR_OBJECT_NOT_FOUND.
+func (sd *SECURITY_DESCRIPTOR) DACL() (dacl *ACL, defaulted bool, err error) {
+	var present bool
+	err = getSecurityDescriptorDacl(sd, &present, &dacl, &defaulted)
+	if !present {
+		err = ERROR_OBJECT_NOT_FOUND
+	}
+	return
+}
+
+// SetDACL sets the absolute security descriptor DACL.
+func (absoluteSD *SECURITY_DESCRIPTOR) SetDACL(dacl *ACL, present, defaulted bool) error {
+	return setSecurityDescriptorDacl(absoluteSD, present, dacl, defaulted)
+}
+
+// SACL returns the security descriptor SACL and whether it was defaulted. The sacl return value may be nil
+// if a SACL exists but is an "empty SACL", meaning fully permissive. If the SACL does not exist, err returns
+// ERROR_OBJECT_NOT_FOUND.
+func (sd *SECURITY_DESCRIPTOR) SACL() (sacl *ACL, defaulted bool, err error) {
+	var present bool
+	err = getSecurityDescriptorSacl(sd, &present, &sacl, &defaulted)
+	if !present {
+		err = ERROR_OBJECT_NOT_FOUND
+	}
+	return
+}
+
+// SetSACL sets the absolute security descriptor SACL.
+func (absoluteSD *SECURITY_DESCRIPTOR) SetSACL(sacl *ACL, present, defaulted bool) error {
+	return setSecurityDescriptorSacl(absoluteSD, present, sacl, defaulted)
+}
+
+// Owner returns the security descriptor owner and whether it was defaulted.
+func (sd *SECURITY_DESCRIPTOR) Owner() (owner *SID, defaulted bool, err error) {
+	err = getSecurityDescriptorOwner(sd, &owner, &defaulted)
+	return
+}
+
+// SetOwner sets the absolute security descriptor owner.
+func (absoluteSD *SECURITY_DESCRIPTOR) SetOwner(owner *SID, defaulted bool) error {
+	return setSecurityDescriptorOwner(absoluteSD, owner, defaulted)
+}
+
+// Group returns the security descriptor group and whether it was defaulted.
+func (sd *SECURITY_DESCRIPTOR) Group() (group *SID, defaulted bool, err error) {
+	err = getSecurityDescriptorGroup(sd, &group, &defaulted)
+	return
+}
+
+// SetGroup sets the absolute security descriptor owner.
+func (absoluteSD *SECURITY_DESCRIPTOR) SetGroup(group *SID, defaulted bool) error {
+	return setSecurityDescriptorGroup(absoluteSD, group, defaulted)
+}
+
+// Length returns the length of the security descriptor.
+func (sd *SECURITY_DESCRIPTOR) Length() uint32 {
+	return getSecurityDescriptorLength(sd)
+}
+
+// IsValid returns whether the security descriptor is valid.
+func (sd *SECURITY_DESCRIPTOR) IsValid() bool {
+	return isValidSecurityDescriptor(sd)
+}
+
+// String returns the SDDL form of the security descriptor, with a function signature that can be
+// used with %v formatting directives.
+func (sd *SECURITY_DESCRIPTOR) String() string {
+	var sddl *uint16
+	err := convertSecurityDescriptorToStringSecurityDescriptor(sd, 1, 0xff, &sddl, nil)
+	if err != nil {
+		return ""
+	}
+	defer LocalFree(Handle(unsafe.Pointer(sddl)))
+	return UTF16ToString((*[(1 << 30) - 1]uint16)(unsafe.Pointer(sddl))[:])
+}
+
+// ToAbsolute converts a self-relative security descriptor into an absolute one.
+func (selfRelativeSD *SECURITY_DESCRIPTOR) ToAbsolute() (absoluteSD *SECURITY_DESCRIPTOR, err error) {
+	control, _, err := selfRelativeSD.Control()
+	if err != nil {
+		return
+	}
+	if control&SE_SELF_RELATIVE == 0 {
+		err = ERROR_INVALID_PARAMETER
+		return
+	}
+	var absoluteSDSize, daclSize, saclSize, ownerSize, groupSize uint32
+	err = makeAbsoluteSD(selfRelativeSD, nil, &absoluteSDSize,
+		nil, &daclSize, nil, &saclSize, nil, &ownerSize, nil, &groupSize)
+	switch err {
+	case ERROR_INSUFFICIENT_BUFFER:
+	case nil:
+		// makeAbsoluteSD is expected to fail, but it succeeds.
+		return nil, ERROR_INTERNAL_ERROR
+	default:
+		return nil, err
+	}
+	if absoluteSDSize > 0 {
+		absoluteSD = (*SECURITY_DESCRIPTOR)(unsafe.Pointer(&make([]byte, absoluteSDSize)[0]))
+	}
+	var (
+		dacl  *ACL
+		sacl  *ACL
+		owner *SID
+		group *SID
+	)
+	if daclSize > 0 {
+		dacl = (*ACL)(unsafe.Pointer(&make([]byte, daclSize)[0]))
+	}
+	if saclSize > 0 {
+		sacl = (*ACL)(unsafe.Pointer(&make([]byte, saclSize)[0]))
+	}
+	if ownerSize > 0 {
+		owner = (*SID)(unsafe.Pointer(&make([]byte, ownerSize)[0]))
+	}
+	if groupSize > 0 {
+		group = (*SID)(unsafe.Pointer(&make([]byte, groupSize)[0]))
+	}
+	err = makeAbsoluteSD(selfRelativeSD, absoluteSD, &absoluteSDSize,
+		dacl, &daclSize, sacl, &saclSize, owner, &ownerSize, group, &groupSize)
+	return
+}
+
+// ToSelfRelative converts an absolute security descriptor into a self-relative one.
+func (absoluteSD *SECURITY_DESCRIPTOR) ToSelfRelative() (selfRelativeSD *SECURITY_DESCRIPTOR, err error) {
+	control, _, err := absoluteSD.Control()
+	if err != nil {
+		return
+	}
+	if control&SE_SELF_RELATIVE != 0 {
+		err = ERROR_INVALID_PARAMETER
+		return
+	}
+	var selfRelativeSDSize uint32
+	err = makeSelfRelativeSD(absoluteSD, nil, &selfRelativeSDSize)
+	switch err {
+	case ERROR_INSUFFICIENT_BUFFER:
+	case nil:
+		// makeSelfRelativeSD is expected to fail, but it succeeds.
+		return nil, ERROR_INTERNAL_ERROR
+	default:
+		return nil, err
+	}
+	if selfRelativeSDSize > 0 {
+		selfRelativeSD = (*SECURITY_DESCRIPTOR)(unsafe.Pointer(&make([]byte, selfRelativeSDSize)[0]))
+	}
+	err = makeSelfRelativeSD(absoluteSD, selfRelativeSD, &selfRelativeSDSize)
+	return
+}
+
+func (selfRelativeSD *SECURITY_DESCRIPTOR) copySelfRelativeSecurityDescriptor() *SECURITY_DESCRIPTOR {
+	sdBytes := make([]byte, selfRelativeSD.Length())
+	copy(sdBytes, (*[(1 << 31) - 1]byte)(unsafe.Pointer(selfRelativeSD))[:len(sdBytes)])
+	return (*SECURITY_DESCRIPTOR)(unsafe.Pointer(&sdBytes[0]))
+}
+
+// SecurityDescriptorFromString converts an SDDL string describing a security descriptor into a
+// self-relative security descriptor object allocated on the Go heap.
+func SecurityDescriptorFromString(sddl string) (sd *SECURITY_DESCRIPTOR, err error) {
+	var winHeapSD *SECURITY_DESCRIPTOR
+	err = convertStringSecurityDescriptorToSecurityDescriptor(sddl, 1, &winHeapSD, nil)
+	if err != nil {
+		return
+	}
+	defer LocalFree(Handle(unsafe.Pointer(winHeapSD)))
+	return winHeapSD.copySelfRelativeSecurityDescriptor(), nil
+}
+
+// GetSecurityInfo queries the security information for a given handle and returns the self-relative security
+// descriptor result on the Go heap.
+func GetSecurityInfo(handle Handle, objectType SE_OBJECT_TYPE, securityInformation SECURITY_INFORMATION) (sd *SECURITY_DESCRIPTOR, err error) {
+	var winHeapSD *SECURITY_DESCRIPTOR
+	err = getSecurityInfo(handle, objectType, securityInformation, nil, nil, nil, nil, &winHeapSD)
+	if err != nil {
+		return
+	}
+	defer LocalFree(Handle(unsafe.Pointer(winHeapSD)))
+	return winHeapSD.copySelfRelativeSecurityDescriptor(), nil
+}
+
+// GetNamedSecurityInfo queries the security information for a given named object and returns the self-relative security
+// descriptor result on the Go heap.
+func GetNamedSecurityInfo(objectName string, objectType SE_OBJECT_TYPE, securityInformation SECURITY_INFORMATION) (sd *SECURITY_DESCRIPTOR, err error) {
+	var winHeapSD *SECURITY_DESCRIPTOR
+	err = getNamedSecurityInfo(objectName, objectType, securityInformation, nil, nil, nil, nil, &winHeapSD)
+	if err != nil {
+		return
+	}
+	defer LocalFree(Handle(unsafe.Pointer(winHeapSD)))
+	return winHeapSD.copySelfRelativeSecurityDescriptor(), nil
+}
+
+// BuildSecurityDescriptor makes a new security descriptor using the input trustees, explicit access lists, and
+// prior security descriptor to be merged, any of which can be nil, returning the self-relative security descriptor
+// result on the Go heap.
+func BuildSecurityDescriptor(owner *TRUSTEE, group *TRUSTEE, accessEntries []EXPLICIT_ACCESS, auditEntries []EXPLICIT_ACCESS, mergedSecurityDescriptor *SECURITY_DESCRIPTOR) (sd *SECURITY_DESCRIPTOR, err error) {
+	var winHeapSD *SECURITY_DESCRIPTOR
+	var winHeapSDSize uint32
+	var firstAccessEntry *EXPLICIT_ACCESS
+	if len(accessEntries) > 0 {
+		firstAccessEntry = &accessEntries[0]
+	}
+	var firstAuditEntry *EXPLICIT_ACCESS
+	if len(auditEntries) > 0 {
+		firstAuditEntry = &auditEntries[0]
+	}
+	err = buildSecurityDescriptor(owner, group, uint32(len(accessEntries)), firstAccessEntry, uint32(len(auditEntries)), firstAuditEntry, mergedSecurityDescriptor, &winHeapSDSize, &winHeapSD)
+	if err != nil {
+		return
+	}
+	defer LocalFree(Handle(unsafe.Pointer(winHeapSD)))
+	return winHeapSD.copySelfRelativeSecurityDescriptor(), nil
+}
+
+// NewSecurityDescriptor creates and initializes a new absolute security descriptor.
+func NewSecurityDescriptor() (absoluteSD *SECURITY_DESCRIPTOR, err error) {
+	absoluteSD = &SECURITY_DESCRIPTOR{}
+	err = initializeSecurityDescriptor(absoluteSD, 1)
+	return
+}
+
+// ACLFromEntries returns a new ACL on the Go heap containing a list of explicit entries as well as those of another ACL.
+// Both explicitEntries and mergedACL are optional and can be nil.
+func ACLFromEntries(explicitEntries []EXPLICIT_ACCESS, mergedACL *ACL) (acl *ACL, err error) {
+	var firstExplicitEntry *EXPLICIT_ACCESS
+	if len(explicitEntries) > 0 {
+		firstExplicitEntry = &explicitEntries[0]
+	}
+	var winHeapACL *ACL
+	err = setEntriesInAcl(uint32(len(explicitEntries)), firstExplicitEntry, mergedACL, &winHeapACL)
+	if err != nil {
+		return
+	}
+	defer LocalFree(Handle(unsafe.Pointer(winHeapACL)))
+	aclBytes := make([]byte, winHeapACL.aclSize)
+	copy(aclBytes, (*[(1 << 31) - 1]byte)(unsafe.Pointer(winHeapACL))[:len(aclBytes)])
+	return (*ACL)(unsafe.Pointer(&aclBytes[0])), nil
+}
diff --git a/vendor/golang.org/x/sys/windows/service.go b/vendor/golang.org/x/sys/windows/service.go
new file mode 100644
index 0000000..847e00b
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/service.go
@@ -0,0 +1,229 @@
+// Copyright 2012 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build windows
+
+package windows
+
+const (
+	SC_MANAGER_CONNECT            = 1
+	SC_MANAGER_CREATE_SERVICE     = 2
+	SC_MANAGER_ENUMERATE_SERVICE  = 4
+	SC_MANAGER_LOCK               = 8
+	SC_MANAGER_QUERY_LOCK_STATUS  = 16
+	SC_MANAGER_MODIFY_BOOT_CONFIG = 32
+	SC_MANAGER_ALL_ACCESS         = 0xf003f
+)
+
+//sys	OpenSCManager(machineName *uint16, databaseName *uint16, access uint32) (handle Handle, err error) [failretval==0] = advapi32.OpenSCManagerW
+
+const (
+	SERVICE_KERNEL_DRIVER       = 1
+	SERVICE_FILE_SYSTEM_DRIVER  = 2
+	SERVICE_ADAPTER             = 4
+	SERVICE_RECOGNIZER_DRIVER   = 8
+	SERVICE_WIN32_OWN_PROCESS   = 16
+	SERVICE_WIN32_SHARE_PROCESS = 32
+	SERVICE_WIN32               = SERVICE_WIN32_OWN_PROCESS | SERVICE_WIN32_SHARE_PROCESS
+	SERVICE_INTERACTIVE_PROCESS = 256
+	SERVICE_DRIVER              = SERVICE_KERNEL_DRIVER | SERVICE_FILE_SYSTEM_DRIVER | SERVICE_RECOGNIZER_DRIVER
+	SERVICE_TYPE_ALL            = SERVICE_WIN32 | SERVICE_ADAPTER | SERVICE_DRIVER | SERVICE_INTERACTIVE_PROCESS
+
+	SERVICE_BOOT_START   = 0
+	SERVICE_SYSTEM_START = 1
+	SERVICE_AUTO_START   = 2
+	SERVICE_DEMAND_START = 3
+	SERVICE_DISABLED     = 4
+
+	SERVICE_ERROR_IGNORE   = 0
+	SERVICE_ERROR_NORMAL   = 1
+	SERVICE_ERROR_SEVERE   = 2
+	SERVICE_ERROR_CRITICAL = 3
+
+	SC_STATUS_PROCESS_INFO = 0
+
+	SC_ACTION_NONE        = 0
+	SC_ACTION_RESTART     = 1
+	SC_ACTION_REBOOT      = 2
+	SC_ACTION_RUN_COMMAND = 3
+
+	SERVICE_STOPPED          = 1
+	SERVICE_START_PENDING    = 2
+	SERVICE_STOP_PENDING     = 3
+	SERVICE_RUNNING          = 4
+	SERVICE_CONTINUE_PENDING = 5
+	SERVICE_PAUSE_PENDING    = 6
+	SERVICE_PAUSED           = 7
+	SERVICE_NO_CHANGE        = 0xffffffff
+
+	SERVICE_ACCEPT_STOP                  = 1
+	SERVICE_ACCEPT_PAUSE_CONTINUE        = 2
+	SERVICE_ACCEPT_SHUTDOWN              = 4
+	SERVICE_ACCEPT_PARAMCHANGE           = 8
+	SERVICE_ACCEPT_NETBINDCHANGE         = 16
+	SERVICE_ACCEPT_HARDWAREPROFILECHANGE = 32
+	SERVICE_ACCEPT_POWEREVENT            = 64
+	SERVICE_ACCEPT_SESSIONCHANGE         = 128
+
+	SERVICE_CONTROL_STOP                  = 1
+	SERVICE_CONTROL_PAUSE                 = 2
+	SERVICE_CONTROL_CONTINUE              = 3
+	SERVICE_CONTROL_INTERROGATE           = 4
+	SERVICE_CONTROL_SHUTDOWN              = 5
+	SERVICE_CONTROL_PARAMCHANGE           = 6
+	SERVICE_CONTROL_NETBINDADD            = 7
+	SERVICE_CONTROL_NETBINDREMOVE         = 8
+	SERVICE_CONTROL_NETBINDENABLE         = 9
+	SERVICE_CONTROL_NETBINDDISABLE        = 10
+	SERVICE_CONTROL_DEVICEEVENT           = 11
+	SERVICE_CONTROL_HARDWAREPROFILECHANGE = 12
+	SERVICE_CONTROL_POWEREVENT            = 13
+	SERVICE_CONTROL_SESSIONCHANGE         = 14
+
+	SERVICE_ACTIVE    = 1
+	SERVICE_INACTIVE  = 2
+	SERVICE_STATE_ALL = 3
+
+	SERVICE_QUERY_CONFIG         = 1
+	SERVICE_CHANGE_CONFIG        = 2
+	SERVICE_QUERY_STATUS         = 4
+	SERVICE_ENUMERATE_DEPENDENTS = 8
+	SERVICE_START                = 16
+	SERVICE_STOP                 = 32
+	SERVICE_PAUSE_CONTINUE       = 64
+	SERVICE_INTERROGATE          = 128
+	SERVICE_USER_DEFINED_CONTROL = 256
+	SERVICE_ALL_ACCESS           = STANDARD_RIGHTS_REQUIRED | SERVICE_QUERY_CONFIG | SERVICE_CHANGE_CONFIG | SERVICE_QUERY_STATUS | SERVICE_ENUMERATE_DEPENDENTS | SERVICE_START | SERVICE_STOP | SERVICE_PAUSE_CONTINUE | SERVICE_INTERROGATE | SERVICE_USER_DEFINED_CONTROL
+
+	SERVICE_RUNS_IN_SYSTEM_PROCESS = 1
+
+	SERVICE_CONFIG_DESCRIPTION              = 1
+	SERVICE_CONFIG_FAILURE_ACTIONS          = 2
+	SERVICE_CONFIG_DELAYED_AUTO_START_INFO  = 3
+	SERVICE_CONFIG_FAILURE_ACTIONS_FLAG     = 4
+	SERVICE_CONFIG_SERVICE_SID_INFO         = 5
+	SERVICE_CONFIG_REQUIRED_PRIVILEGES_INFO = 6
+	SERVICE_CONFIG_PRESHUTDOWN_INFO         = 7
+	SERVICE_CONFIG_TRIGGER_INFO             = 8
+	SERVICE_CONFIG_PREFERRED_NODE           = 9
+	SERVICE_CONFIG_LAUNCH_PROTECTED         = 12
+
+	SERVICE_SID_TYPE_NONE         = 0
+	SERVICE_SID_TYPE_UNRESTRICTED = 1
+	SERVICE_SID_TYPE_RESTRICTED   = 2 | SERVICE_SID_TYPE_UNRESTRICTED
+
+	SC_ENUM_PROCESS_INFO = 0
+
+	SERVICE_NOTIFY_STATUS_CHANGE    = 2
+	SERVICE_NOTIFY_STOPPED          = 0x00000001
+	SERVICE_NOTIFY_START_PENDING    = 0x00000002
+	SERVICE_NOTIFY_STOP_PENDING     = 0x00000004
+	SERVICE_NOTIFY_RUNNING          = 0x00000008
+	SERVICE_NOTIFY_CONTINUE_PENDING = 0x00000010
+	SERVICE_NOTIFY_PAUSE_PENDING    = 0x00000020
+	SERVICE_NOTIFY_PAUSED           = 0x00000040
+	SERVICE_NOTIFY_CREATED          = 0x00000080
+	SERVICE_NOTIFY_DELETED          = 0x00000100
+	SERVICE_NOTIFY_DELETE_PENDING   = 0x00000200
+)
+
+type SERVICE_STATUS struct {
+	ServiceType             uint32
+	CurrentState            uint32
+	ControlsAccepted        uint32
+	Win32ExitCode           uint32
+	ServiceSpecificExitCode uint32
+	CheckPoint              uint32
+	WaitHint                uint32
+}
+
+type SERVICE_TABLE_ENTRY struct {
+	ServiceName *uint16
+	ServiceProc uintptr
+}
+
+type QUERY_SERVICE_CONFIG struct {
+	ServiceType      uint32
+	StartType        uint32
+	ErrorControl     uint32
+	BinaryPathName   *uint16
+	LoadOrderGroup   *uint16
+	TagId            uint32
+	Dependencies     *uint16
+	ServiceStartName *uint16
+	DisplayName      *uint16
+}
+
+type SERVICE_DESCRIPTION struct {
+	Description *uint16
+}
+
+type SERVICE_DELAYED_AUTO_START_INFO struct {
+	IsDelayedAutoStartUp uint32
+}
+
+type SERVICE_STATUS_PROCESS struct {
+	ServiceType             uint32
+	CurrentState            uint32
+	ControlsAccepted        uint32
+	Win32ExitCode           uint32
+	ServiceSpecificExitCode uint32
+	CheckPoint              uint32
+	WaitHint                uint32
+	ProcessId               uint32
+	ServiceFlags            uint32
+}
+
+type ENUM_SERVICE_STATUS_PROCESS struct {
+	ServiceName          *uint16
+	DisplayName          *uint16
+	ServiceStatusProcess SERVICE_STATUS_PROCESS
+}
+
+type SERVICE_NOTIFY struct {
+	Version               uint32
+	NotifyCallback        uintptr
+	Context               uintptr
+	NotificationStatus    uint32
+	ServiceStatus         SERVICE_STATUS_PROCESS
+	NotificationTriggered uint32
+	ServiceNames          *uint16
+}
+
+type SERVICE_FAILURE_ACTIONS struct {
+	ResetPeriod  uint32
+	RebootMsg    *uint16
+	Command      *uint16
+	ActionsCount uint32
+	Actions      *SC_ACTION
+}
+
+type SC_ACTION struct {
+	Type  uint32
+	Delay uint32
+}
+
+type QUERY_SERVICE_LOCK_STATUS struct {
+	IsLocked     uint32
+	LockOwner    *uint16
+	LockDuration uint32
+}
+
+//sys	CloseServiceHandle(handle Handle) (err error) = advapi32.CloseServiceHandle
+//sys	CreateService(mgr Handle, serviceName *uint16, displayName *uint16, access uint32, srvType uint32, startType uint32, errCtl uint32, pathName *uint16, loadOrderGroup *uint16, tagId *uint32, dependencies *uint16, serviceStartName *uint16, password *uint16) (handle Handle, err error) [failretval==0] = advapi32.CreateServiceW
+//sys	OpenService(mgr Handle, serviceName *uint16, access uint32) (handle Handle, err error) [failretval==0] = advapi32.OpenServiceW
+//sys	DeleteService(service Handle) (err error) = advapi32.DeleteService
+//sys	StartService(service Handle, numArgs uint32, argVectors **uint16) (err error) = advapi32.StartServiceW
+//sys	QueryServiceStatus(service Handle, status *SERVICE_STATUS) (err error) = advapi32.QueryServiceStatus
+//sys	QueryServiceLockStatus(mgr Handle, lockStatus *QUERY_SERVICE_LOCK_STATUS, bufSize uint32, bytesNeeded *uint32) (err error) = advapi32.QueryServiceLockStatusW
+//sys	ControlService(service Handle, control uint32, status *SERVICE_STATUS) (err error) = advapi32.ControlService
+//sys	StartServiceCtrlDispatcher(serviceTable *SERVICE_TABLE_ENTRY) (err error) = advapi32.StartServiceCtrlDispatcherW
+//sys	SetServiceStatus(service Handle, serviceStatus *SERVICE_STATUS) (err error) = advapi32.SetServiceStatus
+//sys	ChangeServiceConfig(service Handle, serviceType uint32, startType uint32, errorControl uint32, binaryPathName *uint16, loadOrderGroup *uint16, tagId *uint32, dependencies *uint16, serviceStartName *uint16, password *uint16, displayName *uint16) (err error) = advapi32.ChangeServiceConfigW
+//sys	QueryServiceConfig(service Handle, serviceConfig *QUERY_SERVICE_CONFIG, bufSize uint32, bytesNeeded *uint32) (err error) = advapi32.QueryServiceConfigW
+//sys	ChangeServiceConfig2(service Handle, infoLevel uint32, info *byte) (err error) = advapi32.ChangeServiceConfig2W
+//sys	QueryServiceConfig2(service Handle, infoLevel uint32, buff *byte, buffSize uint32, bytesNeeded *uint32) (err error) = advapi32.QueryServiceConfig2W
+//sys	EnumServicesStatusEx(mgr Handle, infoLevel uint32, serviceType uint32, serviceState uint32, services *byte, bufSize uint32, bytesNeeded *uint32, servicesReturned *uint32, resumeHandle *uint32, groupName *uint16) (err error) = advapi32.EnumServicesStatusExW
+//sys	QueryServiceStatusEx(service Handle, infoLevel uint32, buff *byte, buffSize uint32, bytesNeeded *uint32) (err error) = advapi32.QueryServiceStatusEx
+//sys	NotifyServiceStatusChange(service Handle, notifyMask uint32, notifier *SERVICE_NOTIFY) (ret error) = advapi32.NotifyServiceStatusChangeW
diff --git a/vendor/golang.org/x/sys/windows/str.go b/vendor/golang.org/x/sys/windows/str.go
new file mode 100644
index 0000000..917cc2a
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/str.go
@@ -0,0 +1,22 @@
+// Copyright 2009 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build windows
+
+package windows
+
+func itoa(val int) string { // do it here rather than with fmt to avoid dependency
+	if val < 0 {
+		return "-" + itoa(-val)
+	}
+	var buf [32]byte // big enough for int64
+	i := len(buf) - 1
+	for val >= 10 {
+		buf[i] = byte(val%10 + '0')
+		i--
+		val /= 10
+	}
+	buf[i] = byte(val + '0')
+	return string(buf[i:])
+}
diff --git a/vendor/golang.org/x/sys/windows/syscall.go b/vendor/golang.org/x/sys/windows/syscall.go
new file mode 100644
index 0000000..af828a9
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/syscall.go
@@ -0,0 +1,74 @@
+// Copyright 2009 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build windows
+
+// Package windows contains an interface to the low-level operating system
+// primitives. OS details vary depending on the underlying system, and
+// by default, godoc will display the OS-specific documentation for the current
+// system. If you want godoc to display syscall documentation for another
+// system, set $GOOS and $GOARCH to the desired system. For example, if
+// you want to view documentation for freebsd/arm on linux/amd64, set $GOOS
+// to freebsd and $GOARCH to arm.
+//
+// The primary use of this package is inside other packages that provide a more
+// portable interface to the system, such as "os", "time" and "net".  Use
+// those packages rather than this one if you can.
+//
+// For details of the functions and data types in this package consult
+// the manuals for the appropriate operating system.
+//
+// These calls return err == nil to indicate success; otherwise
+// err represents an operating system error describing the failure and
+// holds a value of type syscall.Errno.
+package windows // import "golang.org/x/sys/windows"
+
+import (
+	"syscall"
+)
+
+// ByteSliceFromString returns a NUL-terminated slice of bytes
+// containing the text of s. If s contains a NUL byte at any
+// location, it returns (nil, syscall.EINVAL).
+func ByteSliceFromString(s string) ([]byte, error) {
+	for i := 0; i < len(s); i++ {
+		if s[i] == 0 {
+			return nil, syscall.EINVAL
+		}
+	}
+	a := make([]byte, len(s)+1)
+	copy(a, s)
+	return a, nil
+}
+
+// BytePtrFromString returns a pointer to a NUL-terminated array of
+// bytes containing the text of s. If s contains a NUL byte at any
+// location, it returns (nil, syscall.EINVAL).
+func BytePtrFromString(s string) (*byte, error) {
+	a, err := ByteSliceFromString(s)
+	if err != nil {
+		return nil, err
+	}
+	return &a[0], nil
+}
+
+// Single-word zero for use when we need a valid pointer to 0 bytes.
+// See mksyscall.pl.
+var _zero uintptr
+
+func (ts *Timespec) Unix() (sec int64, nsec int64) {
+	return int64(ts.Sec), int64(ts.Nsec)
+}
+
+func (tv *Timeval) Unix() (sec int64, nsec int64) {
+	return int64(tv.Sec), int64(tv.Usec) * 1000
+}
+
+func (ts *Timespec) Nano() int64 {
+	return int64(ts.Sec)*1e9 + int64(ts.Nsec)
+}
+
+func (tv *Timeval) Nano() int64 {
+	return int64(tv.Sec)*1e9 + int64(tv.Usec)*1000
+}
diff --git a/vendor/golang.org/x/sys/windows/syscall_windows.go b/vendor/golang.org/x/sys/windows/syscall_windows.go
new file mode 100644
index 0000000..e3b16c2
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/syscall_windows.go
@@ -0,0 +1,1370 @@
+// Copyright 2009 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// Windows system calls.
+
+package windows
+
+import (
+	errorspkg "errors"
+	"sync"
+	"syscall"
+	"time"
+	"unicode/utf16"
+	"unsafe"
+)
+
+type Handle uintptr
+
+const (
+	InvalidHandle = ^Handle(0)
+
+	// Flags for DefineDosDevice.
+	DDD_EXACT_MATCH_ON_REMOVE = 0x00000004
+	DDD_NO_BROADCAST_SYSTEM   = 0x00000008
+	DDD_RAW_TARGET_PATH       = 0x00000001
+	DDD_REMOVE_DEFINITION     = 0x00000002
+
+	// Return values for GetDriveType.
+	DRIVE_UNKNOWN     = 0
+	DRIVE_NO_ROOT_DIR = 1
+	DRIVE_REMOVABLE   = 2
+	DRIVE_FIXED       = 3
+	DRIVE_REMOTE      = 4
+	DRIVE_CDROM       = 5
+	DRIVE_RAMDISK     = 6
+
+	// File system flags from GetVolumeInformation and GetVolumeInformationByHandle.
+	FILE_CASE_SENSITIVE_SEARCH        = 0x00000001
+	FILE_CASE_PRESERVED_NAMES         = 0x00000002
+	FILE_FILE_COMPRESSION             = 0x00000010
+	FILE_DAX_VOLUME                   = 0x20000000
+	FILE_NAMED_STREAMS                = 0x00040000
+	FILE_PERSISTENT_ACLS              = 0x00000008
+	FILE_READ_ONLY_VOLUME             = 0x00080000
+	FILE_SEQUENTIAL_WRITE_ONCE        = 0x00100000
+	FILE_SUPPORTS_ENCRYPTION          = 0x00020000
+	FILE_SUPPORTS_EXTENDED_ATTRIBUTES = 0x00800000
+	FILE_SUPPORTS_HARD_LINKS          = 0x00400000
+	FILE_SUPPORTS_OBJECT_IDS          = 0x00010000
+	FILE_SUPPORTS_OPEN_BY_FILE_ID     = 0x01000000
+	FILE_SUPPORTS_REPARSE_POINTS      = 0x00000080
+	FILE_SUPPORTS_SPARSE_FILES        = 0x00000040
+	FILE_SUPPORTS_TRANSACTIONS        = 0x00200000
+	FILE_SUPPORTS_USN_JOURNAL         = 0x02000000
+	FILE_UNICODE_ON_DISK              = 0x00000004
+	FILE_VOLUME_IS_COMPRESSED         = 0x00008000
+	FILE_VOLUME_QUOTAS                = 0x00000020
+
+	// Flags for LockFileEx.
+	LOCKFILE_FAIL_IMMEDIATELY = 0x00000001
+	LOCKFILE_EXCLUSIVE_LOCK   = 0x00000002
+
+	// Return values of SleepEx and other APC functions
+	STATUS_USER_APC    = 0x000000C0
+	WAIT_IO_COMPLETION = STATUS_USER_APC
+)
+
+// StringToUTF16 is deprecated. Use UTF16FromString instead.
+// If s contains a NUL byte this function panics instead of
+// returning an error.
+func StringToUTF16(s string) []uint16 {
+	a, err := UTF16FromString(s)
+	if err != nil {
+		panic("windows: string with NUL passed to StringToUTF16")
+	}
+	return a
+}
+
+// UTF16FromString returns the UTF-16 encoding of the UTF-8 string
+// s, with a terminating NUL added. If s contains a NUL byte at any
+// location, it returns (nil, syscall.EINVAL).
+func UTF16FromString(s string) ([]uint16, error) {
+	for i := 0; i < len(s); i++ {
+		if s[i] == 0 {
+			return nil, syscall.EINVAL
+		}
+	}
+	return utf16.Encode([]rune(s + "\x00")), nil
+}
+
+// UTF16ToString returns the UTF-8 encoding of the UTF-16 sequence s,
+// with a terminating NUL removed.
+func UTF16ToString(s []uint16) string {
+	for i, v := range s {
+		if v == 0 {
+			s = s[0:i]
+			break
+		}
+	}
+	return string(utf16.Decode(s))
+}
+
+// StringToUTF16Ptr is deprecated. Use UTF16PtrFromString instead.
+// If s contains a NUL byte this function panics instead of
+// returning an error.
+func StringToUTF16Ptr(s string) *uint16 { return &StringToUTF16(s)[0] }
+
+// UTF16PtrFromString returns pointer to the UTF-16 encoding of
+// the UTF-8 string s, with a terminating NUL added. If s
+// contains a NUL byte at any location, it returns (nil, syscall.EINVAL).
+func UTF16PtrFromString(s string) (*uint16, error) {
+	a, err := UTF16FromString(s)
+	if err != nil {
+		return nil, err
+	}
+	return &a[0], nil
+}
+
+func Getpagesize() int { return 4096 }
+
+// NewCallback converts a Go function to a function pointer conforming to the stdcall calling convention.
+// This is useful when interoperating with Windows code requiring callbacks.
+// The argument is expected to be a function with with one uintptr-sized result. The function must not have arguments with size larger than the size of uintptr.
+func NewCallback(fn interface{}) uintptr {
+	return syscall.NewCallback(fn)
+}
+
+// NewCallbackCDecl converts a Go function to a function pointer conforming to the cdecl calling convention.
+// This is useful when interoperating with Windows code requiring callbacks.
+// The argument is expected to be a function with with one uintptr-sized result. The function must not have arguments with size larger than the size of uintptr.
+func NewCallbackCDecl(fn interface{}) uintptr {
+	return syscall.NewCallbackCDecl(fn)
+}
+
+// windows api calls
+
+//sys	GetLastError() (lasterr error)
+//sys	LoadLibrary(libname string) (handle Handle, err error) = LoadLibraryW
+//sys	LoadLibraryEx(libname string, zero Handle, flags uintptr) (handle Handle, err error) = LoadLibraryExW
+//sys	FreeLibrary(handle Handle) (err error)
+//sys	GetProcAddress(module Handle, procname string) (proc uintptr, err error)
+//sys	GetVersion() (ver uint32, err error)
+//sys	FormatMessage(flags uint32, msgsrc uintptr, msgid uint32, langid uint32, buf []uint16, args *byte) (n uint32, err error) = FormatMessageW
+//sys	ExitProcess(exitcode uint32)
+//sys	IsWow64Process(handle Handle, isWow64 *bool) (err error) = IsWow64Process
+//sys	CreateFile(name *uint16, access uint32, mode uint32, sa *SecurityAttributes, createmode uint32, attrs uint32, templatefile Handle) (handle Handle, err error) [failretval==InvalidHandle] = CreateFileW
+//sys	ReadFile(handle Handle, buf []byte, done *uint32, overlapped *Overlapped) (err error)
+//sys	WriteFile(handle Handle, buf []byte, done *uint32, overlapped *Overlapped) (err error)
+//sys	GetOverlappedResult(handle Handle, overlapped *Overlapped, done *uint32, wait bool) (err error)
+//sys	SetFilePointer(handle Handle, lowoffset int32, highoffsetptr *int32, whence uint32) (newlowoffset uint32, err error) [failretval==0xffffffff]
+//sys	CloseHandle(handle Handle) (err error)
+//sys	GetStdHandle(stdhandle uint32) (handle Handle, err error) [failretval==InvalidHandle]
+//sys	SetStdHandle(stdhandle uint32, handle Handle) (err error)
+//sys	findFirstFile1(name *uint16, data *win32finddata1) (handle Handle, err error) [failretval==InvalidHandle] = FindFirstFileW
+//sys	findNextFile1(handle Handle, data *win32finddata1) (err error) = FindNextFileW
+//sys	FindClose(handle Handle) (err error)
+//sys	GetFileInformationByHandle(handle Handle, data *ByHandleFileInformation) (err error)
+//sys	GetFileInformationByHandleEx(handle Handle, class uint32, outBuffer *byte, outBufferLen uint32) (err error)
+//sys	GetCurrentDirectory(buflen uint32, buf *uint16) (n uint32, err error) = GetCurrentDirectoryW
+//sys	SetCurrentDirectory(path *uint16) (err error) = SetCurrentDirectoryW
+//sys	CreateDirectory(path *uint16, sa *SecurityAttributes) (err error) = CreateDirectoryW
+//sys	RemoveDirectory(path *uint16) (err error) = RemoveDirectoryW
+//sys	DeleteFile(path *uint16) (err error) = DeleteFileW
+//sys	MoveFile(from *uint16, to *uint16) (err error) = MoveFileW
+//sys	MoveFileEx(from *uint16, to *uint16, flags uint32) (err error) = MoveFileExW
+//sys	LockFileEx(file Handle, flags uint32, reserved uint32, bytesLow uint32, bytesHigh uint32, overlapped *Overlapped) (err error)
+//sys	UnlockFileEx(file Handle, reserved uint32, bytesLow uint32, bytesHigh uint32, overlapped *Overlapped) (err error)
+//sys	GetComputerName(buf *uint16, n *uint32) (err error) = GetComputerNameW
+//sys	GetComputerNameEx(nametype uint32, buf *uint16, n *uint32) (err error) = GetComputerNameExW
+//sys	SetEndOfFile(handle Handle) (err error)
+//sys	GetSystemTimeAsFileTime(time *Filetime)
+//sys	GetSystemTimePreciseAsFileTime(time *Filetime)
+//sys	GetTimeZoneInformation(tzi *Timezoneinformation) (rc uint32, err error) [failretval==0xffffffff]
+//sys	CreateIoCompletionPort(filehandle Handle, cphandle Handle, key uint32, threadcnt uint32) (handle Handle, err error)
+//sys	GetQueuedCompletionStatus(cphandle Handle, qty *uint32, key *uint32, overlapped **Overlapped, timeout uint32) (err error)
+//sys	PostQueuedCompletionStatus(cphandle Handle, qty uint32, key uint32, overlapped *Overlapped) (err error)
+//sys	CancelIo(s Handle) (err error)
+//sys	CancelIoEx(s Handle, o *Overlapped) (err error)
+//sys	CreateProcess(appName *uint16, commandLine *uint16, procSecurity *SecurityAttributes, threadSecurity *SecurityAttributes, inheritHandles bool, creationFlags uint32, env *uint16, currentDir *uint16, startupInfo *StartupInfo, outProcInfo *ProcessInformation) (err error) = CreateProcessW
+//sys	OpenProcess(desiredAccess uint32, inheritHandle bool, processId uint32) (handle Handle, err error)
+//sys	ShellExecute(hwnd Handle, verb *uint16, file *uint16, args *uint16, cwd *uint16, showCmd int32) (err error) = shell32.ShellExecuteW
+//sys	shGetKnownFolderPath(id *KNOWNFOLDERID, flags uint32, token Token, path **uint16) (ret error) = shell32.SHGetKnownFolderPath
+//sys	TerminateProcess(handle Handle, exitcode uint32) (err error)
+//sys	GetExitCodeProcess(handle Handle, exitcode *uint32) (err error)
+//sys	GetStartupInfo(startupInfo *StartupInfo) (err error) = GetStartupInfoW
+//sys	GetProcessTimes(handle Handle, creationTime *Filetime, exitTime *Filetime, kernelTime *Filetime, userTime *Filetime) (err error)
+//sys	DuplicateHandle(hSourceProcessHandle Handle, hSourceHandle Handle, hTargetProcessHandle Handle, lpTargetHandle *Handle, dwDesiredAccess uint32, bInheritHandle bool, dwOptions uint32) (err error)
+//sys	WaitForSingleObject(handle Handle, waitMilliseconds uint32) (event uint32, err error) [failretval==0xffffffff]
+//sys	waitForMultipleObjects(count uint32, handles uintptr, waitAll bool, waitMilliseconds uint32) (event uint32, err error) [failretval==0xffffffff] = WaitForMultipleObjects
+//sys	GetTempPath(buflen uint32, buf *uint16) (n uint32, err error) = GetTempPathW
+//sys	CreatePipe(readhandle *Handle, writehandle *Handle, sa *SecurityAttributes, size uint32) (err error)
+//sys	GetFileType(filehandle Handle) (n uint32, err error)
+//sys	CryptAcquireContext(provhandle *Handle, container *uint16, provider *uint16, provtype uint32, flags uint32) (err error) = advapi32.CryptAcquireContextW
+//sys	CryptReleaseContext(provhandle Handle, flags uint32) (err error) = advapi32.CryptReleaseContext
+//sys	CryptGenRandom(provhandle Handle, buflen uint32, buf *byte) (err error) = advapi32.CryptGenRandom
+//sys	GetEnvironmentStrings() (envs *uint16, err error) [failretval==nil] = kernel32.GetEnvironmentStringsW
+//sys	FreeEnvironmentStrings(envs *uint16) (err error) = kernel32.FreeEnvironmentStringsW
+//sys	GetEnvironmentVariable(name *uint16, buffer *uint16, size uint32) (n uint32, err error) = kernel32.GetEnvironmentVariableW
+//sys	SetEnvironmentVariable(name *uint16, value *uint16) (err error) = kernel32.SetEnvironmentVariableW
+//sys	CreateEnvironmentBlock(block **uint16, token Token, inheritExisting bool) (err error) = userenv.CreateEnvironmentBlock
+//sys	DestroyEnvironmentBlock(block *uint16) (err error) = userenv.DestroyEnvironmentBlock
+//sys	getTickCount64() (ms uint64) = kernel32.GetTickCount64
+//sys	SetFileTime(handle Handle, ctime *Filetime, atime *Filetime, wtime *Filetime) (err error)
+//sys	GetFileAttributes(name *uint16) (attrs uint32, err error) [failretval==INVALID_FILE_ATTRIBUTES] = kernel32.GetFileAttributesW
+//sys	SetFileAttributes(name *uint16, attrs uint32) (err error) = kernel32.SetFileAttributesW
+//sys	GetFileAttributesEx(name *uint16, level uint32, info *byte) (err error) = kernel32.GetFileAttributesExW
+//sys	GetCommandLine() (cmd *uint16) = kernel32.GetCommandLineW
+//sys	CommandLineToArgv(cmd *uint16, argc *int32) (argv *[8192]*[8192]uint16, err error) [failretval==nil] = shell32.CommandLineToArgvW
+//sys	LocalFree(hmem Handle) (handle Handle, err error) [failretval!=0]
+//sys	SetHandleInformation(handle Handle, mask uint32, flags uint32) (err error)
+//sys	FlushFileBuffers(handle Handle) (err error)
+//sys	GetFullPathName(path *uint16, buflen uint32, buf *uint16, fname **uint16) (n uint32, err error) = kernel32.GetFullPathNameW
+//sys	GetLongPathName(path *uint16, buf *uint16, buflen uint32) (n uint32, err error) = kernel32.GetLongPathNameW
+//sys	GetShortPathName(longpath *uint16, shortpath *uint16, buflen uint32) (n uint32, err error) = kernel32.GetShortPathNameW
+//sys	CreateFileMapping(fhandle Handle, sa *SecurityAttributes, prot uint32, maxSizeHigh uint32, maxSizeLow uint32, name *uint16) (handle Handle, err error) = kernel32.CreateFileMappingW
+//sys	MapViewOfFile(handle Handle, access uint32, offsetHigh uint32, offsetLow uint32, length uintptr) (addr uintptr, err error)
+//sys	UnmapViewOfFile(addr uintptr) (err error)
+//sys	FlushViewOfFile(addr uintptr, length uintptr) (err error)
+//sys	VirtualLock(addr uintptr, length uintptr) (err error)
+//sys	VirtualUnlock(addr uintptr, length uintptr) (err error)
+//sys	VirtualAlloc(address uintptr, size uintptr, alloctype uint32, protect uint32) (value uintptr, err error) = kernel32.VirtualAlloc
+//sys	VirtualFree(address uintptr, size uintptr, freetype uint32) (err error) = kernel32.VirtualFree
+//sys	VirtualProtect(address uintptr, size uintptr, newprotect uint32, oldprotect *uint32) (err error) = kernel32.VirtualProtect
+//sys	TransmitFile(s Handle, handle Handle, bytesToWrite uint32, bytsPerSend uint32, overlapped *Overlapped, transmitFileBuf *TransmitFileBuffers, flags uint32) (err error) = mswsock.TransmitFile
+//sys	ReadDirectoryChanges(handle Handle, buf *byte, buflen uint32, watchSubTree bool, mask uint32, retlen *uint32, overlapped *Overlapped, completionRoutine uintptr) (err error) = kernel32.ReadDirectoryChangesW
+//sys	CertOpenSystemStore(hprov Handle, name *uint16) (store Handle, err error) = crypt32.CertOpenSystemStoreW
+//sys   CertOpenStore(storeProvider uintptr, msgAndCertEncodingType uint32, cryptProv uintptr, flags uint32, para uintptr) (handle Handle, err error) [failretval==InvalidHandle] = crypt32.CertOpenStore
+//sys	CertEnumCertificatesInStore(store Handle, prevContext *CertContext) (context *CertContext, err error) [failretval==nil] = crypt32.CertEnumCertificatesInStore
+//sys   CertAddCertificateContextToStore(store Handle, certContext *CertContext, addDisposition uint32, storeContext **CertContext) (err error) = crypt32.CertAddCertificateContextToStore
+//sys	CertCloseStore(store Handle, flags uint32) (err error) = crypt32.CertCloseStore
+//sys   CertGetCertificateChain(engine Handle, leaf *CertContext, time *Filetime, additionalStore Handle, para *CertChainPara, flags uint32, reserved uintptr, chainCtx **CertChainContext) (err error) = crypt32.CertGetCertificateChain
+//sys   CertFreeCertificateChain(ctx *CertChainContext) = crypt32.CertFreeCertificateChain
+//sys   CertCreateCertificateContext(certEncodingType uint32, certEncoded *byte, encodedLen uint32) (context *CertContext, err error) [failretval==nil] = crypt32.CertCreateCertificateContext
+//sys   CertFreeCertificateContext(ctx *CertContext) (err error) = crypt32.CertFreeCertificateContext
+//sys   CertVerifyCertificateChainPolicy(policyOID uintptr, chain *CertChainContext, para *CertChainPolicyPara, status *CertChainPolicyStatus) (err error) = crypt32.CertVerifyCertificateChainPolicy
+//sys	RegOpenKeyEx(key Handle, subkey *uint16, options uint32, desiredAccess uint32, result *Handle) (regerrno error) = advapi32.RegOpenKeyExW
+//sys	RegCloseKey(key Handle) (regerrno error) = advapi32.RegCloseKey
+//sys	RegQueryInfoKey(key Handle, class *uint16, classLen *uint32, reserved *uint32, subkeysLen *uint32, maxSubkeyLen *uint32, maxClassLen *uint32, valuesLen *uint32, maxValueNameLen *uint32, maxValueLen *uint32, saLen *uint32, lastWriteTime *Filetime) (regerrno error) = advapi32.RegQueryInfoKeyW
+//sys	RegEnumKeyEx(key Handle, index uint32, name *uint16, nameLen *uint32, reserved *uint32, class *uint16, classLen *uint32, lastWriteTime *Filetime) (regerrno error) = advapi32.RegEnumKeyExW
+//sys	RegQueryValueEx(key Handle, name *uint16, reserved *uint32, valtype *uint32, buf *byte, buflen *uint32) (regerrno error) = advapi32.RegQueryValueExW
+//sys	GetCurrentProcessId() (pid uint32) = kernel32.GetCurrentProcessId
+//sys	GetConsoleMode(console Handle, mode *uint32) (err error) = kernel32.GetConsoleMode
+//sys	SetConsoleMode(console Handle, mode uint32) (err error) = kernel32.SetConsoleMode
+//sys	GetConsoleScreenBufferInfo(console Handle, info *ConsoleScreenBufferInfo) (err error) = kernel32.GetConsoleScreenBufferInfo
+//sys	WriteConsole(console Handle, buf *uint16, towrite uint32, written *uint32, reserved *byte) (err error) = kernel32.WriteConsoleW
+//sys	ReadConsole(console Handle, buf *uint16, toread uint32, read *uint32, inputControl *byte) (err error) = kernel32.ReadConsoleW
+//sys	CreateToolhelp32Snapshot(flags uint32, processId uint32) (handle Handle, err error) [failretval==InvalidHandle] = kernel32.CreateToolhelp32Snapshot
+//sys	Process32First(snapshot Handle, procEntry *ProcessEntry32) (err error) = kernel32.Process32FirstW
+//sys	Process32Next(snapshot Handle, procEntry *ProcessEntry32) (err error) = kernel32.Process32NextW
+//sys	Thread32First(snapshot Handle, threadEntry *ThreadEntry32) (err error)
+//sys	Thread32Next(snapshot Handle, threadEntry *ThreadEntry32) (err error)
+//sys	DeviceIoControl(handle Handle, ioControlCode uint32, inBuffer *byte, inBufferSize uint32, outBuffer *byte, outBufferSize uint32, bytesReturned *uint32, overlapped *Overlapped) (err error)
+// This function returns 1 byte BOOLEAN rather than the 4 byte BOOL.
+//sys	CreateSymbolicLink(symlinkfilename *uint16, targetfilename *uint16, flags uint32) (err error) [failretval&0xff==0] = CreateSymbolicLinkW
+//sys	CreateHardLink(filename *uint16, existingfilename *uint16, reserved uintptr) (err error) [failretval&0xff==0] = CreateHardLinkW
+//sys	GetCurrentThreadId() (id uint32)
+//sys	CreateEvent(eventAttrs *SecurityAttributes, manualReset uint32, initialState uint32, name *uint16) (handle Handle, err error) = kernel32.CreateEventW
+//sys	CreateEventEx(eventAttrs *SecurityAttributes, name *uint16, flags uint32, desiredAccess uint32) (handle Handle, err error) = kernel32.CreateEventExW
+//sys	OpenEvent(desiredAccess uint32, inheritHandle bool, name *uint16) (handle Handle, err error) = kernel32.OpenEventW
+//sys	SetEvent(event Handle) (err error) = kernel32.SetEvent
+//sys	ResetEvent(event Handle) (err error) = kernel32.ResetEvent
+//sys	PulseEvent(event Handle) (err error) = kernel32.PulseEvent
+//sys	CreateMutex(mutexAttrs *SecurityAttributes, initialOwner bool, name *uint16) (handle Handle, err error) = kernel32.CreateMutexW
+//sys	CreateMutexEx(mutexAttrs *SecurityAttributes, name *uint16, flags uint32, desiredAccess uint32) (handle Handle, err error) = kernel32.CreateMutexExW
+//sys	OpenMutex(desiredAccess uint32, inheritHandle bool, name *uint16) (handle Handle, err error) = kernel32.OpenMutexW
+//sys	ReleaseMutex(mutex Handle) (err error) = kernel32.ReleaseMutex
+//sys	SleepEx(milliseconds uint32, alertable bool) (ret uint32) = kernel32.SleepEx
+//sys	CreateJobObject(jobAttr *SecurityAttributes, name *uint16) (handle Handle, err error) = kernel32.CreateJobObjectW
+//sys	AssignProcessToJobObject(job Handle, process Handle) (err error) = kernel32.AssignProcessToJobObject
+//sys	TerminateJobObject(job Handle, exitCode uint32) (err error) = kernel32.TerminateJobObject
+//sys	SetErrorMode(mode uint32) (ret uint32) = kernel32.SetErrorMode
+//sys	ResumeThread(thread Handle) (ret uint32, err error) [failretval==0xffffffff] = kernel32.ResumeThread
+//sys	SetPriorityClass(process Handle, priorityClass uint32) (err error) = kernel32.SetPriorityClass
+//sys	GetPriorityClass(process Handle) (ret uint32, err error) = kernel32.GetPriorityClass
+//sys	SetInformationJobObject(job Handle, JobObjectInformationClass uint32, JobObjectInformation uintptr, JobObjectInformationLength uint32) (ret int, err error)
+//sys	GenerateConsoleCtrlEvent(ctrlEvent uint32, processGroupID uint32) (err error)
+//sys	GetProcessId(process Handle) (id uint32, err error)
+//sys	OpenThread(desiredAccess uint32, inheritHandle bool, threadId uint32) (handle Handle, err error)
+//sys	SetProcessPriorityBoost(process Handle, disable bool) (err error) = kernel32.SetProcessPriorityBoost
+
+// Volume Management Functions
+//sys	DefineDosDevice(flags uint32, deviceName *uint16, targetPath *uint16) (err error) = DefineDosDeviceW
+//sys	DeleteVolumeMountPoint(volumeMountPoint *uint16) (err error) = DeleteVolumeMountPointW
+//sys	FindFirstVolume(volumeName *uint16, bufferLength uint32) (handle Handle, err error) [failretval==InvalidHandle] = FindFirstVolumeW
+//sys	FindFirstVolumeMountPoint(rootPathName *uint16, volumeMountPoint *uint16, bufferLength uint32) (handle Handle, err error) [failretval==InvalidHandle] = FindFirstVolumeMountPointW
+//sys	FindNextVolume(findVolume Handle, volumeName *uint16, bufferLength uint32) (err error) = FindNextVolumeW
+//sys	FindNextVolumeMountPoint(findVolumeMountPoint Handle, volumeMountPoint *uint16, bufferLength uint32) (err error) = FindNextVolumeMountPointW
+//sys	FindVolumeClose(findVolume Handle) (err error)
+//sys	FindVolumeMountPointClose(findVolumeMountPoint Handle) (err error)
+//sys	GetDriveType(rootPathName *uint16) (driveType uint32) = GetDriveTypeW
+//sys	GetLogicalDrives() (drivesBitMask uint32, err error) [failretval==0]
+//sys	GetLogicalDriveStrings(bufferLength uint32, buffer *uint16) (n uint32, err error) [failretval==0] = GetLogicalDriveStringsW
+//sys	GetVolumeInformation(rootPathName *uint16, volumeNameBuffer *uint16, volumeNameSize uint32, volumeNameSerialNumber *uint32, maximumComponentLength *uint32, fileSystemFlags *uint32, fileSystemNameBuffer *uint16, fileSystemNameSize uint32) (err error) = GetVolumeInformationW
+//sys	GetVolumeInformationByHandle(file Handle, volumeNameBuffer *uint16, volumeNameSize uint32, volumeNameSerialNumber *uint32, maximumComponentLength *uint32, fileSystemFlags *uint32, fileSystemNameBuffer *uint16, fileSystemNameSize uint32) (err error) = GetVolumeInformationByHandleW
+//sys	GetVolumeNameForVolumeMountPoint(volumeMountPoint *uint16, volumeName *uint16, bufferlength uint32) (err error) = GetVolumeNameForVolumeMountPointW
+//sys	GetVolumePathName(fileName *uint16, volumePathName *uint16, bufferLength uint32) (err error) = GetVolumePathNameW
+//sys	GetVolumePathNamesForVolumeName(volumeName *uint16, volumePathNames *uint16, bufferLength uint32, returnLength *uint32) (err error) = GetVolumePathNamesForVolumeNameW
+//sys	QueryDosDevice(deviceName *uint16, targetPath *uint16, max uint32) (n uint32, err error) [failretval==0] = QueryDosDeviceW
+//sys	SetVolumeLabel(rootPathName *uint16, volumeName *uint16) (err error) = SetVolumeLabelW
+//sys	SetVolumeMountPoint(volumeMountPoint *uint16, volumeName *uint16) (err error) = SetVolumeMountPointW
+//sys	MessageBox(hwnd Handle, text *uint16, caption *uint16, boxtype uint32) (ret int32, err error) [failretval==0] = user32.MessageBoxW
+//sys	ExitWindowsEx(flags uint32, reason uint32) (err error) = user32.ExitWindowsEx
+//sys	InitiateSystemShutdownEx(machineName *uint16, message *uint16, timeout uint32, forceAppsClosed bool, rebootAfterShutdown bool, reason uint32) (err error) = advapi32.InitiateSystemShutdownExW
+//sys	SetProcessShutdownParameters(level uint32, flags uint32) (err error) = kernel32.SetProcessShutdownParameters
+//sys	GetProcessShutdownParameters(level *uint32, flags *uint32) (err error) = kernel32.GetProcessShutdownParameters
+//sys	clsidFromString(lpsz *uint16, pclsid *GUID) (ret error) = ole32.CLSIDFromString
+//sys	stringFromGUID2(rguid *GUID, lpsz *uint16, cchMax int32) (chars int32) = ole32.StringFromGUID2
+//sys	coCreateGuid(pguid *GUID) (ret error) = ole32.CoCreateGuid
+//sys	CoTaskMemFree(address unsafe.Pointer) = ole32.CoTaskMemFree
+//sys	rtlGetVersion(info *OsVersionInfoEx) (ret error) = ntdll.RtlGetVersion
+//sys	rtlGetNtVersionNumbers(majorVersion *uint32, minorVersion *uint32, buildNumber *uint32) = ntdll.RtlGetNtVersionNumbers
+
+// syscall interface implementation for other packages
+
+// GetCurrentProcess returns the handle for the current process.
+// It is a pseudo handle that does not need to be closed.
+// The returned error is always nil.
+//
+// Deprecated: use CurrentProcess for the same Handle without the nil
+// error.
+func GetCurrentProcess() (Handle, error) {
+	return CurrentProcess(), nil
+}
+
+// CurrentProcess returns the handle for the current process.
+// It is a pseudo handle that does not need to be closed.
+func CurrentProcess() Handle { return Handle(^uintptr(1 - 1)) }
+
+// GetCurrentThread returns the handle for the current thread.
+// It is a pseudo handle that does not need to be closed.
+// The returned error is always nil.
+//
+// Deprecated: use CurrentThread for the same Handle without the nil
+// error.
+func GetCurrentThread() (Handle, error) {
+	return CurrentThread(), nil
+}
+
+// CurrentThread returns the handle for the current thread.
+// It is a pseudo handle that does not need to be closed.
+func CurrentThread() Handle { return Handle(^uintptr(2 - 1)) }
+
+// GetProcAddressByOrdinal retrieves the address of the exported
+// function from module by ordinal.
+func GetProcAddressByOrdinal(module Handle, ordinal uintptr) (proc uintptr, err error) {
+	r0, _, e1 := syscall.Syscall(procGetProcAddress.Addr(), 2, uintptr(module), ordinal, 0)
+	proc = uintptr(r0)
+	if proc == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func Exit(code int) { ExitProcess(uint32(code)) }
+
+func makeInheritSa() *SecurityAttributes {
+	var sa SecurityAttributes
+	sa.Length = uint32(unsafe.Sizeof(sa))
+	sa.InheritHandle = 1
+	return &sa
+}
+
+func Open(path string, mode int, perm uint32) (fd Handle, err error) {
+	if len(path) == 0 {
+		return InvalidHandle, ERROR_FILE_NOT_FOUND
+	}
+	pathp, err := UTF16PtrFromString(path)
+	if err != nil {
+		return InvalidHandle, err
+	}
+	var access uint32
+	switch mode & (O_RDONLY | O_WRONLY | O_RDWR) {
+	case O_RDONLY:
+		access = GENERIC_READ
+	case O_WRONLY:
+		access = GENERIC_WRITE
+	case O_RDWR:
+		access = GENERIC_READ | GENERIC_WRITE
+	}
+	if mode&O_CREAT != 0 {
+		access |= GENERIC_WRITE
+	}
+	if mode&O_APPEND != 0 {
+		access &^= GENERIC_WRITE
+		access |= FILE_APPEND_DATA
+	}
+	sharemode := uint32(FILE_SHARE_READ | FILE_SHARE_WRITE)
+	var sa *SecurityAttributes
+	if mode&O_CLOEXEC == 0 {
+		sa = makeInheritSa()
+	}
+	var createmode uint32
+	switch {
+	case mode&(O_CREAT|O_EXCL) == (O_CREAT | O_EXCL):
+		createmode = CREATE_NEW
+	case mode&(O_CREAT|O_TRUNC) == (O_CREAT | O_TRUNC):
+		createmode = CREATE_ALWAYS
+	case mode&O_CREAT == O_CREAT:
+		createmode = OPEN_ALWAYS
+	case mode&O_TRUNC == O_TRUNC:
+		createmode = TRUNCATE_EXISTING
+	default:
+		createmode = OPEN_EXISTING
+	}
+	h, e := CreateFile(pathp, access, sharemode, sa, createmode, FILE_ATTRIBUTE_NORMAL, 0)
+	return h, e
+}
+
+func Read(fd Handle, p []byte) (n int, err error) {
+	var done uint32
+	e := ReadFile(fd, p, &done, nil)
+	if e != nil {
+		if e == ERROR_BROKEN_PIPE {
+			// NOTE(brainman): work around ERROR_BROKEN_PIPE is returned on reading EOF from stdin
+			return 0, nil
+		}
+		return 0, e
+	}
+	if raceenabled {
+		if done > 0 {
+			raceWriteRange(unsafe.Pointer(&p[0]), int(done))
+		}
+		raceAcquire(unsafe.Pointer(&ioSync))
+	}
+	return int(done), nil
+}
+
+func Write(fd Handle, p []byte) (n int, err error) {
+	if raceenabled {
+		raceReleaseMerge(unsafe.Pointer(&ioSync))
+	}
+	var done uint32
+	e := WriteFile(fd, p, &done, nil)
+	if e != nil {
+		return 0, e
+	}
+	if raceenabled && done > 0 {
+		raceReadRange(unsafe.Pointer(&p[0]), int(done))
+	}
+	return int(done), nil
+}
+
+var ioSync int64
+
+func Seek(fd Handle, offset int64, whence int) (newoffset int64, err error) {
+	var w uint32
+	switch whence {
+	case 0:
+		w = FILE_BEGIN
+	case 1:
+		w = FILE_CURRENT
+	case 2:
+		w = FILE_END
+	}
+	hi := int32(offset >> 32)
+	lo := int32(offset)
+	// use GetFileType to check pipe, pipe can't do seek
+	ft, _ := GetFileType(fd)
+	if ft == FILE_TYPE_PIPE {
+		return 0, syscall.EPIPE
+	}
+	rlo, e := SetFilePointer(fd, lo, &hi, w)
+	if e != nil {
+		return 0, e
+	}
+	return int64(hi)<<32 + int64(rlo), nil
+}
+
+func Close(fd Handle) (err error) {
+	return CloseHandle(fd)
+}
+
+var (
+	Stdin  = getStdHandle(STD_INPUT_HANDLE)
+	Stdout = getStdHandle(STD_OUTPUT_HANDLE)
+	Stderr = getStdHandle(STD_ERROR_HANDLE)
+)
+
+func getStdHandle(stdhandle uint32) (fd Handle) {
+	r, _ := GetStdHandle(stdhandle)
+	CloseOnExec(r)
+	return r
+}
+
+const ImplementsGetwd = true
+
+func Getwd() (wd string, err error) {
+	b := make([]uint16, 300)
+	n, e := GetCurrentDirectory(uint32(len(b)), &b[0])
+	if e != nil {
+		return "", e
+	}
+	return string(utf16.Decode(b[0:n])), nil
+}
+
+func Chdir(path string) (err error) {
+	pathp, err := UTF16PtrFromString(path)
+	if err != nil {
+		return err
+	}
+	return SetCurrentDirectory(pathp)
+}
+
+func Mkdir(path string, mode uint32) (err error) {
+	pathp, err := UTF16PtrFromString(path)
+	if err != nil {
+		return err
+	}
+	return CreateDirectory(pathp, nil)
+}
+
+func Rmdir(path string) (err error) {
+	pathp, err := UTF16PtrFromString(path)
+	if err != nil {
+		return err
+	}
+	return RemoveDirectory(pathp)
+}
+
+func Unlink(path string) (err error) {
+	pathp, err := UTF16PtrFromString(path)
+	if err != nil {
+		return err
+	}
+	return DeleteFile(pathp)
+}
+
+func Rename(oldpath, newpath string) (err error) {
+	from, err := UTF16PtrFromString(oldpath)
+	if err != nil {
+		return err
+	}
+	to, err := UTF16PtrFromString(newpath)
+	if err != nil {
+		return err
+	}
+	return MoveFileEx(from, to, MOVEFILE_REPLACE_EXISTING)
+}
+
+func ComputerName() (name string, err error) {
+	var n uint32 = MAX_COMPUTERNAME_LENGTH + 1
+	b := make([]uint16, n)
+	e := GetComputerName(&b[0], &n)
+	if e != nil {
+		return "", e
+	}
+	return string(utf16.Decode(b[0:n])), nil
+}
+
+func DurationSinceBoot() time.Duration {
+	return time.Duration(getTickCount64()) * time.Millisecond
+}
+
+func Ftruncate(fd Handle, length int64) (err error) {
+	curoffset, e := Seek(fd, 0, 1)
+	if e != nil {
+		return e
+	}
+	defer Seek(fd, curoffset, 0)
+	_, e = Seek(fd, length, 0)
+	if e != nil {
+		return e
+	}
+	e = SetEndOfFile(fd)
+	if e != nil {
+		return e
+	}
+	return nil
+}
+
+func Gettimeofday(tv *Timeval) (err error) {
+	var ft Filetime
+	GetSystemTimeAsFileTime(&ft)
+	*tv = NsecToTimeval(ft.Nanoseconds())
+	return nil
+}
+
+func Pipe(p []Handle) (err error) {
+	if len(p) != 2 {
+		return syscall.EINVAL
+	}
+	var r, w Handle
+	e := CreatePipe(&r, &w, makeInheritSa(), 0)
+	if e != nil {
+		return e
+	}
+	p[0] = r
+	p[1] = w
+	return nil
+}
+
+func Utimes(path string, tv []Timeval) (err error) {
+	if len(tv) != 2 {
+		return syscall.EINVAL
+	}
+	pathp, e := UTF16PtrFromString(path)
+	if e != nil {
+		return e
+	}
+	h, e := CreateFile(pathp,
+		FILE_WRITE_ATTRIBUTES, FILE_SHARE_WRITE, nil,
+		OPEN_EXISTING, FILE_FLAG_BACKUP_SEMANTICS, 0)
+	if e != nil {
+		return e
+	}
+	defer Close(h)
+	a := NsecToFiletime(tv[0].Nanoseconds())
+	w := NsecToFiletime(tv[1].Nanoseconds())
+	return SetFileTime(h, nil, &a, &w)
+}
+
+func UtimesNano(path string, ts []Timespec) (err error) {
+	if len(ts) != 2 {
+		return syscall.EINVAL
+	}
+	pathp, e := UTF16PtrFromString(path)
+	if e != nil {
+		return e
+	}
+	h, e := CreateFile(pathp,
+		FILE_WRITE_ATTRIBUTES, FILE_SHARE_WRITE, nil,
+		OPEN_EXISTING, FILE_FLAG_BACKUP_SEMANTICS, 0)
+	if e != nil {
+		return e
+	}
+	defer Close(h)
+	a := NsecToFiletime(TimespecToNsec(ts[0]))
+	w := NsecToFiletime(TimespecToNsec(ts[1]))
+	return SetFileTime(h, nil, &a, &w)
+}
+
+func Fsync(fd Handle) (err error) {
+	return FlushFileBuffers(fd)
+}
+
+func Chmod(path string, mode uint32) (err error) {
+	p, e := UTF16PtrFromString(path)
+	if e != nil {
+		return e
+	}
+	attrs, e := GetFileAttributes(p)
+	if e != nil {
+		return e
+	}
+	if mode&S_IWRITE != 0 {
+		attrs &^= FILE_ATTRIBUTE_READONLY
+	} else {
+		attrs |= FILE_ATTRIBUTE_READONLY
+	}
+	return SetFileAttributes(p, attrs)
+}
+
+func LoadGetSystemTimePreciseAsFileTime() error {
+	return procGetSystemTimePreciseAsFileTime.Find()
+}
+
+func LoadCancelIoEx() error {
+	return procCancelIoEx.Find()
+}
+
+func LoadSetFileCompletionNotificationModes() error {
+	return procSetFileCompletionNotificationModes.Find()
+}
+
+func WaitForMultipleObjects(handles []Handle, waitAll bool, waitMilliseconds uint32) (event uint32, err error) {
+	// Every other win32 array API takes arguments as "pointer, count", except for this function. So we
+	// can't declare it as a usual [] type, because mksyscall will use the opposite order. We therefore
+	// trivially stub this ourselves.
+
+	var handlePtr *Handle
+	if len(handles) > 0 {
+		handlePtr = &handles[0]
+	}
+	return waitForMultipleObjects(uint32(len(handles)), uintptr(unsafe.Pointer(handlePtr)), waitAll, waitMilliseconds)
+}
+
+// net api calls
+
+const socket_error = uintptr(^uint32(0))
+
+//sys	WSAStartup(verreq uint32, data *WSAData) (sockerr error) = ws2_32.WSAStartup
+//sys	WSACleanup() (err error) [failretval==socket_error] = ws2_32.WSACleanup
+//sys	WSAIoctl(s Handle, iocc uint32, inbuf *byte, cbif uint32, outbuf *byte, cbob uint32, cbbr *uint32, overlapped *Overlapped, completionRoutine uintptr) (err error) [failretval==socket_error] = ws2_32.WSAIoctl
+//sys	socket(af int32, typ int32, protocol int32) (handle Handle, err error) [failretval==InvalidHandle] = ws2_32.socket
+//sys	Setsockopt(s Handle, level int32, optname int32, optval *byte, optlen int32) (err error) [failretval==socket_error] = ws2_32.setsockopt
+//sys	Getsockopt(s Handle, level int32, optname int32, optval *byte, optlen *int32) (err error) [failretval==socket_error] = ws2_32.getsockopt
+//sys	bind(s Handle, name unsafe.Pointer, namelen int32) (err error) [failretval==socket_error] = ws2_32.bind
+//sys	connect(s Handle, name unsafe.Pointer, namelen int32) (err error) [failretval==socket_error] = ws2_32.connect
+//sys	getsockname(s Handle, rsa *RawSockaddrAny, addrlen *int32) (err error) [failretval==socket_error] = ws2_32.getsockname
+//sys	getpeername(s Handle, rsa *RawSockaddrAny, addrlen *int32) (err error) [failretval==socket_error] = ws2_32.getpeername
+//sys	listen(s Handle, backlog int32) (err error) [failretval==socket_error] = ws2_32.listen
+//sys	shutdown(s Handle, how int32) (err error) [failretval==socket_error] = ws2_32.shutdown
+//sys	Closesocket(s Handle) (err error) [failretval==socket_error] = ws2_32.closesocket
+//sys	AcceptEx(ls Handle, as Handle, buf *byte, rxdatalen uint32, laddrlen uint32, raddrlen uint32, recvd *uint32, overlapped *Overlapped) (err error) = mswsock.AcceptEx
+//sys	GetAcceptExSockaddrs(buf *byte, rxdatalen uint32, laddrlen uint32, raddrlen uint32, lrsa **RawSockaddrAny, lrsalen *int32, rrsa **RawSockaddrAny, rrsalen *int32) = mswsock.GetAcceptExSockaddrs
+//sys	WSARecv(s Handle, bufs *WSABuf, bufcnt uint32, recvd *uint32, flags *uint32, overlapped *Overlapped, croutine *byte) (err error) [failretval==socket_error] = ws2_32.WSARecv
+//sys	WSASend(s Handle, bufs *WSABuf, bufcnt uint32, sent *uint32, flags uint32, overlapped *Overlapped, croutine *byte) (err error) [failretval==socket_error] = ws2_32.WSASend
+//sys	WSARecvFrom(s Handle, bufs *WSABuf, bufcnt uint32, recvd *uint32, flags *uint32,  from *RawSockaddrAny, fromlen *int32, overlapped *Overlapped, croutine *byte) (err error) [failretval==socket_error] = ws2_32.WSARecvFrom
+//sys	WSASendTo(s Handle, bufs *WSABuf, bufcnt uint32, sent *uint32, flags uint32, to *RawSockaddrAny, tolen int32,  overlapped *Overlapped, croutine *byte) (err error) [failretval==socket_error] = ws2_32.WSASendTo
+//sys	GetHostByName(name string) (h *Hostent, err error) [failretval==nil] = ws2_32.gethostbyname
+//sys	GetServByName(name string, proto string) (s *Servent, err error) [failretval==nil] = ws2_32.getservbyname
+//sys	Ntohs(netshort uint16) (u uint16) = ws2_32.ntohs
+//sys	GetProtoByName(name string) (p *Protoent, err error) [failretval==nil] = ws2_32.getprotobyname
+//sys	DnsQuery(name string, qtype uint16, options uint32, extra *byte, qrs **DNSRecord, pr *byte) (status error) = dnsapi.DnsQuery_W
+//sys	DnsRecordListFree(rl *DNSRecord, freetype uint32) = dnsapi.DnsRecordListFree
+//sys	DnsNameCompare(name1 *uint16, name2 *uint16) (same bool) = dnsapi.DnsNameCompare_W
+//sys	GetAddrInfoW(nodename *uint16, servicename *uint16, hints *AddrinfoW, result **AddrinfoW) (sockerr error) = ws2_32.GetAddrInfoW
+//sys	FreeAddrInfoW(addrinfo *AddrinfoW) = ws2_32.FreeAddrInfoW
+//sys	GetIfEntry(pIfRow *MibIfRow) (errcode error) = iphlpapi.GetIfEntry
+//sys	GetAdaptersInfo(ai *IpAdapterInfo, ol *uint32) (errcode error) = iphlpapi.GetAdaptersInfo
+//sys	SetFileCompletionNotificationModes(handle Handle, flags uint8) (err error) = kernel32.SetFileCompletionNotificationModes
+//sys	WSAEnumProtocols(protocols *int32, protocolBuffer *WSAProtocolInfo, bufferLength *uint32) (n int32, err error) [failretval==-1] = ws2_32.WSAEnumProtocolsW
+//sys	GetAdaptersAddresses(family uint32, flags uint32, reserved uintptr, adapterAddresses *IpAdapterAddresses, sizePointer *uint32) (errcode error) = iphlpapi.GetAdaptersAddresses
+//sys	GetACP() (acp uint32) = kernel32.GetACP
+//sys	MultiByteToWideChar(codePage uint32, dwFlags uint32, str *byte, nstr int32, wchar *uint16, nwchar int32) (nwrite int32, err error) = kernel32.MultiByteToWideChar
+
+// For testing: clients can set this flag to force
+// creation of IPv6 sockets to return EAFNOSUPPORT.
+var SocketDisableIPv6 bool
+
+type RawSockaddrInet4 struct {
+	Family uint16
+	Port   uint16
+	Addr   [4]byte /* in_addr */
+	Zero   [8]uint8
+}
+
+type RawSockaddrInet6 struct {
+	Family   uint16
+	Port     uint16
+	Flowinfo uint32
+	Addr     [16]byte /* in6_addr */
+	Scope_id uint32
+}
+
+type RawSockaddr struct {
+	Family uint16
+	Data   [14]int8
+}
+
+type RawSockaddrAny struct {
+	Addr RawSockaddr
+	Pad  [100]int8
+}
+
+type Sockaddr interface {
+	sockaddr() (ptr unsafe.Pointer, len int32, err error) // lowercase; only we can define Sockaddrs
+}
+
+type SockaddrInet4 struct {
+	Port int
+	Addr [4]byte
+	raw  RawSockaddrInet4
+}
+
+func (sa *SockaddrInet4) sockaddr() (unsafe.Pointer, int32, error) {
+	if sa.Port < 0 || sa.Port > 0xFFFF {
+		return nil, 0, syscall.EINVAL
+	}
+	sa.raw.Family = AF_INET
+	p := (*[2]byte)(unsafe.Pointer(&sa.raw.Port))
+	p[0] = byte(sa.Port >> 8)
+	p[1] = byte(sa.Port)
+	for i := 0; i < len(sa.Addr); i++ {
+		sa.raw.Addr[i] = sa.Addr[i]
+	}
+	return unsafe.Pointer(&sa.raw), int32(unsafe.Sizeof(sa.raw)), nil
+}
+
+type SockaddrInet6 struct {
+	Port   int
+	ZoneId uint32
+	Addr   [16]byte
+	raw    RawSockaddrInet6
+}
+
+func (sa *SockaddrInet6) sockaddr() (unsafe.Pointer, int32, error) {
+	if sa.Port < 0 || sa.Port > 0xFFFF {
+		return nil, 0, syscall.EINVAL
+	}
+	sa.raw.Family = AF_INET6
+	p := (*[2]byte)(unsafe.Pointer(&sa.raw.Port))
+	p[0] = byte(sa.Port >> 8)
+	p[1] = byte(sa.Port)
+	sa.raw.Scope_id = sa.ZoneId
+	for i := 0; i < len(sa.Addr); i++ {
+		sa.raw.Addr[i] = sa.Addr[i]
+	}
+	return unsafe.Pointer(&sa.raw), int32(unsafe.Sizeof(sa.raw)), nil
+}
+
+type RawSockaddrUnix struct {
+	Family uint16
+	Path   [UNIX_PATH_MAX]int8
+}
+
+type SockaddrUnix struct {
+	Name string
+	raw  RawSockaddrUnix
+}
+
+func (sa *SockaddrUnix) sockaddr() (unsafe.Pointer, int32, error) {
+	name := sa.Name
+	n := len(name)
+	if n > len(sa.raw.Path) {
+		return nil, 0, syscall.EINVAL
+	}
+	if n == len(sa.raw.Path) && name[0] != '@' {
+		return nil, 0, syscall.EINVAL
+	}
+	sa.raw.Family = AF_UNIX
+	for i := 0; i < n; i++ {
+		sa.raw.Path[i] = int8(name[i])
+	}
+	// length is family (uint16), name, NUL.
+	sl := int32(2)
+	if n > 0 {
+		sl += int32(n) + 1
+	}
+	if sa.raw.Path[0] == '@' {
+		sa.raw.Path[0] = 0
+		// Don't count trailing NUL for abstract address.
+		sl--
+	}
+
+	return unsafe.Pointer(&sa.raw), sl, nil
+}
+
+func (rsa *RawSockaddrAny) Sockaddr() (Sockaddr, error) {
+	switch rsa.Addr.Family {
+	case AF_UNIX:
+		pp := (*RawSockaddrUnix)(unsafe.Pointer(rsa))
+		sa := new(SockaddrUnix)
+		if pp.Path[0] == 0 {
+			// "Abstract" Unix domain socket.
+			// Rewrite leading NUL as @ for textual display.
+			// (This is the standard convention.)
+			// Not friendly to overwrite in place,
+			// but the callers below don't care.
+			pp.Path[0] = '@'
+		}
+
+		// Assume path ends at NUL.
+		// This is not technically the Linux semantics for
+		// abstract Unix domain sockets--they are supposed
+		// to be uninterpreted fixed-size binary blobs--but
+		// everyone uses this convention.
+		n := 0
+		for n < len(pp.Path) && pp.Path[n] != 0 {
+			n++
+		}
+		bytes := (*[10000]byte)(unsafe.Pointer(&pp.Path[0]))[0:n]
+		sa.Name = string(bytes)
+		return sa, nil
+
+	case AF_INET:
+		pp := (*RawSockaddrInet4)(unsafe.Pointer(rsa))
+		sa := new(SockaddrInet4)
+		p := (*[2]byte)(unsafe.Pointer(&pp.Port))
+		sa.Port = int(p[0])<<8 + int(p[1])
+		for i := 0; i < len(sa.Addr); i++ {
+			sa.Addr[i] = pp.Addr[i]
+		}
+		return sa, nil
+
+	case AF_INET6:
+		pp := (*RawSockaddrInet6)(unsafe.Pointer(rsa))
+		sa := new(SockaddrInet6)
+		p := (*[2]byte)(unsafe.Pointer(&pp.Port))
+		sa.Port = int(p[0])<<8 + int(p[1])
+		sa.ZoneId = pp.Scope_id
+		for i := 0; i < len(sa.Addr); i++ {
+			sa.Addr[i] = pp.Addr[i]
+		}
+		return sa, nil
+	}
+	return nil, syscall.EAFNOSUPPORT
+}
+
+func Socket(domain, typ, proto int) (fd Handle, err error) {
+	if domain == AF_INET6 && SocketDisableIPv6 {
+		return InvalidHandle, syscall.EAFNOSUPPORT
+	}
+	return socket(int32(domain), int32(typ), int32(proto))
+}
+
+func SetsockoptInt(fd Handle, level, opt int, value int) (err error) {
+	v := int32(value)
+	return Setsockopt(fd, int32(level), int32(opt), (*byte)(unsafe.Pointer(&v)), int32(unsafe.Sizeof(v)))
+}
+
+func Bind(fd Handle, sa Sockaddr) (err error) {
+	ptr, n, err := sa.sockaddr()
+	if err != nil {
+		return err
+	}
+	return bind(fd, ptr, n)
+}
+
+func Connect(fd Handle, sa Sockaddr) (err error) {
+	ptr, n, err := sa.sockaddr()
+	if err != nil {
+		return err
+	}
+	return connect(fd, ptr, n)
+}
+
+func Getsockname(fd Handle) (sa Sockaddr, err error) {
+	var rsa RawSockaddrAny
+	l := int32(unsafe.Sizeof(rsa))
+	if err = getsockname(fd, &rsa, &l); err != nil {
+		return
+	}
+	return rsa.Sockaddr()
+}
+
+func Getpeername(fd Handle) (sa Sockaddr, err error) {
+	var rsa RawSockaddrAny
+	l := int32(unsafe.Sizeof(rsa))
+	if err = getpeername(fd, &rsa, &l); err != nil {
+		return
+	}
+	return rsa.Sockaddr()
+}
+
+func Listen(s Handle, n int) (err error) {
+	return listen(s, int32(n))
+}
+
+func Shutdown(fd Handle, how int) (err error) {
+	return shutdown(fd, int32(how))
+}
+
+func WSASendto(s Handle, bufs *WSABuf, bufcnt uint32, sent *uint32, flags uint32, to Sockaddr, overlapped *Overlapped, croutine *byte) (err error) {
+	rsa, l, err := to.sockaddr()
+	if err != nil {
+		return err
+	}
+	return WSASendTo(s, bufs, bufcnt, sent, flags, (*RawSockaddrAny)(unsafe.Pointer(rsa)), l, overlapped, croutine)
+}
+
+func LoadGetAddrInfo() error {
+	return procGetAddrInfoW.Find()
+}
+
+var connectExFunc struct {
+	once sync.Once
+	addr uintptr
+	err  error
+}
+
+func LoadConnectEx() error {
+	connectExFunc.once.Do(func() {
+		var s Handle
+		s, connectExFunc.err = Socket(AF_INET, SOCK_STREAM, IPPROTO_TCP)
+		if connectExFunc.err != nil {
+			return
+		}
+		defer CloseHandle(s)
+		var n uint32
+		connectExFunc.err = WSAIoctl(s,
+			SIO_GET_EXTENSION_FUNCTION_POINTER,
+			(*byte)(unsafe.Pointer(&WSAID_CONNECTEX)),
+			uint32(unsafe.Sizeof(WSAID_CONNECTEX)),
+			(*byte)(unsafe.Pointer(&connectExFunc.addr)),
+			uint32(unsafe.Sizeof(connectExFunc.addr)),
+			&n, nil, 0)
+	})
+	return connectExFunc.err
+}
+
+func connectEx(s Handle, name unsafe.Pointer, namelen int32, sendBuf *byte, sendDataLen uint32, bytesSent *uint32, overlapped *Overlapped) (err error) {
+	r1, _, e1 := syscall.Syscall9(connectExFunc.addr, 7, uintptr(s), uintptr(name), uintptr(namelen), uintptr(unsafe.Pointer(sendBuf)), uintptr(sendDataLen), uintptr(unsafe.Pointer(bytesSent)), uintptr(unsafe.Pointer(overlapped)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = error(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func ConnectEx(fd Handle, sa Sockaddr, sendBuf *byte, sendDataLen uint32, bytesSent *uint32, overlapped *Overlapped) error {
+	err := LoadConnectEx()
+	if err != nil {
+		return errorspkg.New("failed to find ConnectEx: " + err.Error())
+	}
+	ptr, n, err := sa.sockaddr()
+	if err != nil {
+		return err
+	}
+	return connectEx(fd, ptr, n, sendBuf, sendDataLen, bytesSent, overlapped)
+}
+
+var sendRecvMsgFunc struct {
+	once     sync.Once
+	sendAddr uintptr
+	recvAddr uintptr
+	err      error
+}
+
+func loadWSASendRecvMsg() error {
+	sendRecvMsgFunc.once.Do(func() {
+		var s Handle
+		s, sendRecvMsgFunc.err = Socket(AF_INET, SOCK_DGRAM, IPPROTO_UDP)
+		if sendRecvMsgFunc.err != nil {
+			return
+		}
+		defer CloseHandle(s)
+		var n uint32
+		sendRecvMsgFunc.err = WSAIoctl(s,
+			SIO_GET_EXTENSION_FUNCTION_POINTER,
+			(*byte)(unsafe.Pointer(&WSAID_WSARECVMSG)),
+			uint32(unsafe.Sizeof(WSAID_WSARECVMSG)),
+			(*byte)(unsafe.Pointer(&sendRecvMsgFunc.recvAddr)),
+			uint32(unsafe.Sizeof(sendRecvMsgFunc.recvAddr)),
+			&n, nil, 0)
+		if sendRecvMsgFunc.err != nil {
+			return
+		}
+		sendRecvMsgFunc.err = WSAIoctl(s,
+			SIO_GET_EXTENSION_FUNCTION_POINTER,
+			(*byte)(unsafe.Pointer(&WSAID_WSASENDMSG)),
+			uint32(unsafe.Sizeof(WSAID_WSASENDMSG)),
+			(*byte)(unsafe.Pointer(&sendRecvMsgFunc.sendAddr)),
+			uint32(unsafe.Sizeof(sendRecvMsgFunc.sendAddr)),
+			&n, nil, 0)
+	})
+	return sendRecvMsgFunc.err
+}
+
+func WSASendMsg(fd Handle, msg *WSAMsg, flags uint32, bytesSent *uint32, overlapped *Overlapped, croutine *byte) error {
+	err := loadWSASendRecvMsg()
+	if err != nil {
+		return err
+	}
+	r1, _, e1 := syscall.Syscall6(sendRecvMsgFunc.sendAddr, 6, uintptr(fd), uintptr(unsafe.Pointer(msg)), uintptr(flags), uintptr(unsafe.Pointer(bytesSent)), uintptr(unsafe.Pointer(overlapped)), uintptr(unsafe.Pointer(croutine)))
+	if r1 == socket_error {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return err
+}
+
+func WSARecvMsg(fd Handle, msg *WSAMsg, bytesReceived *uint32, overlapped *Overlapped, croutine *byte) error {
+	err := loadWSASendRecvMsg()
+	if err != nil {
+		return err
+	}
+	r1, _, e1 := syscall.Syscall6(sendRecvMsgFunc.recvAddr, 5, uintptr(fd), uintptr(unsafe.Pointer(msg)), uintptr(unsafe.Pointer(bytesReceived)), uintptr(unsafe.Pointer(overlapped)), uintptr(unsafe.Pointer(croutine)), 0)
+	if r1 == socket_error {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return err
+}
+
+// Invented structures to support what package os expects.
+type Rusage struct {
+	CreationTime Filetime
+	ExitTime     Filetime
+	KernelTime   Filetime
+	UserTime     Filetime
+}
+
+type WaitStatus struct {
+	ExitCode uint32
+}
+
+func (w WaitStatus) Exited() bool { return true }
+
+func (w WaitStatus) ExitStatus() int { return int(w.ExitCode) }
+
+func (w WaitStatus) Signal() Signal { return -1 }
+
+func (w WaitStatus) CoreDump() bool { return false }
+
+func (w WaitStatus) Stopped() bool { return false }
+
+func (w WaitStatus) Continued() bool { return false }
+
+func (w WaitStatus) StopSignal() Signal { return -1 }
+
+func (w WaitStatus) Signaled() bool { return false }
+
+func (w WaitStatus) TrapCause() int { return -1 }
+
+// Timespec is an invented structure on Windows, but here for
+// consistency with the corresponding package for other operating systems.
+type Timespec struct {
+	Sec  int64
+	Nsec int64
+}
+
+func TimespecToNsec(ts Timespec) int64 { return int64(ts.Sec)*1e9 + int64(ts.Nsec) }
+
+func NsecToTimespec(nsec int64) (ts Timespec) {
+	ts.Sec = nsec / 1e9
+	ts.Nsec = nsec % 1e9
+	return
+}
+
+// TODO(brainman): fix all needed for net
+
+func Accept(fd Handle) (nfd Handle, sa Sockaddr, err error) { return 0, nil, syscall.EWINDOWS }
+func Recvfrom(fd Handle, p []byte, flags int) (n int, from Sockaddr, err error) {
+	return 0, nil, syscall.EWINDOWS
+}
+func Sendto(fd Handle, p []byte, flags int, to Sockaddr) (err error)       { return syscall.EWINDOWS }
+func SetsockoptTimeval(fd Handle, level, opt int, tv *Timeval) (err error) { return syscall.EWINDOWS }
+
+// The Linger struct is wrong but we only noticed after Go 1.
+// sysLinger is the real system call structure.
+
+// BUG(brainman): The definition of Linger is not appropriate for direct use
+// with Setsockopt and Getsockopt.
+// Use SetsockoptLinger instead.
+
+type Linger struct {
+	Onoff  int32
+	Linger int32
+}
+
+type sysLinger struct {
+	Onoff  uint16
+	Linger uint16
+}
+
+type IPMreq struct {
+	Multiaddr [4]byte /* in_addr */
+	Interface [4]byte /* in_addr */
+}
+
+type IPv6Mreq struct {
+	Multiaddr [16]byte /* in6_addr */
+	Interface uint32
+}
+
+func GetsockoptInt(fd Handle, level, opt int) (int, error) { return -1, syscall.EWINDOWS }
+
+func SetsockoptLinger(fd Handle, level, opt int, l *Linger) (err error) {
+	sys := sysLinger{Onoff: uint16(l.Onoff), Linger: uint16(l.Linger)}
+	return Setsockopt(fd, int32(level), int32(opt), (*byte)(unsafe.Pointer(&sys)), int32(unsafe.Sizeof(sys)))
+}
+
+func SetsockoptInet4Addr(fd Handle, level, opt int, value [4]byte) (err error) {
+	return Setsockopt(fd, int32(level), int32(opt), (*byte)(unsafe.Pointer(&value[0])), 4)
+}
+func SetsockoptIPMreq(fd Handle, level, opt int, mreq *IPMreq) (err error) {
+	return Setsockopt(fd, int32(level), int32(opt), (*byte)(unsafe.Pointer(mreq)), int32(unsafe.Sizeof(*mreq)))
+}
+func SetsockoptIPv6Mreq(fd Handle, level, opt int, mreq *IPv6Mreq) (err error) {
+	return syscall.EWINDOWS
+}
+
+func Getpid() (pid int) { return int(GetCurrentProcessId()) }
+
+func FindFirstFile(name *uint16, data *Win32finddata) (handle Handle, err error) {
+	// NOTE(rsc): The Win32finddata struct is wrong for the system call:
+	// the two paths are each one uint16 short. Use the correct struct,
+	// a win32finddata1, and then copy the results out.
+	// There is no loss of expressivity here, because the final
+	// uint16, if it is used, is supposed to be a NUL, and Go doesn't need that.
+	// For Go 1.1, we might avoid the allocation of win32finddata1 here
+	// by adding a final Bug [2]uint16 field to the struct and then
+	// adjusting the fields in the result directly.
+	var data1 win32finddata1
+	handle, err = findFirstFile1(name, &data1)
+	if err == nil {
+		copyFindData(data, &data1)
+	}
+	return
+}
+
+func FindNextFile(handle Handle, data *Win32finddata) (err error) {
+	var data1 win32finddata1
+	err = findNextFile1(handle, &data1)
+	if err == nil {
+		copyFindData(data, &data1)
+	}
+	return
+}
+
+func getProcessEntry(pid int) (*ProcessEntry32, error) {
+	snapshot, err := CreateToolhelp32Snapshot(TH32CS_SNAPPROCESS, 0)
+	if err != nil {
+		return nil, err
+	}
+	defer CloseHandle(snapshot)
+	var procEntry ProcessEntry32
+	procEntry.Size = uint32(unsafe.Sizeof(procEntry))
+	if err = Process32First(snapshot, &procEntry); err != nil {
+		return nil, err
+	}
+	for {
+		if procEntry.ProcessID == uint32(pid) {
+			return &procEntry, nil
+		}
+		err = Process32Next(snapshot, &procEntry)
+		if err != nil {
+			return nil, err
+		}
+	}
+}
+
+func Getppid() (ppid int) {
+	pe, err := getProcessEntry(Getpid())
+	if err != nil {
+		return -1
+	}
+	return int(pe.ParentProcessID)
+}
+
+// TODO(brainman): fix all needed for os
+func Fchdir(fd Handle) (err error)             { return syscall.EWINDOWS }
+func Link(oldpath, newpath string) (err error) { return syscall.EWINDOWS }
+func Symlink(path, link string) (err error)    { return syscall.EWINDOWS }
+
+func Fchmod(fd Handle, mode uint32) (err error)        { return syscall.EWINDOWS }
+func Chown(path string, uid int, gid int) (err error)  { return syscall.EWINDOWS }
+func Lchown(path string, uid int, gid int) (err error) { return syscall.EWINDOWS }
+func Fchown(fd Handle, uid int, gid int) (err error)   { return syscall.EWINDOWS }
+
+func Getuid() (uid int)                  { return -1 }
+func Geteuid() (euid int)                { return -1 }
+func Getgid() (gid int)                  { return -1 }
+func Getegid() (egid int)                { return -1 }
+func Getgroups() (gids []int, err error) { return nil, syscall.EWINDOWS }
+
+type Signal int
+
+func (s Signal) Signal() {}
+
+func (s Signal) String() string {
+	if 0 <= s && int(s) < len(signals) {
+		str := signals[s]
+		if str != "" {
+			return str
+		}
+	}
+	return "signal " + itoa(int(s))
+}
+
+func LoadCreateSymbolicLink() error {
+	return procCreateSymbolicLinkW.Find()
+}
+
+// Readlink returns the destination of the named symbolic link.
+func Readlink(path string, buf []byte) (n int, err error) {
+	fd, err := CreateFile(StringToUTF16Ptr(path), GENERIC_READ, 0, nil, OPEN_EXISTING,
+		FILE_FLAG_OPEN_REPARSE_POINT|FILE_FLAG_BACKUP_SEMANTICS, 0)
+	if err != nil {
+		return -1, err
+	}
+	defer CloseHandle(fd)
+
+	rdbbuf := make([]byte, MAXIMUM_REPARSE_DATA_BUFFER_SIZE)
+	var bytesReturned uint32
+	err = DeviceIoControl(fd, FSCTL_GET_REPARSE_POINT, nil, 0, &rdbbuf[0], uint32(len(rdbbuf)), &bytesReturned, nil)
+	if err != nil {
+		return -1, err
+	}
+
+	rdb := (*reparseDataBuffer)(unsafe.Pointer(&rdbbuf[0]))
+	var s string
+	switch rdb.ReparseTag {
+	case IO_REPARSE_TAG_SYMLINK:
+		data := (*symbolicLinkReparseBuffer)(unsafe.Pointer(&rdb.reparseBuffer))
+		p := (*[0xffff]uint16)(unsafe.Pointer(&data.PathBuffer[0]))
+		s = UTF16ToString(p[data.PrintNameOffset/2 : (data.PrintNameLength-data.PrintNameOffset)/2])
+	case IO_REPARSE_TAG_MOUNT_POINT:
+		data := (*mountPointReparseBuffer)(unsafe.Pointer(&rdb.reparseBuffer))
+		p := (*[0xffff]uint16)(unsafe.Pointer(&data.PathBuffer[0]))
+		s = UTF16ToString(p[data.PrintNameOffset/2 : (data.PrintNameLength-data.PrintNameOffset)/2])
+	default:
+		// the path is not a symlink or junction but another type of reparse
+		// point
+		return -1, syscall.ENOENT
+	}
+	n = copy(buf, []byte(s))
+
+	return n, nil
+}
+
+// GUIDFromString parses a string in the form of
+// "{XXXXXX-XXXX-XXXX-XXXX-XXXXXXXXXXXX}" into a GUID.
+func GUIDFromString(str string) (GUID, error) {
+	guid := GUID{}
+	str16, err := syscall.UTF16PtrFromString(str)
+	if err != nil {
+		return guid, err
+	}
+	err = clsidFromString(str16, &guid)
+	if err != nil {
+		return guid, err
+	}
+	return guid, nil
+}
+
+// GenerateGUID creates a new random GUID.
+func GenerateGUID() (GUID, error) {
+	guid := GUID{}
+	err := coCreateGuid(&guid)
+	if err != nil {
+		return guid, err
+	}
+	return guid, nil
+}
+
+// String returns the canonical string form of the GUID,
+// in the form of "{XXXXXX-XXXX-XXXX-XXXX-XXXXXXXXXXXX}".
+func (guid GUID) String() string {
+	var str [100]uint16
+	chars := stringFromGUID2(&guid, &str[0], int32(len(str)))
+	if chars <= 1 {
+		return ""
+	}
+	return string(utf16.Decode(str[:chars-1]))
+}
+
+// KnownFolderPath returns a well-known folder path for the current user, specified by one of
+// the FOLDERID_ constants, and chosen and optionally created based on a KF_ flag.
+func KnownFolderPath(folderID *KNOWNFOLDERID, flags uint32) (string, error) {
+	return Token(0).KnownFolderPath(folderID, flags)
+}
+
+// KnownFolderPath returns a well-known folder path for the user token, specified by one of
+// the FOLDERID_ constants, and chosen and optionally created based on a KF_ flag.
+func (t Token) KnownFolderPath(folderID *KNOWNFOLDERID, flags uint32) (string, error) {
+	var p *uint16
+	err := shGetKnownFolderPath(folderID, flags, t, &p)
+	if err != nil {
+		return "", err
+	}
+	defer CoTaskMemFree(unsafe.Pointer(p))
+	return UTF16ToString((*[(1 << 30) - 1]uint16)(unsafe.Pointer(p))[:]), nil
+}
+
+// RtlGetVersion returns the version of the underlying operating system, ignoring
+// manifest semantics but is affected by the application compatibility layer.
+func RtlGetVersion() *OsVersionInfoEx {
+	info := &OsVersionInfoEx{}
+	info.osVersionInfoSize = uint32(unsafe.Sizeof(*info))
+	// According to documentation, this function always succeeds.
+	// The function doesn't even check the validity of the
+	// osVersionInfoSize member. Disassembling ntdll.dll indicates
+	// that the documentation is indeed correct about that.
+	_ = rtlGetVersion(info)
+	return info
+}
+
+// RtlGetNtVersionNumbers returns the version of the underlying operating system,
+// ignoring manifest semantics and the application compatibility layer.
+func RtlGetNtVersionNumbers() (majorVersion, minorVersion, buildNumber uint32) {
+	rtlGetNtVersionNumbers(&majorVersion, &minorVersion, &buildNumber)
+	buildNumber &= 0xffff
+	return
+}
diff --git a/vendor/golang.org/x/sys/windows/types_windows.go b/vendor/golang.org/x/sys/windows/types_windows.go
new file mode 100644
index 0000000..a548234
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/types_windows.go
@@ -0,0 +1,1737 @@
+// Copyright 2011 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package windows
+
+import (
+	"net"
+	"syscall"
+	"unsafe"
+)
+
+const (
+	// Invented values to support what package os expects.
+	O_RDONLY   = 0x00000
+	O_WRONLY   = 0x00001
+	O_RDWR     = 0x00002
+	O_CREAT    = 0x00040
+	O_EXCL     = 0x00080
+	O_NOCTTY   = 0x00100
+	O_TRUNC    = 0x00200
+	O_NONBLOCK = 0x00800
+	O_APPEND   = 0x00400
+	O_SYNC     = 0x01000
+	O_ASYNC    = 0x02000
+	O_CLOEXEC  = 0x80000
+)
+
+const (
+	// More invented values for signals
+	SIGHUP  = Signal(0x1)
+	SIGINT  = Signal(0x2)
+	SIGQUIT = Signal(0x3)
+	SIGILL  = Signal(0x4)
+	SIGTRAP = Signal(0x5)
+	SIGABRT = Signal(0x6)
+	SIGBUS  = Signal(0x7)
+	SIGFPE  = Signal(0x8)
+	SIGKILL = Signal(0x9)
+	SIGSEGV = Signal(0xb)
+	SIGPIPE = Signal(0xd)
+	SIGALRM = Signal(0xe)
+	SIGTERM = Signal(0xf)
+)
+
+var signals = [...]string{
+	1:  "hangup",
+	2:  "interrupt",
+	3:  "quit",
+	4:  "illegal instruction",
+	5:  "trace/breakpoint trap",
+	6:  "aborted",
+	7:  "bus error",
+	8:  "floating point exception",
+	9:  "killed",
+	10: "user defined signal 1",
+	11: "segmentation fault",
+	12: "user defined signal 2",
+	13: "broken pipe",
+	14: "alarm clock",
+	15: "terminated",
+}
+
+const (
+	FILE_LIST_DIRECTORY   = 0x00000001
+	FILE_APPEND_DATA      = 0x00000004
+	FILE_WRITE_ATTRIBUTES = 0x00000100
+
+	FILE_SHARE_READ   = 0x00000001
+	FILE_SHARE_WRITE  = 0x00000002
+	FILE_SHARE_DELETE = 0x00000004
+
+	FILE_ATTRIBUTE_READONLY              = 0x00000001
+	FILE_ATTRIBUTE_HIDDEN                = 0x00000002
+	FILE_ATTRIBUTE_SYSTEM                = 0x00000004
+	FILE_ATTRIBUTE_DIRECTORY             = 0x00000010
+	FILE_ATTRIBUTE_ARCHIVE               = 0x00000020
+	FILE_ATTRIBUTE_DEVICE                = 0x00000040
+	FILE_ATTRIBUTE_NORMAL                = 0x00000080
+	FILE_ATTRIBUTE_TEMPORARY             = 0x00000100
+	FILE_ATTRIBUTE_SPARSE_FILE           = 0x00000200
+	FILE_ATTRIBUTE_REPARSE_POINT         = 0x00000400
+	FILE_ATTRIBUTE_COMPRESSED            = 0x00000800
+	FILE_ATTRIBUTE_OFFLINE               = 0x00001000
+	FILE_ATTRIBUTE_NOT_CONTENT_INDEXED   = 0x00002000
+	FILE_ATTRIBUTE_ENCRYPTED             = 0x00004000
+	FILE_ATTRIBUTE_INTEGRITY_STREAM      = 0x00008000
+	FILE_ATTRIBUTE_VIRTUAL               = 0x00010000
+	FILE_ATTRIBUTE_NO_SCRUB_DATA         = 0x00020000
+	FILE_ATTRIBUTE_RECALL_ON_OPEN        = 0x00040000
+	FILE_ATTRIBUTE_RECALL_ON_DATA_ACCESS = 0x00400000
+
+	INVALID_FILE_ATTRIBUTES = 0xffffffff
+
+	CREATE_NEW        = 1
+	CREATE_ALWAYS     = 2
+	OPEN_EXISTING     = 3
+	OPEN_ALWAYS       = 4
+	TRUNCATE_EXISTING = 5
+
+	FILE_FLAG_OPEN_REQUIRING_OPLOCK = 0x00040000
+	FILE_FLAG_FIRST_PIPE_INSTANCE   = 0x00080000
+	FILE_FLAG_OPEN_NO_RECALL        = 0x00100000
+	FILE_FLAG_OPEN_REPARSE_POINT    = 0x00200000
+	FILE_FLAG_SESSION_AWARE         = 0x00800000
+	FILE_FLAG_POSIX_SEMANTICS       = 0x01000000
+	FILE_FLAG_BACKUP_SEMANTICS      = 0x02000000
+	FILE_FLAG_DELETE_ON_CLOSE       = 0x04000000
+	FILE_FLAG_SEQUENTIAL_SCAN       = 0x08000000
+	FILE_FLAG_RANDOM_ACCESS         = 0x10000000
+	FILE_FLAG_NO_BUFFERING          = 0x20000000
+	FILE_FLAG_OVERLAPPED            = 0x40000000
+	FILE_FLAG_WRITE_THROUGH         = 0x80000000
+
+	HANDLE_FLAG_INHERIT    = 0x00000001
+	STARTF_USESTDHANDLES   = 0x00000100
+	STARTF_USESHOWWINDOW   = 0x00000001
+	DUPLICATE_CLOSE_SOURCE = 0x00000001
+	DUPLICATE_SAME_ACCESS  = 0x00000002
+
+	STD_INPUT_HANDLE  = -10 & (1<<32 - 1)
+	STD_OUTPUT_HANDLE = -11 & (1<<32 - 1)
+	STD_ERROR_HANDLE  = -12 & (1<<32 - 1)
+
+	FILE_BEGIN   = 0
+	FILE_CURRENT = 1
+	FILE_END     = 2
+
+	LANG_ENGLISH       = 0x09
+	SUBLANG_ENGLISH_US = 0x01
+
+	FORMAT_MESSAGE_ALLOCATE_BUFFER = 256
+	FORMAT_MESSAGE_IGNORE_INSERTS  = 512
+	FORMAT_MESSAGE_FROM_STRING     = 1024
+	FORMAT_MESSAGE_FROM_HMODULE    = 2048
+	FORMAT_MESSAGE_FROM_SYSTEM     = 4096
+	FORMAT_MESSAGE_ARGUMENT_ARRAY  = 8192
+	FORMAT_MESSAGE_MAX_WIDTH_MASK  = 255
+
+	MAX_PATH      = 260
+	MAX_LONG_PATH = 32768
+
+	MAX_COMPUTERNAME_LENGTH = 15
+
+	TIME_ZONE_ID_UNKNOWN  = 0
+	TIME_ZONE_ID_STANDARD = 1
+
+	TIME_ZONE_ID_DAYLIGHT = 2
+	IGNORE                = 0
+	INFINITE              = 0xffffffff
+
+	WAIT_ABANDONED = 0x00000080
+	WAIT_OBJECT_0  = 0x00000000
+	WAIT_FAILED    = 0xFFFFFFFF
+
+	// Access rights for process.
+	PROCESS_CREATE_PROCESS            = 0x0080
+	PROCESS_CREATE_THREAD             = 0x0002
+	PROCESS_DUP_HANDLE                = 0x0040
+	PROCESS_QUERY_INFORMATION         = 0x0400
+	PROCESS_QUERY_LIMITED_INFORMATION = 0x1000
+	PROCESS_SET_INFORMATION           = 0x0200
+	PROCESS_SET_QUOTA                 = 0x0100
+	PROCESS_SUSPEND_RESUME            = 0x0800
+	PROCESS_TERMINATE                 = 0x0001
+	PROCESS_VM_OPERATION              = 0x0008
+	PROCESS_VM_READ                   = 0x0010
+	PROCESS_VM_WRITE                  = 0x0020
+
+	// Access rights for thread.
+	THREAD_DIRECT_IMPERSONATION      = 0x0200
+	THREAD_GET_CONTEXT               = 0x0008
+	THREAD_IMPERSONATE               = 0x0100
+	THREAD_QUERY_INFORMATION         = 0x0040
+	THREAD_QUERY_LIMITED_INFORMATION = 0x0800
+	THREAD_SET_CONTEXT               = 0x0010
+	THREAD_SET_INFORMATION           = 0x0020
+	THREAD_SET_LIMITED_INFORMATION   = 0x0400
+	THREAD_SET_THREAD_TOKEN          = 0x0080
+	THREAD_SUSPEND_RESUME            = 0x0002
+	THREAD_TERMINATE                 = 0x0001
+
+	FILE_MAP_COPY    = 0x01
+	FILE_MAP_WRITE   = 0x02
+	FILE_MAP_READ    = 0x04
+	FILE_MAP_EXECUTE = 0x20
+
+	CTRL_C_EVENT        = 0
+	CTRL_BREAK_EVENT    = 1
+	CTRL_CLOSE_EVENT    = 2
+	CTRL_LOGOFF_EVENT   = 5
+	CTRL_SHUTDOWN_EVENT = 6
+
+	// Windows reserves errors >= 1<<29 for application use.
+	APPLICATION_ERROR = 1 << 29
+)
+
+const (
+	// Process creation flags.
+	CREATE_BREAKAWAY_FROM_JOB        = 0x01000000
+	CREATE_DEFAULT_ERROR_MODE        = 0x04000000
+	CREATE_NEW_CONSOLE               = 0x00000010
+	CREATE_NEW_PROCESS_GROUP         = 0x00000200
+	CREATE_NO_WINDOW                 = 0x08000000
+	CREATE_PROTECTED_PROCESS         = 0x00040000
+	CREATE_PRESERVE_CODE_AUTHZ_LEVEL = 0x02000000
+	CREATE_SEPARATE_WOW_VDM          = 0x00000800
+	CREATE_SHARED_WOW_VDM            = 0x00001000
+	CREATE_SUSPENDED                 = 0x00000004
+	CREATE_UNICODE_ENVIRONMENT       = 0x00000400
+	DEBUG_ONLY_THIS_PROCESS          = 0x00000002
+	DEBUG_PROCESS                    = 0x00000001
+	DETACHED_PROCESS                 = 0x00000008
+	EXTENDED_STARTUPINFO_PRESENT     = 0x00080000
+	INHERIT_PARENT_AFFINITY          = 0x00010000
+)
+
+const (
+	// flags for CreateToolhelp32Snapshot
+	TH32CS_SNAPHEAPLIST = 0x01
+	TH32CS_SNAPPROCESS  = 0x02
+	TH32CS_SNAPTHREAD   = 0x04
+	TH32CS_SNAPMODULE   = 0x08
+	TH32CS_SNAPMODULE32 = 0x10
+	TH32CS_SNAPALL      = TH32CS_SNAPHEAPLIST | TH32CS_SNAPMODULE | TH32CS_SNAPPROCESS | TH32CS_SNAPTHREAD
+	TH32CS_INHERIT      = 0x80000000
+)
+
+const (
+	// filters for ReadDirectoryChangesW
+	FILE_NOTIFY_CHANGE_FILE_NAME   = 0x001
+	FILE_NOTIFY_CHANGE_DIR_NAME    = 0x002
+	FILE_NOTIFY_CHANGE_ATTRIBUTES  = 0x004
+	FILE_NOTIFY_CHANGE_SIZE        = 0x008
+	FILE_NOTIFY_CHANGE_LAST_WRITE  = 0x010
+	FILE_NOTIFY_CHANGE_LAST_ACCESS = 0x020
+	FILE_NOTIFY_CHANGE_CREATION    = 0x040
+	FILE_NOTIFY_CHANGE_SECURITY    = 0x100
+)
+
+const (
+	// do not reorder
+	FILE_ACTION_ADDED = iota + 1
+	FILE_ACTION_REMOVED
+	FILE_ACTION_MODIFIED
+	FILE_ACTION_RENAMED_OLD_NAME
+	FILE_ACTION_RENAMED_NEW_NAME
+)
+
+const (
+	// wincrypt.h
+	PROV_RSA_FULL                    = 1
+	PROV_RSA_SIG                     = 2
+	PROV_DSS                         = 3
+	PROV_FORTEZZA                    = 4
+	PROV_MS_EXCHANGE                 = 5
+	PROV_SSL                         = 6
+	PROV_RSA_SCHANNEL                = 12
+	PROV_DSS_DH                      = 13
+	PROV_EC_ECDSA_SIG                = 14
+	PROV_EC_ECNRA_SIG                = 15
+	PROV_EC_ECDSA_FULL               = 16
+	PROV_EC_ECNRA_FULL               = 17
+	PROV_DH_SCHANNEL                 = 18
+	PROV_SPYRUS_LYNKS                = 20
+	PROV_RNG                         = 21
+	PROV_INTEL_SEC                   = 22
+	PROV_REPLACE_OWF                 = 23
+	PROV_RSA_AES                     = 24
+	CRYPT_VERIFYCONTEXT              = 0xF0000000
+	CRYPT_NEWKEYSET                  = 0x00000008
+	CRYPT_DELETEKEYSET               = 0x00000010
+	CRYPT_MACHINE_KEYSET             = 0x00000020
+	CRYPT_SILENT                     = 0x00000040
+	CRYPT_DEFAULT_CONTAINER_OPTIONAL = 0x00000080
+
+	USAGE_MATCH_TYPE_AND = 0
+	USAGE_MATCH_TYPE_OR  = 1
+
+	/* msgAndCertEncodingType values for CertOpenStore function */
+	X509_ASN_ENCODING   = 0x00000001
+	PKCS_7_ASN_ENCODING = 0x00010000
+
+	/* storeProvider values for CertOpenStore function */
+	CERT_STORE_PROV_MSG               = 1
+	CERT_STORE_PROV_MEMORY            = 2
+	CERT_STORE_PROV_FILE              = 3
+	CERT_STORE_PROV_REG               = 4
+	CERT_STORE_PROV_PKCS7             = 5
+	CERT_STORE_PROV_SERIALIZED        = 6
+	CERT_STORE_PROV_FILENAME_A        = 7
+	CERT_STORE_PROV_FILENAME_W        = 8
+	CERT_STORE_PROV_FILENAME          = CERT_STORE_PROV_FILENAME_W
+	CERT_STORE_PROV_SYSTEM_A          = 9
+	CERT_STORE_PROV_SYSTEM_W          = 10
+	CERT_STORE_PROV_SYSTEM            = CERT_STORE_PROV_SYSTEM_W
+	CERT_STORE_PROV_COLLECTION        = 11
+	CERT_STORE_PROV_SYSTEM_REGISTRY_A = 12
+	CERT_STORE_PROV_SYSTEM_REGISTRY_W = 13
+	CERT_STORE_PROV_SYSTEM_REGISTRY   = CERT_STORE_PROV_SYSTEM_REGISTRY_W
+	CERT_STORE_PROV_PHYSICAL_W        = 14
+	CERT_STORE_PROV_PHYSICAL          = CERT_STORE_PROV_PHYSICAL_W
+	CERT_STORE_PROV_SMART_CARD_W      = 15
+	CERT_STORE_PROV_SMART_CARD        = CERT_STORE_PROV_SMART_CARD_W
+	CERT_STORE_PROV_LDAP_W            = 16
+	CERT_STORE_PROV_LDAP              = CERT_STORE_PROV_LDAP_W
+	CERT_STORE_PROV_PKCS12            = 17
+
+	/* store characteristics (low WORD of flag) for CertOpenStore function */
+	CERT_STORE_NO_CRYPT_RELEASE_FLAG            = 0x00000001
+	CERT_STORE_SET_LOCALIZED_NAME_FLAG          = 0x00000002
+	CERT_STORE_DEFER_CLOSE_UNTIL_LAST_FREE_FLAG = 0x00000004
+	CERT_STORE_DELETE_FLAG                      = 0x00000010
+	CERT_STORE_UNSAFE_PHYSICAL_FLAG             = 0x00000020
+	CERT_STORE_SHARE_STORE_FLAG                 = 0x00000040
+	CERT_STORE_SHARE_CONTEXT_FLAG               = 0x00000080
+	CERT_STORE_MANIFOLD_FLAG                    = 0x00000100
+	CERT_STORE_ENUM_ARCHIVED_FLAG               = 0x00000200
+	CERT_STORE_UPDATE_KEYID_FLAG                = 0x00000400
+	CERT_STORE_BACKUP_RESTORE_FLAG              = 0x00000800
+	CERT_STORE_MAXIMUM_ALLOWED_FLAG             = 0x00001000
+	CERT_STORE_CREATE_NEW_FLAG                  = 0x00002000
+	CERT_STORE_OPEN_EXISTING_FLAG               = 0x00004000
+	CERT_STORE_READONLY_FLAG                    = 0x00008000
+
+	/* store locations (high WORD of flag) for CertOpenStore function */
+	CERT_SYSTEM_STORE_CURRENT_USER               = 0x00010000
+	CERT_SYSTEM_STORE_LOCAL_MACHINE              = 0x00020000
+	CERT_SYSTEM_STORE_CURRENT_SERVICE            = 0x00040000
+	CERT_SYSTEM_STORE_SERVICES                   = 0x00050000
+	CERT_SYSTEM_STORE_USERS                      = 0x00060000
+	CERT_SYSTEM_STORE_CURRENT_USER_GROUP_POLICY  = 0x00070000
+	CERT_SYSTEM_STORE_LOCAL_MACHINE_GROUP_POLICY = 0x00080000
+	CERT_SYSTEM_STORE_LOCAL_MACHINE_ENTERPRISE   = 0x00090000
+	CERT_SYSTEM_STORE_UNPROTECTED_FLAG           = 0x40000000
+	CERT_SYSTEM_STORE_RELOCATE_FLAG              = 0x80000000
+
+	/* Miscellaneous high-WORD flags for CertOpenStore function */
+	CERT_REGISTRY_STORE_REMOTE_FLAG      = 0x00010000
+	CERT_REGISTRY_STORE_SERIALIZED_FLAG  = 0x00020000
+	CERT_REGISTRY_STORE_ROAMING_FLAG     = 0x00040000
+	CERT_REGISTRY_STORE_MY_IE_DIRTY_FLAG = 0x00080000
+	CERT_REGISTRY_STORE_LM_GPT_FLAG      = 0x01000000
+	CERT_REGISTRY_STORE_CLIENT_GPT_FLAG  = 0x80000000
+	CERT_FILE_STORE_COMMIT_ENABLE_FLAG   = 0x00010000
+	CERT_LDAP_STORE_SIGN_FLAG            = 0x00010000
+	CERT_LDAP_STORE_AREC_EXCLUSIVE_FLAG  = 0x00020000
+	CERT_LDAP_STORE_OPENED_FLAG          = 0x00040000
+	CERT_LDAP_STORE_UNBIND_FLAG          = 0x00080000
+
+	/* addDisposition values for CertAddCertificateContextToStore function */
+	CERT_STORE_ADD_NEW                                 = 1
+	CERT_STORE_ADD_USE_EXISTING                        = 2
+	CERT_STORE_ADD_REPLACE_EXISTING                    = 3
+	CERT_STORE_ADD_ALWAYS                              = 4
+	CERT_STORE_ADD_REPLACE_EXISTING_INHERIT_PROPERTIES = 5
+	CERT_STORE_ADD_NEWER                               = 6
+	CERT_STORE_ADD_NEWER_INHERIT_PROPERTIES            = 7
+
+	/* ErrorStatus values for CertTrustStatus struct */
+	CERT_TRUST_NO_ERROR                          = 0x00000000
+	CERT_TRUST_IS_NOT_TIME_VALID                 = 0x00000001
+	CERT_TRUST_IS_REVOKED                        = 0x00000004
+	CERT_TRUST_IS_NOT_SIGNATURE_VALID            = 0x00000008
+	CERT_TRUST_IS_NOT_VALID_FOR_USAGE            = 0x00000010
+	CERT_TRUST_IS_UNTRUSTED_ROOT                 = 0x00000020
+	CERT_TRUST_REVOCATION_STATUS_UNKNOWN         = 0x00000040
+	CERT_TRUST_IS_CYCLIC                         = 0x00000080
+	CERT_TRUST_INVALID_EXTENSION                 = 0x00000100
+	CERT_TRUST_INVALID_POLICY_CONSTRAINTS        = 0x00000200
+	CERT_TRUST_INVALID_BASIC_CONSTRAINTS         = 0x00000400
+	CERT_TRUST_INVALID_NAME_CONSTRAINTS          = 0x00000800
+	CERT_TRUST_HAS_NOT_SUPPORTED_NAME_CONSTRAINT = 0x00001000
+	CERT_TRUST_HAS_NOT_DEFINED_NAME_CONSTRAINT   = 0x00002000
+	CERT_TRUST_HAS_NOT_PERMITTED_NAME_CONSTRAINT = 0x00004000
+	CERT_TRUST_HAS_EXCLUDED_NAME_CONSTRAINT      = 0x00008000
+	CERT_TRUST_IS_PARTIAL_CHAIN                  = 0x00010000
+	CERT_TRUST_CTL_IS_NOT_TIME_VALID             = 0x00020000
+	CERT_TRUST_CTL_IS_NOT_SIGNATURE_VALID        = 0x00040000
+	CERT_TRUST_CTL_IS_NOT_VALID_FOR_USAGE        = 0x00080000
+	CERT_TRUST_HAS_WEAK_SIGNATURE                = 0x00100000
+	CERT_TRUST_IS_OFFLINE_REVOCATION             = 0x01000000
+	CERT_TRUST_NO_ISSUANCE_CHAIN_POLICY          = 0x02000000
+	CERT_TRUST_IS_EXPLICIT_DISTRUST              = 0x04000000
+	CERT_TRUST_HAS_NOT_SUPPORTED_CRITICAL_EXT    = 0x08000000
+
+	/* InfoStatus values for CertTrustStatus struct */
+	CERT_TRUST_HAS_EXACT_MATCH_ISSUER        = 0x00000001
+	CERT_TRUST_HAS_KEY_MATCH_ISSUER          = 0x00000002
+	CERT_TRUST_HAS_NAME_MATCH_ISSUER         = 0x00000004
+	CERT_TRUST_IS_SELF_SIGNED                = 0x00000008
+	CERT_TRUST_HAS_PREFERRED_ISSUER          = 0x00000100
+	CERT_TRUST_HAS_ISSUANCE_CHAIN_POLICY     = 0x00000400
+	CERT_TRUST_HAS_VALID_NAME_CONSTRAINTS    = 0x00000400
+	CERT_TRUST_IS_PEER_TRUSTED               = 0x00000800
+	CERT_TRUST_HAS_CRL_VALIDITY_EXTENDED     = 0x00001000
+	CERT_TRUST_IS_FROM_EXCLUSIVE_TRUST_STORE = 0x00002000
+	CERT_TRUST_IS_CA_TRUSTED                 = 0x00004000
+	CERT_TRUST_IS_COMPLEX_CHAIN              = 0x00010000
+
+	/* policyOID values for CertVerifyCertificateChainPolicy function */
+	CERT_CHAIN_POLICY_BASE              = 1
+	CERT_CHAIN_POLICY_AUTHENTICODE      = 2
+	CERT_CHAIN_POLICY_AUTHENTICODE_TS   = 3
+	CERT_CHAIN_POLICY_SSL               = 4
+	CERT_CHAIN_POLICY_BASIC_CONSTRAINTS = 5
+	CERT_CHAIN_POLICY_NT_AUTH           = 6
+	CERT_CHAIN_POLICY_MICROSOFT_ROOT    = 7
+	CERT_CHAIN_POLICY_EV                = 8
+	CERT_CHAIN_POLICY_SSL_F12           = 9
+
+	/* AuthType values for SSLExtraCertChainPolicyPara struct */
+	AUTHTYPE_CLIENT = 1
+	AUTHTYPE_SERVER = 2
+
+	/* Checks values for SSLExtraCertChainPolicyPara struct */
+	SECURITY_FLAG_IGNORE_REVOCATION        = 0x00000080
+	SECURITY_FLAG_IGNORE_UNKNOWN_CA        = 0x00000100
+	SECURITY_FLAG_IGNORE_WRONG_USAGE       = 0x00000200
+	SECURITY_FLAG_IGNORE_CERT_CN_INVALID   = 0x00001000
+	SECURITY_FLAG_IGNORE_CERT_DATE_INVALID = 0x00002000
+)
+
+const (
+	// flags for SetErrorMode
+	SEM_FAILCRITICALERRORS     = 0x0001
+	SEM_NOALIGNMENTFAULTEXCEPT = 0x0004
+	SEM_NOGPFAULTERRORBOX      = 0x0002
+	SEM_NOOPENFILEERRORBOX     = 0x8000
+)
+
+const (
+	// Priority class.
+	ABOVE_NORMAL_PRIORITY_CLASS   = 0x00008000
+	BELOW_NORMAL_PRIORITY_CLASS   = 0x00004000
+	HIGH_PRIORITY_CLASS           = 0x00000080
+	IDLE_PRIORITY_CLASS           = 0x00000040
+	NORMAL_PRIORITY_CLASS         = 0x00000020
+	PROCESS_MODE_BACKGROUND_BEGIN = 0x00100000
+	PROCESS_MODE_BACKGROUND_END   = 0x00200000
+	REALTIME_PRIORITY_CLASS       = 0x00000100
+)
+
+var (
+	OID_PKIX_KP_SERVER_AUTH = []byte("1.3.6.1.5.5.7.3.1\x00")
+	OID_SERVER_GATED_CRYPTO = []byte("1.3.6.1.4.1.311.10.3.3\x00")
+	OID_SGC_NETSCAPE        = []byte("2.16.840.1.113730.4.1\x00")
+)
+
+// Pointer represents a pointer to an arbitrary Windows type.
+//
+// Pointer-typed fields may point to one of many different types. It's
+// up to the caller to provide a pointer to the appropriate type, cast
+// to Pointer. The caller must obey the unsafe.Pointer rules while
+// doing so.
+type Pointer *struct{}
+
+// Invented values to support what package os expects.
+type Timeval struct {
+	Sec  int32
+	Usec int32
+}
+
+func (tv *Timeval) Nanoseconds() int64 {
+	return (int64(tv.Sec)*1e6 + int64(tv.Usec)) * 1e3
+}
+
+func NsecToTimeval(nsec int64) (tv Timeval) {
+	tv.Sec = int32(nsec / 1e9)
+	tv.Usec = int32(nsec % 1e9 / 1e3)
+	return
+}
+
+type Overlapped struct {
+	Internal     uintptr
+	InternalHigh uintptr
+	Offset       uint32
+	OffsetHigh   uint32
+	HEvent       Handle
+}
+
+type FileNotifyInformation struct {
+	NextEntryOffset uint32
+	Action          uint32
+	FileNameLength  uint32
+	FileName        uint16
+}
+
+type Filetime struct {
+	LowDateTime  uint32
+	HighDateTime uint32
+}
+
+// Nanoseconds returns Filetime ft in nanoseconds
+// since Epoch (00:00:00 UTC, January 1, 1970).
+func (ft *Filetime) Nanoseconds() int64 {
+	// 100-nanosecond intervals since January 1, 1601
+	nsec := int64(ft.HighDateTime)<<32 + int64(ft.LowDateTime)
+	// change starting time to the Epoch (00:00:00 UTC, January 1, 1970)
+	nsec -= 116444736000000000
+	// convert into nanoseconds
+	nsec *= 100
+	return nsec
+}
+
+func NsecToFiletime(nsec int64) (ft Filetime) {
+	// convert into 100-nanosecond
+	nsec /= 100
+	// change starting time to January 1, 1601
+	nsec += 116444736000000000
+	// split into high / low
+	ft.LowDateTime = uint32(nsec & 0xffffffff)
+	ft.HighDateTime = uint32(nsec >> 32 & 0xffffffff)
+	return ft
+}
+
+type Win32finddata struct {
+	FileAttributes    uint32
+	CreationTime      Filetime
+	LastAccessTime    Filetime
+	LastWriteTime     Filetime
+	FileSizeHigh      uint32
+	FileSizeLow       uint32
+	Reserved0         uint32
+	Reserved1         uint32
+	FileName          [MAX_PATH - 1]uint16
+	AlternateFileName [13]uint16
+}
+
+// This is the actual system call structure.
+// Win32finddata is what we committed to in Go 1.
+type win32finddata1 struct {
+	FileAttributes    uint32
+	CreationTime      Filetime
+	LastAccessTime    Filetime
+	LastWriteTime     Filetime
+	FileSizeHigh      uint32
+	FileSizeLow       uint32
+	Reserved0         uint32
+	Reserved1         uint32
+	FileName          [MAX_PATH]uint16
+	AlternateFileName [14]uint16
+}
+
+func copyFindData(dst *Win32finddata, src *win32finddata1) {
+	dst.FileAttributes = src.FileAttributes
+	dst.CreationTime = src.CreationTime
+	dst.LastAccessTime = src.LastAccessTime
+	dst.LastWriteTime = src.LastWriteTime
+	dst.FileSizeHigh = src.FileSizeHigh
+	dst.FileSizeLow = src.FileSizeLow
+	dst.Reserved0 = src.Reserved0
+	dst.Reserved1 = src.Reserved1
+
+	// The src is 1 element bigger than dst, but it must be NUL.
+	copy(dst.FileName[:], src.FileName[:])
+	copy(dst.AlternateFileName[:], src.AlternateFileName[:])
+}
+
+type ByHandleFileInformation struct {
+	FileAttributes     uint32
+	CreationTime       Filetime
+	LastAccessTime     Filetime
+	LastWriteTime      Filetime
+	VolumeSerialNumber uint32
+	FileSizeHigh       uint32
+	FileSizeLow        uint32
+	NumberOfLinks      uint32
+	FileIndexHigh      uint32
+	FileIndexLow       uint32
+}
+
+const (
+	GetFileExInfoStandard = 0
+	GetFileExMaxInfoLevel = 1
+)
+
+type Win32FileAttributeData struct {
+	FileAttributes uint32
+	CreationTime   Filetime
+	LastAccessTime Filetime
+	LastWriteTime  Filetime
+	FileSizeHigh   uint32
+	FileSizeLow    uint32
+}
+
+// ShowWindow constants
+const (
+	// winuser.h
+	SW_HIDE            = 0
+	SW_NORMAL          = 1
+	SW_SHOWNORMAL      = 1
+	SW_SHOWMINIMIZED   = 2
+	SW_SHOWMAXIMIZED   = 3
+	SW_MAXIMIZE        = 3
+	SW_SHOWNOACTIVATE  = 4
+	SW_SHOW            = 5
+	SW_MINIMIZE        = 6
+	SW_SHOWMINNOACTIVE = 7
+	SW_SHOWNA          = 8
+	SW_RESTORE         = 9
+	SW_SHOWDEFAULT     = 10
+	SW_FORCEMINIMIZE   = 11
+)
+
+type StartupInfo struct {
+	Cb            uint32
+	_             *uint16
+	Desktop       *uint16
+	Title         *uint16
+	X             uint32
+	Y             uint32
+	XSize         uint32
+	YSize         uint32
+	XCountChars   uint32
+	YCountChars   uint32
+	FillAttribute uint32
+	Flags         uint32
+	ShowWindow    uint16
+	_             uint16
+	_             *byte
+	StdInput      Handle
+	StdOutput     Handle
+	StdErr        Handle
+}
+
+type ProcessInformation struct {
+	Process   Handle
+	Thread    Handle
+	ProcessId uint32
+	ThreadId  uint32
+}
+
+type ProcessEntry32 struct {
+	Size            uint32
+	Usage           uint32
+	ProcessID       uint32
+	DefaultHeapID   uintptr
+	ModuleID        uint32
+	Threads         uint32
+	ParentProcessID uint32
+	PriClassBase    int32
+	Flags           uint32
+	ExeFile         [MAX_PATH]uint16
+}
+
+type ThreadEntry32 struct {
+	Size           uint32
+	Usage          uint32
+	ThreadID       uint32
+	OwnerProcessID uint32
+	BasePri        int32
+	DeltaPri       int32
+	Flags          uint32
+}
+
+type Systemtime struct {
+	Year         uint16
+	Month        uint16
+	DayOfWeek    uint16
+	Day          uint16
+	Hour         uint16
+	Minute       uint16
+	Second       uint16
+	Milliseconds uint16
+}
+
+type Timezoneinformation struct {
+	Bias         int32
+	StandardName [32]uint16
+	StandardDate Systemtime
+	StandardBias int32
+	DaylightName [32]uint16
+	DaylightDate Systemtime
+	DaylightBias int32
+}
+
+// Socket related.
+
+const (
+	AF_UNSPEC  = 0
+	AF_UNIX    = 1
+	AF_INET    = 2
+	AF_INET6   = 23
+	AF_NETBIOS = 17
+
+	SOCK_STREAM    = 1
+	SOCK_DGRAM     = 2
+	SOCK_RAW       = 3
+	SOCK_SEQPACKET = 5
+
+	IPPROTO_IP   = 0
+	IPPROTO_IPV6 = 0x29
+	IPPROTO_TCP  = 6
+	IPPROTO_UDP  = 17
+
+	SOL_SOCKET                = 0xffff
+	SO_REUSEADDR              = 4
+	SO_KEEPALIVE              = 8
+	SO_DONTROUTE              = 16
+	SO_BROADCAST              = 32
+	SO_LINGER                 = 128
+	SO_RCVBUF                 = 0x1002
+	SO_SNDBUF                 = 0x1001
+	SO_UPDATE_ACCEPT_CONTEXT  = 0x700b
+	SO_UPDATE_CONNECT_CONTEXT = 0x7010
+
+	IOC_OUT                            = 0x40000000
+	IOC_IN                             = 0x80000000
+	IOC_VENDOR                         = 0x18000000
+	IOC_INOUT                          = IOC_IN | IOC_OUT
+	IOC_WS2                            = 0x08000000
+	SIO_GET_EXTENSION_FUNCTION_POINTER = IOC_INOUT | IOC_WS2 | 6
+	SIO_KEEPALIVE_VALS                 = IOC_IN | IOC_VENDOR | 4
+	SIO_UDP_CONNRESET                  = IOC_IN | IOC_VENDOR | 12
+
+	// cf. http://support.microsoft.com/default.aspx?scid=kb;en-us;257460
+
+	IP_TOS             = 0x3
+	IP_TTL             = 0x4
+	IP_MULTICAST_IF    = 0x9
+	IP_MULTICAST_TTL   = 0xa
+	IP_MULTICAST_LOOP  = 0xb
+	IP_ADD_MEMBERSHIP  = 0xc
+	IP_DROP_MEMBERSHIP = 0xd
+
+	IPV6_V6ONLY         = 0x1b
+	IPV6_UNICAST_HOPS   = 0x4
+	IPV6_MULTICAST_IF   = 0x9
+	IPV6_MULTICAST_HOPS = 0xa
+	IPV6_MULTICAST_LOOP = 0xb
+	IPV6_JOIN_GROUP     = 0xc
+	IPV6_LEAVE_GROUP    = 0xd
+
+	MSG_OOB       = 0x1
+	MSG_PEEK      = 0x2
+	MSG_DONTROUTE = 0x4
+	MSG_WAITALL   = 0x8
+
+	MSG_TRUNC  = 0x0100
+	MSG_CTRUNC = 0x0200
+	MSG_BCAST  = 0x0400
+	MSG_MCAST  = 0x0800
+
+	SOMAXCONN = 0x7fffffff
+
+	TCP_NODELAY = 1
+
+	SHUT_RD   = 0
+	SHUT_WR   = 1
+	SHUT_RDWR = 2
+
+	WSADESCRIPTION_LEN = 256
+	WSASYS_STATUS_LEN  = 128
+)
+
+type WSABuf struct {
+	Len uint32
+	Buf *byte
+}
+
+type WSAMsg struct {
+	Name        *syscall.RawSockaddrAny
+	Namelen     int32
+	Buffers     *WSABuf
+	BufferCount uint32
+	Control     WSABuf
+	Flags       uint32
+}
+
+// Invented values to support what package os expects.
+const (
+	S_IFMT   = 0x1f000
+	S_IFIFO  = 0x1000
+	S_IFCHR  = 0x2000
+	S_IFDIR  = 0x4000
+	S_IFBLK  = 0x6000
+	S_IFREG  = 0x8000
+	S_IFLNK  = 0xa000
+	S_IFSOCK = 0xc000
+	S_ISUID  = 0x800
+	S_ISGID  = 0x400
+	S_ISVTX  = 0x200
+	S_IRUSR  = 0x100
+	S_IWRITE = 0x80
+	S_IWUSR  = 0x80
+	S_IXUSR  = 0x40
+)
+
+const (
+	FILE_TYPE_CHAR    = 0x0002
+	FILE_TYPE_DISK    = 0x0001
+	FILE_TYPE_PIPE    = 0x0003
+	FILE_TYPE_REMOTE  = 0x8000
+	FILE_TYPE_UNKNOWN = 0x0000
+)
+
+type Hostent struct {
+	Name     *byte
+	Aliases  **byte
+	AddrType uint16
+	Length   uint16
+	AddrList **byte
+}
+
+type Protoent struct {
+	Name    *byte
+	Aliases **byte
+	Proto   uint16
+}
+
+const (
+	DNS_TYPE_A       = 0x0001
+	DNS_TYPE_NS      = 0x0002
+	DNS_TYPE_MD      = 0x0003
+	DNS_TYPE_MF      = 0x0004
+	DNS_TYPE_CNAME   = 0x0005
+	DNS_TYPE_SOA     = 0x0006
+	DNS_TYPE_MB      = 0x0007
+	DNS_TYPE_MG      = 0x0008
+	DNS_TYPE_MR      = 0x0009
+	DNS_TYPE_NULL    = 0x000a
+	DNS_TYPE_WKS     = 0x000b
+	DNS_TYPE_PTR     = 0x000c
+	DNS_TYPE_HINFO   = 0x000d
+	DNS_TYPE_MINFO   = 0x000e
+	DNS_TYPE_MX      = 0x000f
+	DNS_TYPE_TEXT    = 0x0010
+	DNS_TYPE_RP      = 0x0011
+	DNS_TYPE_AFSDB   = 0x0012
+	DNS_TYPE_X25     = 0x0013
+	DNS_TYPE_ISDN    = 0x0014
+	DNS_TYPE_RT      = 0x0015
+	DNS_TYPE_NSAP    = 0x0016
+	DNS_TYPE_NSAPPTR = 0x0017
+	DNS_TYPE_SIG     = 0x0018
+	DNS_TYPE_KEY     = 0x0019
+	DNS_TYPE_PX      = 0x001a
+	DNS_TYPE_GPOS    = 0x001b
+	DNS_TYPE_AAAA    = 0x001c
+	DNS_TYPE_LOC     = 0x001d
+	DNS_TYPE_NXT     = 0x001e
+	DNS_TYPE_EID     = 0x001f
+	DNS_TYPE_NIMLOC  = 0x0020
+	DNS_TYPE_SRV     = 0x0021
+	DNS_TYPE_ATMA    = 0x0022
+	DNS_TYPE_NAPTR   = 0x0023
+	DNS_TYPE_KX      = 0x0024
+	DNS_TYPE_CERT    = 0x0025
+	DNS_TYPE_A6      = 0x0026
+	DNS_TYPE_DNAME   = 0x0027
+	DNS_TYPE_SINK    = 0x0028
+	DNS_TYPE_OPT     = 0x0029
+	DNS_TYPE_DS      = 0x002B
+	DNS_TYPE_RRSIG   = 0x002E
+	DNS_TYPE_NSEC    = 0x002F
+	DNS_TYPE_DNSKEY  = 0x0030
+	DNS_TYPE_DHCID   = 0x0031
+	DNS_TYPE_UINFO   = 0x0064
+	DNS_TYPE_UID     = 0x0065
+	DNS_TYPE_GID     = 0x0066
+	DNS_TYPE_UNSPEC  = 0x0067
+	DNS_TYPE_ADDRS   = 0x00f8
+	DNS_TYPE_TKEY    = 0x00f9
+	DNS_TYPE_TSIG    = 0x00fa
+	DNS_TYPE_IXFR    = 0x00fb
+	DNS_TYPE_AXFR    = 0x00fc
+	DNS_TYPE_MAILB   = 0x00fd
+	DNS_TYPE_MAILA   = 0x00fe
+	DNS_TYPE_ALL     = 0x00ff
+	DNS_TYPE_ANY     = 0x00ff
+	DNS_TYPE_WINS    = 0xff01
+	DNS_TYPE_WINSR   = 0xff02
+	DNS_TYPE_NBSTAT  = 0xff01
+)
+
+const (
+	// flags inside DNSRecord.Dw
+	DnsSectionQuestion   = 0x0000
+	DnsSectionAnswer     = 0x0001
+	DnsSectionAuthority  = 0x0002
+	DnsSectionAdditional = 0x0003
+)
+
+type DNSSRVData struct {
+	Target   *uint16
+	Priority uint16
+	Weight   uint16
+	Port     uint16
+	Pad      uint16
+}
+
+type DNSPTRData struct {
+	Host *uint16
+}
+
+type DNSMXData struct {
+	NameExchange *uint16
+	Preference   uint16
+	Pad          uint16
+}
+
+type DNSTXTData struct {
+	StringCount uint16
+	StringArray [1]*uint16
+}
+
+type DNSRecord struct {
+	Next     *DNSRecord
+	Name     *uint16
+	Type     uint16
+	Length   uint16
+	Dw       uint32
+	Ttl      uint32
+	Reserved uint32
+	Data     [40]byte
+}
+
+const (
+	TF_DISCONNECT         = 1
+	TF_REUSE_SOCKET       = 2
+	TF_WRITE_BEHIND       = 4
+	TF_USE_DEFAULT_WORKER = 0
+	TF_USE_SYSTEM_THREAD  = 16
+	TF_USE_KERNEL_APC     = 32
+)
+
+type TransmitFileBuffers struct {
+	Head       uintptr
+	HeadLength uint32
+	Tail       uintptr
+	TailLength uint32
+}
+
+const (
+	IFF_UP           = 1
+	IFF_BROADCAST    = 2
+	IFF_LOOPBACK     = 4
+	IFF_POINTTOPOINT = 8
+	IFF_MULTICAST    = 16
+)
+
+const SIO_GET_INTERFACE_LIST = 0x4004747F
+
+// TODO(mattn): SockaddrGen is union of sockaddr/sockaddr_in/sockaddr_in6_old.
+// will be fixed to change variable type as suitable.
+
+type SockaddrGen [24]byte
+
+type InterfaceInfo struct {
+	Flags            uint32
+	Address          SockaddrGen
+	BroadcastAddress SockaddrGen
+	Netmask          SockaddrGen
+}
+
+type IpAddressString struct {
+	String [16]byte
+}
+
+type IpMaskString IpAddressString
+
+type IpAddrString struct {
+	Next      *IpAddrString
+	IpAddress IpAddressString
+	IpMask    IpMaskString
+	Context   uint32
+}
+
+const MAX_ADAPTER_NAME_LENGTH = 256
+const MAX_ADAPTER_DESCRIPTION_LENGTH = 128
+const MAX_ADAPTER_ADDRESS_LENGTH = 8
+
+type IpAdapterInfo struct {
+	Next                *IpAdapterInfo
+	ComboIndex          uint32
+	AdapterName         [MAX_ADAPTER_NAME_LENGTH + 4]byte
+	Description         [MAX_ADAPTER_DESCRIPTION_LENGTH + 4]byte
+	AddressLength       uint32
+	Address             [MAX_ADAPTER_ADDRESS_LENGTH]byte
+	Index               uint32
+	Type                uint32
+	DhcpEnabled         uint32
+	CurrentIpAddress    *IpAddrString
+	IpAddressList       IpAddrString
+	GatewayList         IpAddrString
+	DhcpServer          IpAddrString
+	HaveWins            bool
+	PrimaryWinsServer   IpAddrString
+	SecondaryWinsServer IpAddrString
+	LeaseObtained       int64
+	LeaseExpires        int64
+}
+
+const MAXLEN_PHYSADDR = 8
+const MAX_INTERFACE_NAME_LEN = 256
+const MAXLEN_IFDESCR = 256
+
+type MibIfRow struct {
+	Name            [MAX_INTERFACE_NAME_LEN]uint16
+	Index           uint32
+	Type            uint32
+	Mtu             uint32
+	Speed           uint32
+	PhysAddrLen     uint32
+	PhysAddr        [MAXLEN_PHYSADDR]byte
+	AdminStatus     uint32
+	OperStatus      uint32
+	LastChange      uint32
+	InOctets        uint32
+	InUcastPkts     uint32
+	InNUcastPkts    uint32
+	InDiscards      uint32
+	InErrors        uint32
+	InUnknownProtos uint32
+	OutOctets       uint32
+	OutUcastPkts    uint32
+	OutNUcastPkts   uint32
+	OutDiscards     uint32
+	OutErrors       uint32
+	OutQLen         uint32
+	DescrLen        uint32
+	Descr           [MAXLEN_IFDESCR]byte
+}
+
+type CertInfo struct {
+	// Not implemented
+}
+
+type CertContext struct {
+	EncodingType uint32
+	EncodedCert  *byte
+	Length       uint32
+	CertInfo     *CertInfo
+	Store        Handle
+}
+
+type CertChainContext struct {
+	Size                       uint32
+	TrustStatus                CertTrustStatus
+	ChainCount                 uint32
+	Chains                     **CertSimpleChain
+	LowerQualityChainCount     uint32
+	LowerQualityChains         **CertChainContext
+	HasRevocationFreshnessTime uint32
+	RevocationFreshnessTime    uint32
+}
+
+type CertTrustListInfo struct {
+	// Not implemented
+}
+
+type CertSimpleChain struct {
+	Size                       uint32
+	TrustStatus                CertTrustStatus
+	NumElements                uint32
+	Elements                   **CertChainElement
+	TrustListInfo              *CertTrustListInfo
+	HasRevocationFreshnessTime uint32
+	RevocationFreshnessTime    uint32
+}
+
+type CertChainElement struct {
+	Size              uint32
+	CertContext       *CertContext
+	TrustStatus       CertTrustStatus
+	RevocationInfo    *CertRevocationInfo
+	IssuanceUsage     *CertEnhKeyUsage
+	ApplicationUsage  *CertEnhKeyUsage
+	ExtendedErrorInfo *uint16
+}
+
+type CertRevocationCrlInfo struct {
+	// Not implemented
+}
+
+type CertRevocationInfo struct {
+	Size             uint32
+	RevocationResult uint32
+	RevocationOid    *byte
+	OidSpecificInfo  Pointer
+	HasFreshnessTime uint32
+	FreshnessTime    uint32
+	CrlInfo          *CertRevocationCrlInfo
+}
+
+type CertTrustStatus struct {
+	ErrorStatus uint32
+	InfoStatus  uint32
+}
+
+type CertUsageMatch struct {
+	Type  uint32
+	Usage CertEnhKeyUsage
+}
+
+type CertEnhKeyUsage struct {
+	Length           uint32
+	UsageIdentifiers **byte
+}
+
+type CertChainPara struct {
+	Size                         uint32
+	RequestedUsage               CertUsageMatch
+	RequstedIssuancePolicy       CertUsageMatch
+	URLRetrievalTimeout          uint32
+	CheckRevocationFreshnessTime uint32
+	RevocationFreshnessTime      uint32
+	CacheResync                  *Filetime
+}
+
+type CertChainPolicyPara struct {
+	Size            uint32
+	Flags           uint32
+	ExtraPolicyPara Pointer
+}
+
+type SSLExtraCertChainPolicyPara struct {
+	Size       uint32
+	AuthType   uint32
+	Checks     uint32
+	ServerName *uint16
+}
+
+type CertChainPolicyStatus struct {
+	Size              uint32
+	Error             uint32
+	ChainIndex        uint32
+	ElementIndex      uint32
+	ExtraPolicyStatus Pointer
+}
+
+const (
+	// do not reorder
+	HKEY_CLASSES_ROOT = 0x80000000 + iota
+	HKEY_CURRENT_USER
+	HKEY_LOCAL_MACHINE
+	HKEY_USERS
+	HKEY_PERFORMANCE_DATA
+	HKEY_CURRENT_CONFIG
+	HKEY_DYN_DATA
+
+	KEY_QUERY_VALUE        = 1
+	KEY_SET_VALUE          = 2
+	KEY_CREATE_SUB_KEY     = 4
+	KEY_ENUMERATE_SUB_KEYS = 8
+	KEY_NOTIFY             = 16
+	KEY_CREATE_LINK        = 32
+	KEY_WRITE              = 0x20006
+	KEY_EXECUTE            = 0x20019
+	KEY_READ               = 0x20019
+	KEY_WOW64_64KEY        = 0x0100
+	KEY_WOW64_32KEY        = 0x0200
+	KEY_ALL_ACCESS         = 0xf003f
+)
+
+const (
+	// do not reorder
+	REG_NONE = iota
+	REG_SZ
+	REG_EXPAND_SZ
+	REG_BINARY
+	REG_DWORD_LITTLE_ENDIAN
+	REG_DWORD_BIG_ENDIAN
+	REG_LINK
+	REG_MULTI_SZ
+	REG_RESOURCE_LIST
+	REG_FULL_RESOURCE_DESCRIPTOR
+	REG_RESOURCE_REQUIREMENTS_LIST
+	REG_QWORD_LITTLE_ENDIAN
+	REG_DWORD = REG_DWORD_LITTLE_ENDIAN
+	REG_QWORD = REG_QWORD_LITTLE_ENDIAN
+)
+
+const (
+	EVENT_MODIFY_STATE = 0x0002
+	EVENT_ALL_ACCESS   = STANDARD_RIGHTS_REQUIRED | SYNCHRONIZE | 0x3
+
+	MUTANT_QUERY_STATE = 0x0001
+	MUTANT_ALL_ACCESS  = STANDARD_RIGHTS_REQUIRED | SYNCHRONIZE | MUTANT_QUERY_STATE
+
+	SEMAPHORE_MODIFY_STATE = 0x0002
+	SEMAPHORE_ALL_ACCESS   = STANDARD_RIGHTS_REQUIRED | SYNCHRONIZE | 0x3
+
+	TIMER_QUERY_STATE  = 0x0001
+	TIMER_MODIFY_STATE = 0x0002
+	TIMER_ALL_ACCESS   = STANDARD_RIGHTS_REQUIRED | SYNCHRONIZE | TIMER_QUERY_STATE | TIMER_MODIFY_STATE
+
+	MUTEX_MODIFY_STATE = MUTANT_QUERY_STATE
+	MUTEX_ALL_ACCESS   = MUTANT_ALL_ACCESS
+
+	CREATE_EVENT_MANUAL_RESET  = 0x1
+	CREATE_EVENT_INITIAL_SET   = 0x2
+	CREATE_MUTEX_INITIAL_OWNER = 0x1
+)
+
+type AddrinfoW struct {
+	Flags     int32
+	Family    int32
+	Socktype  int32
+	Protocol  int32
+	Addrlen   uintptr
+	Canonname *uint16
+	Addr      uintptr
+	Next      *AddrinfoW
+}
+
+const (
+	AI_PASSIVE     = 1
+	AI_CANONNAME   = 2
+	AI_NUMERICHOST = 4
+)
+
+type GUID struct {
+	Data1 uint32
+	Data2 uint16
+	Data3 uint16
+	Data4 [8]byte
+}
+
+var WSAID_CONNECTEX = GUID{
+	0x25a207b9,
+	0xddf3,
+	0x4660,
+	[8]byte{0x8e, 0xe9, 0x76, 0xe5, 0x8c, 0x74, 0x06, 0x3e},
+}
+
+var WSAID_WSASENDMSG = GUID{
+	0xa441e712,
+	0x754f,
+	0x43ca,
+	[8]byte{0x84, 0xa7, 0x0d, 0xee, 0x44, 0xcf, 0x60, 0x6d},
+}
+
+var WSAID_WSARECVMSG = GUID{
+	0xf689d7c8,
+	0x6f1f,
+	0x436b,
+	[8]byte{0x8a, 0x53, 0xe5, 0x4f, 0xe3, 0x51, 0xc3, 0x22},
+}
+
+const (
+	FILE_SKIP_COMPLETION_PORT_ON_SUCCESS = 1
+	FILE_SKIP_SET_EVENT_ON_HANDLE        = 2
+)
+
+const (
+	WSAPROTOCOL_LEN    = 255
+	MAX_PROTOCOL_CHAIN = 7
+	BASE_PROTOCOL      = 1
+	LAYERED_PROTOCOL   = 0
+
+	XP1_CONNECTIONLESS           = 0x00000001
+	XP1_GUARANTEED_DELIVERY      = 0x00000002
+	XP1_GUARANTEED_ORDER         = 0x00000004
+	XP1_MESSAGE_ORIENTED         = 0x00000008
+	XP1_PSEUDO_STREAM            = 0x00000010
+	XP1_GRACEFUL_CLOSE           = 0x00000020
+	XP1_EXPEDITED_DATA           = 0x00000040
+	XP1_CONNECT_DATA             = 0x00000080
+	XP1_DISCONNECT_DATA          = 0x00000100
+	XP1_SUPPORT_BROADCAST        = 0x00000200
+	XP1_SUPPORT_MULTIPOINT       = 0x00000400
+	XP1_MULTIPOINT_CONTROL_PLANE = 0x00000800
+	XP1_MULTIPOINT_DATA_PLANE    = 0x00001000
+	XP1_QOS_SUPPORTED            = 0x00002000
+	XP1_UNI_SEND                 = 0x00008000
+	XP1_UNI_RECV                 = 0x00010000
+	XP1_IFS_HANDLES              = 0x00020000
+	XP1_PARTIAL_MESSAGE          = 0x00040000
+	XP1_SAN_SUPPORT_SDP          = 0x00080000
+
+	PFL_MULTIPLE_PROTO_ENTRIES  = 0x00000001
+	PFL_RECOMMENDED_PROTO_ENTRY = 0x00000002
+	PFL_HIDDEN                  = 0x00000004
+	PFL_MATCHES_PROTOCOL_ZERO   = 0x00000008
+	PFL_NETWORKDIRECT_PROVIDER  = 0x00000010
+)
+
+type WSAProtocolInfo struct {
+	ServiceFlags1     uint32
+	ServiceFlags2     uint32
+	ServiceFlags3     uint32
+	ServiceFlags4     uint32
+	ProviderFlags     uint32
+	ProviderId        GUID
+	CatalogEntryId    uint32
+	ProtocolChain     WSAProtocolChain
+	Version           int32
+	AddressFamily     int32
+	MaxSockAddr       int32
+	MinSockAddr       int32
+	SocketType        int32
+	Protocol          int32
+	ProtocolMaxOffset int32
+	NetworkByteOrder  int32
+	SecurityScheme    int32
+	MessageSize       uint32
+	ProviderReserved  uint32
+	ProtocolName      [WSAPROTOCOL_LEN + 1]uint16
+}
+
+type WSAProtocolChain struct {
+	ChainLen     int32
+	ChainEntries [MAX_PROTOCOL_CHAIN]uint32
+}
+
+type TCPKeepalive struct {
+	OnOff    uint32
+	Time     uint32
+	Interval uint32
+}
+
+type symbolicLinkReparseBuffer struct {
+	SubstituteNameOffset uint16
+	SubstituteNameLength uint16
+	PrintNameOffset      uint16
+	PrintNameLength      uint16
+	Flags                uint32
+	PathBuffer           [1]uint16
+}
+
+type mountPointReparseBuffer struct {
+	SubstituteNameOffset uint16
+	SubstituteNameLength uint16
+	PrintNameOffset      uint16
+	PrintNameLength      uint16
+	PathBuffer           [1]uint16
+}
+
+type reparseDataBuffer struct {
+	ReparseTag        uint32
+	ReparseDataLength uint16
+	Reserved          uint16
+
+	// GenericReparseBuffer
+	reparseBuffer byte
+}
+
+const (
+	FSCTL_GET_REPARSE_POINT          = 0x900A8
+	MAXIMUM_REPARSE_DATA_BUFFER_SIZE = 16 * 1024
+	IO_REPARSE_TAG_MOUNT_POINT       = 0xA0000003
+	IO_REPARSE_TAG_SYMLINK           = 0xA000000C
+	SYMBOLIC_LINK_FLAG_DIRECTORY     = 0x1
+)
+
+const (
+	ComputerNameNetBIOS                   = 0
+	ComputerNameDnsHostname               = 1
+	ComputerNameDnsDomain                 = 2
+	ComputerNameDnsFullyQualified         = 3
+	ComputerNamePhysicalNetBIOS           = 4
+	ComputerNamePhysicalDnsHostname       = 5
+	ComputerNamePhysicalDnsDomain         = 6
+	ComputerNamePhysicalDnsFullyQualified = 7
+	ComputerNameMax                       = 8
+)
+
+// For MessageBox()
+const (
+	MB_OK                   = 0x00000000
+	MB_OKCANCEL             = 0x00000001
+	MB_ABORTRETRYIGNORE     = 0x00000002
+	MB_YESNOCANCEL          = 0x00000003
+	MB_YESNO                = 0x00000004
+	MB_RETRYCANCEL          = 0x00000005
+	MB_CANCELTRYCONTINUE    = 0x00000006
+	MB_ICONHAND             = 0x00000010
+	MB_ICONQUESTION         = 0x00000020
+	MB_ICONEXCLAMATION      = 0x00000030
+	MB_ICONASTERISK         = 0x00000040
+	MB_USERICON             = 0x00000080
+	MB_ICONWARNING          = MB_ICONEXCLAMATION
+	MB_ICONERROR            = MB_ICONHAND
+	MB_ICONINFORMATION      = MB_ICONASTERISK
+	MB_ICONSTOP             = MB_ICONHAND
+	MB_DEFBUTTON1           = 0x00000000
+	MB_DEFBUTTON2           = 0x00000100
+	MB_DEFBUTTON3           = 0x00000200
+	MB_DEFBUTTON4           = 0x00000300
+	MB_APPLMODAL            = 0x00000000
+	MB_SYSTEMMODAL          = 0x00001000
+	MB_TASKMODAL            = 0x00002000
+	MB_HELP                 = 0x00004000
+	MB_NOFOCUS              = 0x00008000
+	MB_SETFOREGROUND        = 0x00010000
+	MB_DEFAULT_DESKTOP_ONLY = 0x00020000
+	MB_TOPMOST              = 0x00040000
+	MB_RIGHT                = 0x00080000
+	MB_RTLREADING           = 0x00100000
+	MB_SERVICE_NOTIFICATION = 0x00200000
+)
+
+const (
+	MOVEFILE_REPLACE_EXISTING      = 0x1
+	MOVEFILE_COPY_ALLOWED          = 0x2
+	MOVEFILE_DELAY_UNTIL_REBOOT    = 0x4
+	MOVEFILE_WRITE_THROUGH         = 0x8
+	MOVEFILE_CREATE_HARDLINK       = 0x10
+	MOVEFILE_FAIL_IF_NOT_TRACKABLE = 0x20
+)
+
+const GAA_FLAG_INCLUDE_PREFIX = 0x00000010
+
+const (
+	IF_TYPE_OTHER              = 1
+	IF_TYPE_ETHERNET_CSMACD    = 6
+	IF_TYPE_ISO88025_TOKENRING = 9
+	IF_TYPE_PPP                = 23
+	IF_TYPE_SOFTWARE_LOOPBACK  = 24
+	IF_TYPE_ATM                = 37
+	IF_TYPE_IEEE80211          = 71
+	IF_TYPE_TUNNEL             = 131
+	IF_TYPE_IEEE1394           = 144
+)
+
+type SocketAddress struct {
+	Sockaddr       *syscall.RawSockaddrAny
+	SockaddrLength int32
+}
+
+// IP returns an IPv4 or IPv6 address, or nil if the underlying SocketAddress is neither.
+func (addr *SocketAddress) IP() net.IP {
+	if uintptr(addr.SockaddrLength) >= unsafe.Sizeof(RawSockaddrInet4{}) && addr.Sockaddr.Addr.Family == AF_INET {
+		return (*RawSockaddrInet4)(unsafe.Pointer(addr.Sockaddr)).Addr[:]
+	} else if uintptr(addr.SockaddrLength) >= unsafe.Sizeof(RawSockaddrInet6{}) && addr.Sockaddr.Addr.Family == AF_INET6 {
+		return (*RawSockaddrInet6)(unsafe.Pointer(addr.Sockaddr)).Addr[:]
+	}
+	return nil
+}
+
+type IpAdapterUnicastAddress struct {
+	Length             uint32
+	Flags              uint32
+	Next               *IpAdapterUnicastAddress
+	Address            SocketAddress
+	PrefixOrigin       int32
+	SuffixOrigin       int32
+	DadState           int32
+	ValidLifetime      uint32
+	PreferredLifetime  uint32
+	LeaseLifetime      uint32
+	OnLinkPrefixLength uint8
+}
+
+type IpAdapterAnycastAddress struct {
+	Length  uint32
+	Flags   uint32
+	Next    *IpAdapterAnycastAddress
+	Address SocketAddress
+}
+
+type IpAdapterMulticastAddress struct {
+	Length  uint32
+	Flags   uint32
+	Next    *IpAdapterMulticastAddress
+	Address SocketAddress
+}
+
+type IpAdapterDnsServerAdapter struct {
+	Length   uint32
+	Reserved uint32
+	Next     *IpAdapterDnsServerAdapter
+	Address  SocketAddress
+}
+
+type IpAdapterPrefix struct {
+	Length       uint32
+	Flags        uint32
+	Next         *IpAdapterPrefix
+	Address      SocketAddress
+	PrefixLength uint32
+}
+
+type IpAdapterAddresses struct {
+	Length                uint32
+	IfIndex               uint32
+	Next                  *IpAdapterAddresses
+	AdapterName           *byte
+	FirstUnicastAddress   *IpAdapterUnicastAddress
+	FirstAnycastAddress   *IpAdapterAnycastAddress
+	FirstMulticastAddress *IpAdapterMulticastAddress
+	FirstDnsServerAddress *IpAdapterDnsServerAdapter
+	DnsSuffix             *uint16
+	Description           *uint16
+	FriendlyName          *uint16
+	PhysicalAddress       [syscall.MAX_ADAPTER_ADDRESS_LENGTH]byte
+	PhysicalAddressLength uint32
+	Flags                 uint32
+	Mtu                   uint32
+	IfType                uint32
+	OperStatus            uint32
+	Ipv6IfIndex           uint32
+	ZoneIndices           [16]uint32
+	FirstPrefix           *IpAdapterPrefix
+	/* more fields might be present here. */
+}
+
+const (
+	IfOperStatusUp             = 1
+	IfOperStatusDown           = 2
+	IfOperStatusTesting        = 3
+	IfOperStatusUnknown        = 4
+	IfOperStatusDormant        = 5
+	IfOperStatusNotPresent     = 6
+	IfOperStatusLowerLayerDown = 7
+)
+
+// Console related constants used for the mode parameter to SetConsoleMode. See
+// https://docs.microsoft.com/en-us/windows/console/setconsolemode for details.
+
+const (
+	ENABLE_PROCESSED_INPUT        = 0x1
+	ENABLE_LINE_INPUT             = 0x2
+	ENABLE_ECHO_INPUT             = 0x4
+	ENABLE_WINDOW_INPUT           = 0x8
+	ENABLE_MOUSE_INPUT            = 0x10
+	ENABLE_INSERT_MODE            = 0x20
+	ENABLE_QUICK_EDIT_MODE        = 0x40
+	ENABLE_EXTENDED_FLAGS         = 0x80
+	ENABLE_AUTO_POSITION          = 0x100
+	ENABLE_VIRTUAL_TERMINAL_INPUT = 0x200
+
+	ENABLE_PROCESSED_OUTPUT            = 0x1
+	ENABLE_WRAP_AT_EOL_OUTPUT          = 0x2
+	ENABLE_VIRTUAL_TERMINAL_PROCESSING = 0x4
+	DISABLE_NEWLINE_AUTO_RETURN        = 0x8
+	ENABLE_LVB_GRID_WORLDWIDE          = 0x10
+)
+
+type Coord struct {
+	X int16
+	Y int16
+}
+
+type SmallRect struct {
+	Left   int16
+	Top    int16
+	Right  int16
+	Bottom int16
+}
+
+// Used with GetConsoleScreenBuffer to retrieve information about a console
+// screen buffer. See
+// https://docs.microsoft.com/en-us/windows/console/console-screen-buffer-info-str
+// for details.
+
+type ConsoleScreenBufferInfo struct {
+	Size              Coord
+	CursorPosition    Coord
+	Attributes        uint16
+	Window            SmallRect
+	MaximumWindowSize Coord
+}
+
+const UNIX_PATH_MAX = 108 // defined in afunix.h
+
+const (
+	// flags for JOBOBJECT_BASIC_LIMIT_INFORMATION.LimitFlags
+	JOB_OBJECT_LIMIT_ACTIVE_PROCESS             = 0x00000008
+	JOB_OBJECT_LIMIT_AFFINITY                   = 0x00000010
+	JOB_OBJECT_LIMIT_BREAKAWAY_OK               = 0x00000800
+	JOB_OBJECT_LIMIT_DIE_ON_UNHANDLED_EXCEPTION = 0x00000400
+	JOB_OBJECT_LIMIT_JOB_MEMORY                 = 0x00000200
+	JOB_OBJECT_LIMIT_JOB_TIME                   = 0x00000004
+	JOB_OBJECT_LIMIT_KILL_ON_JOB_CLOSE          = 0x00002000
+	JOB_OBJECT_LIMIT_PRESERVE_JOB_TIME          = 0x00000040
+	JOB_OBJECT_LIMIT_PRIORITY_CLASS             = 0x00000020
+	JOB_OBJECT_LIMIT_PROCESS_MEMORY             = 0x00000100
+	JOB_OBJECT_LIMIT_PROCESS_TIME               = 0x00000002
+	JOB_OBJECT_LIMIT_SCHEDULING_CLASS           = 0x00000080
+	JOB_OBJECT_LIMIT_SILENT_BREAKAWAY_OK        = 0x00001000
+	JOB_OBJECT_LIMIT_SUBSET_AFFINITY            = 0x00004000
+	JOB_OBJECT_LIMIT_WORKINGSET                 = 0x00000001
+)
+
+type JOBOBJECT_BASIC_LIMIT_INFORMATION struct {
+	PerProcessUserTimeLimit int64
+	PerJobUserTimeLimit     int64
+	LimitFlags              uint32
+	MinimumWorkingSetSize   uintptr
+	MaximumWorkingSetSize   uintptr
+	ActiveProcessLimit      uint32
+	Affinity                uintptr
+	PriorityClass           uint32
+	SchedulingClass         uint32
+}
+
+type IO_COUNTERS struct {
+	ReadOperationCount  uint64
+	WriteOperationCount uint64
+	OtherOperationCount uint64
+	ReadTransferCount   uint64
+	WriteTransferCount  uint64
+	OtherTransferCount  uint64
+}
+
+type JOBOBJECT_EXTENDED_LIMIT_INFORMATION struct {
+	BasicLimitInformation JOBOBJECT_BASIC_LIMIT_INFORMATION
+	IoInfo                IO_COUNTERS
+	ProcessMemoryLimit    uintptr
+	JobMemoryLimit        uintptr
+	PeakProcessMemoryUsed uintptr
+	PeakJobMemoryUsed     uintptr
+}
+
+const (
+	// UIRestrictionsClass
+	JOB_OBJECT_UILIMIT_DESKTOP          = 0x00000040
+	JOB_OBJECT_UILIMIT_DISPLAYSETTINGS  = 0x00000010
+	JOB_OBJECT_UILIMIT_EXITWINDOWS      = 0x00000080
+	JOB_OBJECT_UILIMIT_GLOBALATOMS      = 0x00000020
+	JOB_OBJECT_UILIMIT_HANDLES          = 0x00000001
+	JOB_OBJECT_UILIMIT_READCLIPBOARD    = 0x00000002
+	JOB_OBJECT_UILIMIT_SYSTEMPARAMETERS = 0x00000008
+	JOB_OBJECT_UILIMIT_WRITECLIPBOARD   = 0x00000004
+)
+
+type JOBOBJECT_BASIC_UI_RESTRICTIONS struct {
+	UIRestrictionsClass uint32
+}
+
+const (
+	// JobObjectInformationClass
+	JobObjectAssociateCompletionPortInformation = 7
+	JobObjectBasicLimitInformation              = 2
+	JobObjectBasicUIRestrictions                = 4
+	JobObjectCpuRateControlInformation          = 15
+	JobObjectEndOfJobTimeInformation            = 6
+	JobObjectExtendedLimitInformation           = 9
+	JobObjectGroupInformation                   = 11
+	JobObjectGroupInformationEx                 = 14
+	JobObjectLimitViolationInformation2         = 35
+	JobObjectNetRateControlInformation          = 32
+	JobObjectNotificationLimitInformation       = 12
+	JobObjectNotificationLimitInformation2      = 34
+	JobObjectSecurityLimitInformation           = 5
+)
+
+const (
+	KF_FLAG_DEFAULT                          = 0x00000000
+	KF_FLAG_FORCE_APP_DATA_REDIRECTION       = 0x00080000
+	KF_FLAG_RETURN_FILTER_REDIRECTION_TARGET = 0x00040000
+	KF_FLAG_FORCE_PACKAGE_REDIRECTION        = 0x00020000
+	KF_FLAG_NO_PACKAGE_REDIRECTION           = 0x00010000
+	KF_FLAG_FORCE_APPCONTAINER_REDIRECTION   = 0x00020000
+	KF_FLAG_NO_APPCONTAINER_REDIRECTION      = 0x00010000
+	KF_FLAG_CREATE                           = 0x00008000
+	KF_FLAG_DONT_VERIFY                      = 0x00004000
+	KF_FLAG_DONT_UNEXPAND                    = 0x00002000
+	KF_FLAG_NO_ALIAS                         = 0x00001000
+	KF_FLAG_INIT                             = 0x00000800
+	KF_FLAG_DEFAULT_PATH                     = 0x00000400
+	KF_FLAG_NOT_PARENT_RELATIVE              = 0x00000200
+	KF_FLAG_SIMPLE_IDLIST                    = 0x00000100
+	KF_FLAG_ALIAS_ONLY                       = 0x80000000
+)
+
+type OsVersionInfoEx struct {
+	osVersionInfoSize uint32
+	MajorVersion      uint32
+	MinorVersion      uint32
+	BuildNumber       uint32
+	PlatformId        uint32
+	CsdVersion        [128]uint16
+	ServicePackMajor  uint16
+	ServicePackMinor  uint16
+	SuiteMask         uint16
+	ProductType       byte
+	_                 byte
+}
+
+const (
+	EWX_LOGOFF          = 0x00000000
+	EWX_SHUTDOWN        = 0x00000001
+	EWX_REBOOT          = 0x00000002
+	EWX_FORCE           = 0x00000004
+	EWX_POWEROFF        = 0x00000008
+	EWX_FORCEIFHUNG     = 0x00000010
+	EWX_QUICKRESOLVE    = 0x00000020
+	EWX_RESTARTAPPS     = 0x00000040
+	EWX_HYBRID_SHUTDOWN = 0x00400000
+	EWX_BOOTOPTIONS     = 0x01000000
+
+	SHTDN_REASON_FLAG_COMMENT_REQUIRED          = 0x01000000
+	SHTDN_REASON_FLAG_DIRTY_PROBLEM_ID_REQUIRED = 0x02000000
+	SHTDN_REASON_FLAG_CLEAN_UI                  = 0x04000000
+	SHTDN_REASON_FLAG_DIRTY_UI                  = 0x08000000
+	SHTDN_REASON_FLAG_USER_DEFINED              = 0x40000000
+	SHTDN_REASON_FLAG_PLANNED                   = 0x80000000
+	SHTDN_REASON_MAJOR_OTHER                    = 0x00000000
+	SHTDN_REASON_MAJOR_NONE                     = 0x00000000
+	SHTDN_REASON_MAJOR_HARDWARE                 = 0x00010000
+	SHTDN_REASON_MAJOR_OPERATINGSYSTEM          = 0x00020000
+	SHTDN_REASON_MAJOR_SOFTWARE                 = 0x00030000
+	SHTDN_REASON_MAJOR_APPLICATION              = 0x00040000
+	SHTDN_REASON_MAJOR_SYSTEM                   = 0x00050000
+	SHTDN_REASON_MAJOR_POWER                    = 0x00060000
+	SHTDN_REASON_MAJOR_LEGACY_API               = 0x00070000
+	SHTDN_REASON_MINOR_OTHER                    = 0x00000000
+	SHTDN_REASON_MINOR_NONE                     = 0x000000ff
+	SHTDN_REASON_MINOR_MAINTENANCE              = 0x00000001
+	SHTDN_REASON_MINOR_INSTALLATION             = 0x00000002
+	SHTDN_REASON_MINOR_UPGRADE                  = 0x00000003
+	SHTDN_REASON_MINOR_RECONFIG                 = 0x00000004
+	SHTDN_REASON_MINOR_HUNG                     = 0x00000005
+	SHTDN_REASON_MINOR_UNSTABLE                 = 0x00000006
+	SHTDN_REASON_MINOR_DISK                     = 0x00000007
+	SHTDN_REASON_MINOR_PROCESSOR                = 0x00000008
+	SHTDN_REASON_MINOR_NETWORKCARD              = 0x00000009
+	SHTDN_REASON_MINOR_POWER_SUPPLY             = 0x0000000a
+	SHTDN_REASON_MINOR_CORDUNPLUGGED            = 0x0000000b
+	SHTDN_REASON_MINOR_ENVIRONMENT              = 0x0000000c
+	SHTDN_REASON_MINOR_HARDWARE_DRIVER          = 0x0000000d
+	SHTDN_REASON_MINOR_OTHERDRIVER              = 0x0000000e
+	SHTDN_REASON_MINOR_BLUESCREEN               = 0x0000000F
+	SHTDN_REASON_MINOR_SERVICEPACK              = 0x00000010
+	SHTDN_REASON_MINOR_HOTFIX                   = 0x00000011
+	SHTDN_REASON_MINOR_SECURITYFIX              = 0x00000012
+	SHTDN_REASON_MINOR_SECURITY                 = 0x00000013
+	SHTDN_REASON_MINOR_NETWORK_CONNECTIVITY     = 0x00000014
+	SHTDN_REASON_MINOR_WMI                      = 0x00000015
+	SHTDN_REASON_MINOR_SERVICEPACK_UNINSTALL    = 0x00000016
+	SHTDN_REASON_MINOR_HOTFIX_UNINSTALL         = 0x00000017
+	SHTDN_REASON_MINOR_SECURITYFIX_UNINSTALL    = 0x00000018
+	SHTDN_REASON_MINOR_MMC                      = 0x00000019
+	SHTDN_REASON_MINOR_SYSTEMRESTORE            = 0x0000001a
+	SHTDN_REASON_MINOR_TERMSRV                  = 0x00000020
+	SHTDN_REASON_MINOR_DC_PROMOTION             = 0x00000021
+	SHTDN_REASON_MINOR_DC_DEMOTION              = 0x00000022
+	SHTDN_REASON_UNKNOWN                        = SHTDN_REASON_MINOR_NONE
+	SHTDN_REASON_LEGACY_API                     = SHTDN_REASON_MAJOR_LEGACY_API | SHTDN_REASON_FLAG_PLANNED
+	SHTDN_REASON_VALID_BIT_MASK                 = 0xc0ffffff
+
+	SHUTDOWN_NORETRY = 0x1
+)
diff --git a/vendor/golang.org/x/sys/windows/types_windows_386.go b/vendor/golang.org/x/sys/windows/types_windows_386.go
new file mode 100644
index 0000000..fe0ddd0
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/types_windows_386.go
@@ -0,0 +1,22 @@
+// Copyright 2011 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package windows
+
+type WSAData struct {
+	Version      uint16
+	HighVersion  uint16
+	Description  [WSADESCRIPTION_LEN + 1]byte
+	SystemStatus [WSASYS_STATUS_LEN + 1]byte
+	MaxSockets   uint16
+	MaxUdpDg     uint16
+	VendorInfo   *byte
+}
+
+type Servent struct {
+	Name    *byte
+	Aliases **byte
+	Port    uint16
+	Proto   *byte
+}
diff --git a/vendor/golang.org/x/sys/windows/types_windows_amd64.go b/vendor/golang.org/x/sys/windows/types_windows_amd64.go
new file mode 100644
index 0000000..7e154c2
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/types_windows_amd64.go
@@ -0,0 +1,22 @@
+// Copyright 2011 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package windows
+
+type WSAData struct {
+	Version      uint16
+	HighVersion  uint16
+	MaxSockets   uint16
+	MaxUdpDg     uint16
+	VendorInfo   *byte
+	Description  [WSADESCRIPTION_LEN + 1]byte
+	SystemStatus [WSASYS_STATUS_LEN + 1]byte
+}
+
+type Servent struct {
+	Name    *byte
+	Aliases **byte
+	Proto   *byte
+	Port    uint16
+}
diff --git a/vendor/golang.org/x/sys/windows/types_windows_arm.go b/vendor/golang.org/x/sys/windows/types_windows_arm.go
new file mode 100644
index 0000000..74571e3
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/types_windows_arm.go
@@ -0,0 +1,22 @@
+// Copyright 2018 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package windows
+
+type WSAData struct {
+	Version      uint16
+	HighVersion  uint16
+	Description  [WSADESCRIPTION_LEN + 1]byte
+	SystemStatus [WSASYS_STATUS_LEN + 1]byte
+	MaxSockets   uint16
+	MaxUdpDg     uint16
+	VendorInfo   *byte
+}
+
+type Servent struct {
+	Name    *byte
+	Aliases **byte
+	Port    uint16
+	Proto   *byte
+}
diff --git a/vendor/golang.org/x/sys/windows/zerrors_windows.go b/vendor/golang.org/x/sys/windows/zerrors_windows.go
new file mode 100644
index 0000000..f021200
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/zerrors_windows.go
@@ -0,0 +1,6853 @@
+// Code generated by 'mkerrors.bash'; DO NOT EDIT.
+
+package windows
+
+import "syscall"
+
+const (
+	FACILITY_NULL                                                                           = 0
+	FACILITY_RPC                                                                            = 1
+	FACILITY_DISPATCH                                                                       = 2
+	FACILITY_STORAGE                                                                        = 3
+	FACILITY_ITF                                                                            = 4
+	FACILITY_WIN32                                                                          = 7
+	FACILITY_WINDOWS                                                                        = 8
+	FACILITY_SSPI                                                                           = 9
+	FACILITY_SECURITY                                                                       = 9
+	FACILITY_CONTROL                                                                        = 10
+	FACILITY_CERT                                                                           = 11
+	FACILITY_INTERNET                                                                       = 12
+	FACILITY_MEDIASERVER                                                                    = 13
+	FACILITY_MSMQ                                                                           = 14
+	FACILITY_SETUPAPI                                                                       = 15
+	FACILITY_SCARD                                                                          = 16
+	FACILITY_COMPLUS                                                                        = 17
+	FACILITY_AAF                                                                            = 18
+	FACILITY_URT                                                                            = 19
+	FACILITY_ACS                                                                            = 20
+	FACILITY_DPLAY                                                                          = 21
+	FACILITY_UMI                                                                            = 22
+	FACILITY_SXS                                                                            = 23
+	FACILITY_WINDOWS_CE                                                                     = 24
+	FACILITY_HTTP                                                                           = 25
+	FACILITY_USERMODE_COMMONLOG                                                             = 26
+	FACILITY_WER                                                                            = 27
+	FACILITY_USERMODE_FILTER_MANAGER                                                        = 31
+	FACILITY_BACKGROUNDCOPY                                                                 = 32
+	FACILITY_CONFIGURATION                                                                  = 33
+	FACILITY_WIA                                                                            = 33
+	FACILITY_STATE_MANAGEMENT                                                               = 34
+	FACILITY_METADIRECTORY                                                                  = 35
+	FACILITY_WINDOWSUPDATE                                                                  = 36
+	FACILITY_DIRECTORYSERVICE                                                               = 37
+	FACILITY_GRAPHICS                                                                       = 38
+	FACILITY_SHELL                                                                          = 39
+	FACILITY_NAP                                                                            = 39
+	FACILITY_TPM_SERVICES                                                                   = 40
+	FACILITY_TPM_SOFTWARE                                                                   = 41
+	FACILITY_UI                                                                             = 42
+	FACILITY_XAML                                                                           = 43
+	FACILITY_ACTION_QUEUE                                                                   = 44
+	FACILITY_PLA                                                                            = 48
+	FACILITY_WINDOWS_SETUP                                                                  = 48
+	FACILITY_FVE                                                                            = 49
+	FACILITY_FWP                                                                            = 50
+	FACILITY_WINRM                                                                          = 51
+	FACILITY_NDIS                                                                           = 52
+	FACILITY_USERMODE_HYPERVISOR                                                            = 53
+	FACILITY_CMI                                                                            = 54
+	FACILITY_USERMODE_VIRTUALIZATION                                                        = 55
+	FACILITY_USERMODE_VOLMGR                                                                = 56
+	FACILITY_BCD                                                                            = 57
+	FACILITY_USERMODE_VHD                                                                   = 58
+	FACILITY_USERMODE_HNS                                                                   = 59
+	FACILITY_SDIAG                                                                          = 60
+	FACILITY_WEBSERVICES                                                                    = 61
+	FACILITY_WINPE                                                                          = 61
+	FACILITY_WPN                                                                            = 62
+	FACILITY_WINDOWS_STORE                                                                  = 63
+	FACILITY_INPUT                                                                          = 64
+	FACILITY_EAP                                                                            = 66
+	FACILITY_WINDOWS_DEFENDER                                                               = 80
+	FACILITY_OPC                                                                            = 81
+	FACILITY_XPS                                                                            = 82
+	FACILITY_MBN                                                                            = 84
+	FACILITY_POWERSHELL                                                                     = 84
+	FACILITY_RAS                                                                            = 83
+	FACILITY_P2P_INT                                                                        = 98
+	FACILITY_P2P                                                                            = 99
+	FACILITY_DAF                                                                            = 100
+	FACILITY_BLUETOOTH_ATT                                                                  = 101
+	FACILITY_AUDIO                                                                          = 102
+	FACILITY_STATEREPOSITORY                                                                = 103
+	FACILITY_VISUALCPP                                                                      = 109
+	FACILITY_SCRIPT                                                                         = 112
+	FACILITY_PARSE                                                                          = 113
+	FACILITY_BLB                                                                            = 120
+	FACILITY_BLB_CLI                                                                        = 121
+	FACILITY_WSBAPP                                                                         = 122
+	FACILITY_BLBUI                                                                          = 128
+	FACILITY_USN                                                                            = 129
+	FACILITY_USERMODE_VOLSNAP                                                               = 130
+	FACILITY_TIERING                                                                        = 131
+	FACILITY_WSB_ONLINE                                                                     = 133
+	FACILITY_ONLINE_ID                                                                      = 134
+	FACILITY_DEVICE_UPDATE_AGENT                                                            = 135
+	FACILITY_DRVSERVICING                                                                   = 136
+	FACILITY_DLS                                                                            = 153
+	FACILITY_DELIVERY_OPTIMIZATION                                                          = 208
+	FACILITY_USERMODE_SPACES                                                                = 231
+	FACILITY_USER_MODE_SECURITY_CORE                                                        = 232
+	FACILITY_USERMODE_LICENSING                                                             = 234
+	FACILITY_SOS                                                                            = 160
+	FACILITY_DEBUGGERS                                                                      = 176
+	FACILITY_SPP                                                                            = 256
+	FACILITY_RESTORE                                                                        = 256
+	FACILITY_DMSERVER                                                                       = 256
+	FACILITY_DEPLOYMENT_SERVICES_SERVER                                                     = 257
+	FACILITY_DEPLOYMENT_SERVICES_IMAGING                                                    = 258
+	FACILITY_DEPLOYMENT_SERVICES_MANAGEMENT                                                 = 259
+	FACILITY_DEPLOYMENT_SERVICES_UTIL                                                       = 260
+	FACILITY_DEPLOYMENT_SERVICES_BINLSVC                                                    = 261
+	FACILITY_DEPLOYMENT_SERVICES_PXE                                                        = 263
+	FACILITY_DEPLOYMENT_SERVICES_TFTP                                                       = 264
+	FACILITY_DEPLOYMENT_SERVICES_TRANSPORT_MANAGEMENT                                       = 272
+	FACILITY_DEPLOYMENT_SERVICES_DRIVER_PROVISIONING                                        = 278
+	FACILITY_DEPLOYMENT_SERVICES_MULTICAST_SERVER                                           = 289
+	FACILITY_DEPLOYMENT_SERVICES_MULTICAST_CLIENT                                           = 290
+	FACILITY_DEPLOYMENT_SERVICES_CONTENT_PROVIDER                                           = 293
+	FACILITY_LINGUISTIC_SERVICES                                                            = 305
+	FACILITY_AUDIOSTREAMING                                                                 = 1094
+	FACILITY_ACCELERATOR                                                                    = 1536
+	FACILITY_WMAAECMA                                                                       = 1996
+	FACILITY_DIRECTMUSIC                                                                    = 2168
+	FACILITY_DIRECT3D10                                                                     = 2169
+	FACILITY_DXGI                                                                           = 2170
+	FACILITY_DXGI_DDI                                                                       = 2171
+	FACILITY_DIRECT3D11                                                                     = 2172
+	FACILITY_DIRECT3D11_DEBUG                                                               = 2173
+	FACILITY_DIRECT3D12                                                                     = 2174
+	FACILITY_DIRECT3D12_DEBUG                                                               = 2175
+	FACILITY_LEAP                                                                           = 2184
+	FACILITY_AUDCLNT                                                                        = 2185
+	FACILITY_WINCODEC_DWRITE_DWM                                                            = 2200
+	FACILITY_WINML                                                                          = 2192
+	FACILITY_DIRECT2D                                                                       = 2201
+	FACILITY_DEFRAG                                                                         = 2304
+	FACILITY_USERMODE_SDBUS                                                                 = 2305
+	FACILITY_JSCRIPT                                                                        = 2306
+	FACILITY_PIDGENX                                                                        = 2561
+	FACILITY_EAS                                                                            = 85
+	FACILITY_WEB                                                                            = 885
+	FACILITY_WEB_SOCKET                                                                     = 886
+	FACILITY_MOBILE                                                                         = 1793
+	FACILITY_SQLITE                                                                         = 1967
+	FACILITY_UTC                                                                            = 1989
+	FACILITY_WEP                                                                            = 2049
+	FACILITY_SYNCENGINE                                                                     = 2050
+	FACILITY_XBOX                                                                           = 2339
+	FACILITY_PIX                                                                            = 2748
+	ERROR_SUCCESS                                                             syscall.Errno = 0
+	NO_ERROR                                                                                = 0
+	SEC_E_OK                                                                  Handle        = 0x00000000
+	ERROR_INVALID_FUNCTION                                                    syscall.Errno = 1
+	ERROR_FILE_NOT_FOUND                                                      syscall.Errno = 2
+	ERROR_PATH_NOT_FOUND                                                      syscall.Errno = 3
+	ERROR_TOO_MANY_OPEN_FILES                                                 syscall.Errno = 4
+	ERROR_ACCESS_DENIED                                                       syscall.Errno = 5
+	ERROR_INVALID_HANDLE                                                      syscall.Errno = 6
+	ERROR_ARENA_TRASHED                                                       syscall.Errno = 7
+	ERROR_NOT_ENOUGH_MEMORY                                                   syscall.Errno = 8
+	ERROR_INVALID_BLOCK                                                       syscall.Errno = 9
+	ERROR_BAD_ENVIRONMENT                                                     syscall.Errno = 10
+	ERROR_BAD_FORMAT                                                          syscall.Errno = 11
+	ERROR_INVALID_ACCESS                                                      syscall.Errno = 12
+	ERROR_INVALID_DATA                                                        syscall.Errno = 13
+	ERROR_OUTOFMEMORY                                                         syscall.Errno = 14
+	ERROR_INVALID_DRIVE                                                       syscall.Errno = 15
+	ERROR_CURRENT_DIRECTORY                                                   syscall.Errno = 16
+	ERROR_NOT_SAME_DEVICE                                                     syscall.Errno = 17
+	ERROR_NO_MORE_FILES                                                       syscall.Errno = 18
+	ERROR_WRITE_PROTECT                                                       syscall.Errno = 19
+	ERROR_BAD_UNIT                                                            syscall.Errno = 20
+	ERROR_NOT_READY                                                           syscall.Errno = 21
+	ERROR_BAD_COMMAND                                                         syscall.Errno = 22
+	ERROR_CRC                                                                 syscall.Errno = 23
+	ERROR_BAD_LENGTH                                                          syscall.Errno = 24
+	ERROR_SEEK                                                                syscall.Errno = 25
+	ERROR_NOT_DOS_DISK                                                        syscall.Errno = 26
+	ERROR_SECTOR_NOT_FOUND                                                    syscall.Errno = 27
+	ERROR_OUT_OF_PAPER                                                        syscall.Errno = 28
+	ERROR_WRITE_FAULT                                                         syscall.Errno = 29
+	ERROR_READ_FAULT                                                          syscall.Errno = 30
+	ERROR_GEN_FAILURE                                                         syscall.Errno = 31
+	ERROR_SHARING_VIOLATION                                                   syscall.Errno = 32
+	ERROR_LOCK_VIOLATION                                                      syscall.Errno = 33
+	ERROR_WRONG_DISK                                                          syscall.Errno = 34
+	ERROR_SHARING_BUFFER_EXCEEDED                                             syscall.Errno = 36
+	ERROR_HANDLE_EOF                                                          syscall.Errno = 38
+	ERROR_HANDLE_DISK_FULL                                                    syscall.Errno = 39
+	ERROR_NOT_SUPPORTED                                                       syscall.Errno = 50
+	ERROR_REM_NOT_LIST                                                        syscall.Errno = 51
+	ERROR_DUP_NAME                                                            syscall.Errno = 52
+	ERROR_BAD_NETPATH                                                         syscall.Errno = 53
+	ERROR_NETWORK_BUSY                                                        syscall.Errno = 54
+	ERROR_DEV_NOT_EXIST                                                       syscall.Errno = 55
+	ERROR_TOO_MANY_CMDS                                                       syscall.Errno = 56
+	ERROR_ADAP_HDW_ERR                                                        syscall.Errno = 57
+	ERROR_BAD_NET_RESP                                                        syscall.Errno = 58
+	ERROR_UNEXP_NET_ERR                                                       syscall.Errno = 59
+	ERROR_BAD_REM_ADAP                                                        syscall.Errno = 60
+	ERROR_PRINTQ_FULL                                                         syscall.Errno = 61
+	ERROR_NO_SPOOL_SPACE                                                      syscall.Errno = 62
+	ERROR_PRINT_CANCELLED                                                     syscall.Errno = 63
+	ERROR_NETNAME_DELETED                                                     syscall.Errno = 64
+	ERROR_NETWORK_ACCESS_DENIED                                               syscall.Errno = 65
+	ERROR_BAD_DEV_TYPE                                                        syscall.Errno = 66
+	ERROR_BAD_NET_NAME                                                        syscall.Errno = 67
+	ERROR_TOO_MANY_NAMES                                                      syscall.Errno = 68
+	ERROR_TOO_MANY_SESS                                                       syscall.Errno = 69
+	ERROR_SHARING_PAUSED                                                      syscall.Errno = 70
+	ERROR_REQ_NOT_ACCEP                                                       syscall.Errno = 71
+	ERROR_REDIR_PAUSED                                                        syscall.Errno = 72
+	ERROR_FILE_EXISTS                                                         syscall.Errno = 80
+	ERROR_CANNOT_MAKE                                                         syscall.Errno = 82
+	ERROR_FAIL_I24                                                            syscall.Errno = 83
+	ERROR_OUT_OF_STRUCTURES                                                   syscall.Errno = 84
+	ERROR_ALREADY_ASSIGNED                                                    syscall.Errno = 85
+	ERROR_INVALID_PASSWORD                                                    syscall.Errno = 86
+	ERROR_INVALID_PARAMETER                                                   syscall.Errno = 87
+	ERROR_NET_WRITE_FAULT                                                     syscall.Errno = 88
+	ERROR_NO_PROC_SLOTS                                                       syscall.Errno = 89
+	ERROR_TOO_MANY_SEMAPHORES                                                 syscall.Errno = 100
+	ERROR_EXCL_SEM_ALREADY_OWNED                                              syscall.Errno = 101
+	ERROR_SEM_IS_SET                                                          syscall.Errno = 102
+	ERROR_TOO_MANY_SEM_REQUESTS                                               syscall.Errno = 103
+	ERROR_INVALID_AT_INTERRUPT_TIME                                           syscall.Errno = 104
+	ERROR_SEM_OWNER_DIED                                                      syscall.Errno = 105
+	ERROR_SEM_USER_LIMIT                                                      syscall.Errno = 106
+	ERROR_DISK_CHANGE                                                         syscall.Errno = 107
+	ERROR_DRIVE_LOCKED                                                        syscall.Errno = 108
+	ERROR_BROKEN_PIPE                                                         syscall.Errno = 109
+	ERROR_OPEN_FAILED                                                         syscall.Errno = 110
+	ERROR_BUFFER_OVERFLOW                                                     syscall.Errno = 111
+	ERROR_DISK_FULL                                                           syscall.Errno = 112
+	ERROR_NO_MORE_SEARCH_HANDLES                                              syscall.Errno = 113
+	ERROR_INVALID_TARGET_HANDLE                                               syscall.Errno = 114
+	ERROR_INVALID_CATEGORY                                                    syscall.Errno = 117
+	ERROR_INVALID_VERIFY_SWITCH                                               syscall.Errno = 118
+	ERROR_BAD_DRIVER_LEVEL                                                    syscall.Errno = 119
+	ERROR_CALL_NOT_IMPLEMENTED                                                syscall.Errno = 120
+	ERROR_SEM_TIMEOUT                                                         syscall.Errno = 121
+	ERROR_INSUFFICIENT_BUFFER                                                 syscall.Errno = 122
+	ERROR_INVALID_NAME                                                        syscall.Errno = 123
+	ERROR_INVALID_LEVEL                                                       syscall.Errno = 124
+	ERROR_NO_VOLUME_LABEL                                                     syscall.Errno = 125
+	ERROR_MOD_NOT_FOUND                                                       syscall.Errno = 126
+	ERROR_PROC_NOT_FOUND                                                      syscall.Errno = 127
+	ERROR_WAIT_NO_CHILDREN                                                    syscall.Errno = 128
+	ERROR_CHILD_NOT_COMPLETE                                                  syscall.Errno = 129
+	ERROR_DIRECT_ACCESS_HANDLE                                                syscall.Errno = 130
+	ERROR_NEGATIVE_SEEK                                                       syscall.Errno = 131
+	ERROR_SEEK_ON_DEVICE                                                      syscall.Errno = 132
+	ERROR_IS_JOIN_TARGET                                                      syscall.Errno = 133
+	ERROR_IS_JOINED                                                           syscall.Errno = 134
+	ERROR_IS_SUBSTED                                                          syscall.Errno = 135
+	ERROR_NOT_JOINED                                                          syscall.Errno = 136
+	ERROR_NOT_SUBSTED                                                         syscall.Errno = 137
+	ERROR_JOIN_TO_JOIN                                                        syscall.Errno = 138
+	ERROR_SUBST_TO_SUBST                                                      syscall.Errno = 139
+	ERROR_JOIN_TO_SUBST                                                       syscall.Errno = 140
+	ERROR_SUBST_TO_JOIN                                                       syscall.Errno = 141
+	ERROR_BUSY_DRIVE                                                          syscall.Errno = 142
+	ERROR_SAME_DRIVE                                                          syscall.Errno = 143
+	ERROR_DIR_NOT_ROOT                                                        syscall.Errno = 144
+	ERROR_DIR_NOT_EMPTY                                                       syscall.Errno = 145
+	ERROR_IS_SUBST_PATH                                                       syscall.Errno = 146
+	ERROR_IS_JOIN_PATH                                                        syscall.Errno = 147
+	ERROR_PATH_BUSY                                                           syscall.Errno = 148
+	ERROR_IS_SUBST_TARGET                                                     syscall.Errno = 149
+	ERROR_SYSTEM_TRACE                                                        syscall.Errno = 150
+	ERROR_INVALID_EVENT_COUNT                                                 syscall.Errno = 151
+	ERROR_TOO_MANY_MUXWAITERS                                                 syscall.Errno = 152
+	ERROR_INVALID_LIST_FORMAT                                                 syscall.Errno = 153
+	ERROR_LABEL_TOO_LONG                                                      syscall.Errno = 154
+	ERROR_TOO_MANY_TCBS                                                       syscall.Errno = 155
+	ERROR_SIGNAL_REFUSED                                                      syscall.Errno = 156
+	ERROR_DISCARDED                                                           syscall.Errno = 157
+	ERROR_NOT_LOCKED                                                          syscall.Errno = 158
+	ERROR_BAD_THREADID_ADDR                                                   syscall.Errno = 159
+	ERROR_BAD_ARGUMENTS                                                       syscall.Errno = 160
+	ERROR_BAD_PATHNAME                                                        syscall.Errno = 161
+	ERROR_SIGNAL_PENDING                                                      syscall.Errno = 162
+	ERROR_MAX_THRDS_REACHED                                                   syscall.Errno = 164
+	ERROR_LOCK_FAILED                                                         syscall.Errno = 167
+	ERROR_BUSY                                                                syscall.Errno = 170
+	ERROR_DEVICE_SUPPORT_IN_PROGRESS                                          syscall.Errno = 171
+	ERROR_CANCEL_VIOLATION                                                    syscall.Errno = 173
+	ERROR_ATOMIC_LOCKS_NOT_SUPPORTED                                          syscall.Errno = 174
+	ERROR_INVALID_SEGMENT_NUMBER                                              syscall.Errno = 180
+	ERROR_INVALID_ORDINAL                                                     syscall.Errno = 182
+	ERROR_ALREADY_EXISTS                                                      syscall.Errno = 183
+	ERROR_INVALID_FLAG_NUMBER                                                 syscall.Errno = 186
+	ERROR_SEM_NOT_FOUND                                                       syscall.Errno = 187
+	ERROR_INVALID_STARTING_CODESEG                                            syscall.Errno = 188
+	ERROR_INVALID_STACKSEG                                                    syscall.Errno = 189
+	ERROR_INVALID_MODULETYPE                                                  syscall.Errno = 190
+	ERROR_INVALID_EXE_SIGNATURE                                               syscall.Errno = 191
+	ERROR_EXE_MARKED_INVALID                                                  syscall.Errno = 192
+	ERROR_BAD_EXE_FORMAT                                                      syscall.Errno = 193
+	ERROR_ITERATED_DATA_EXCEEDS_64k                                           syscall.Errno = 194
+	ERROR_INVALID_MINALLOCSIZE                                                syscall.Errno = 195
+	ERROR_DYNLINK_FROM_INVALID_RING                                           syscall.Errno = 196
+	ERROR_IOPL_NOT_ENABLED                                                    syscall.Errno = 197
+	ERROR_INVALID_SEGDPL                                                      syscall.Errno = 198
+	ERROR_AUTODATASEG_EXCEEDS_64k                                             syscall.Errno = 199
+	ERROR_RING2SEG_MUST_BE_MOVABLE                                            syscall.Errno = 200
+	ERROR_RELOC_CHAIN_XEEDS_SEGLIM                                            syscall.Errno = 201
+	ERROR_INFLOOP_IN_RELOC_CHAIN                                              syscall.Errno = 202
+	ERROR_ENVVAR_NOT_FOUND                                                    syscall.Errno = 203
+	ERROR_NO_SIGNAL_SENT                                                      syscall.Errno = 205
+	ERROR_FILENAME_EXCED_RANGE                                                syscall.Errno = 206
+	ERROR_RING2_STACK_IN_USE                                                  syscall.Errno = 207
+	ERROR_META_EXPANSION_TOO_LONG                                             syscall.Errno = 208
+	ERROR_INVALID_SIGNAL_NUMBER                                               syscall.Errno = 209
+	ERROR_THREAD_1_INACTIVE                                                   syscall.Errno = 210
+	ERROR_LOCKED                                                              syscall.Errno = 212
+	ERROR_TOO_MANY_MODULES                                                    syscall.Errno = 214
+	ERROR_NESTING_NOT_ALLOWED                                                 syscall.Errno = 215
+	ERROR_EXE_MACHINE_TYPE_MISMATCH                                           syscall.Errno = 216
+	ERROR_EXE_CANNOT_MODIFY_SIGNED_BINARY                                     syscall.Errno = 217
+	ERROR_EXE_CANNOT_MODIFY_STRONG_SIGNED_BINARY                              syscall.Errno = 218
+	ERROR_FILE_CHECKED_OUT                                                    syscall.Errno = 220
+	ERROR_CHECKOUT_REQUIRED                                                   syscall.Errno = 221
+	ERROR_BAD_FILE_TYPE                                                       syscall.Errno = 222
+	ERROR_FILE_TOO_LARGE                                                      syscall.Errno = 223
+	ERROR_FORMS_AUTH_REQUIRED                                                 syscall.Errno = 224
+	ERROR_VIRUS_INFECTED                                                      syscall.Errno = 225
+	ERROR_VIRUS_DELETED                                                       syscall.Errno = 226
+	ERROR_PIPE_LOCAL                                                          syscall.Errno = 229
+	ERROR_BAD_PIPE                                                            syscall.Errno = 230
+	ERROR_PIPE_BUSY                                                           syscall.Errno = 231
+	ERROR_NO_DATA                                                             syscall.Errno = 232
+	ERROR_PIPE_NOT_CONNECTED                                                  syscall.Errno = 233
+	ERROR_MORE_DATA                                                           syscall.Errno = 234
+	ERROR_NO_WORK_DONE                                                        syscall.Errno = 235
+	ERROR_VC_DISCONNECTED                                                     syscall.Errno = 240
+	ERROR_INVALID_EA_NAME                                                     syscall.Errno = 254
+	ERROR_EA_LIST_INCONSISTENT                                                syscall.Errno = 255
+	WAIT_TIMEOUT                                                              syscall.Errno = 258
+	ERROR_NO_MORE_ITEMS                                                       syscall.Errno = 259
+	ERROR_CANNOT_COPY                                                         syscall.Errno = 266
+	ERROR_DIRECTORY                                                           syscall.Errno = 267
+	ERROR_EAS_DIDNT_FIT                                                       syscall.Errno = 275
+	ERROR_EA_FILE_CORRUPT                                                     syscall.Errno = 276
+	ERROR_EA_TABLE_FULL                                                       syscall.Errno = 277
+	ERROR_INVALID_EA_HANDLE                                                   syscall.Errno = 278
+	ERROR_EAS_NOT_SUPPORTED                                                   syscall.Errno = 282
+	ERROR_NOT_OWNER                                                           syscall.Errno = 288
+	ERROR_TOO_MANY_POSTS                                                      syscall.Errno = 298
+	ERROR_PARTIAL_COPY                                                        syscall.Errno = 299
+	ERROR_OPLOCK_NOT_GRANTED                                                  syscall.Errno = 300
+	ERROR_INVALID_OPLOCK_PROTOCOL                                             syscall.Errno = 301
+	ERROR_DISK_TOO_FRAGMENTED                                                 syscall.Errno = 302
+	ERROR_DELETE_PENDING                                                      syscall.Errno = 303
+	ERROR_INCOMPATIBLE_WITH_GLOBAL_SHORT_NAME_REGISTRY_SETTING                syscall.Errno = 304
+	ERROR_SHORT_NAMES_NOT_ENABLED_ON_VOLUME                                   syscall.Errno = 305
+	ERROR_SECURITY_STREAM_IS_INCONSISTENT                                     syscall.Errno = 306
+	ERROR_INVALID_LOCK_RANGE                                                  syscall.Errno = 307
+	ERROR_IMAGE_SUBSYSTEM_NOT_PRESENT                                         syscall.Errno = 308
+	ERROR_NOTIFICATION_GUID_ALREADY_DEFINED                                   syscall.Errno = 309
+	ERROR_INVALID_EXCEPTION_HANDLER                                           syscall.Errno = 310
+	ERROR_DUPLICATE_PRIVILEGES                                                syscall.Errno = 311
+	ERROR_NO_RANGES_PROCESSED                                                 syscall.Errno = 312
+	ERROR_NOT_ALLOWED_ON_SYSTEM_FILE                                          syscall.Errno = 313
+	ERROR_DISK_RESOURCES_EXHAUSTED                                            syscall.Errno = 314
+	ERROR_INVALID_TOKEN                                                       syscall.Errno = 315
+	ERROR_DEVICE_FEATURE_NOT_SUPPORTED                                        syscall.Errno = 316
+	ERROR_MR_MID_NOT_FOUND                                                    syscall.Errno = 317
+	ERROR_SCOPE_NOT_FOUND                                                     syscall.Errno = 318
+	ERROR_UNDEFINED_SCOPE                                                     syscall.Errno = 319
+	ERROR_INVALID_CAP                                                         syscall.Errno = 320
+	ERROR_DEVICE_UNREACHABLE                                                  syscall.Errno = 321
+	ERROR_DEVICE_NO_RESOURCES                                                 syscall.Errno = 322
+	ERROR_DATA_CHECKSUM_ERROR                                                 syscall.Errno = 323
+	ERROR_INTERMIXED_KERNEL_EA_OPERATION                                      syscall.Errno = 324
+	ERROR_FILE_LEVEL_TRIM_NOT_SUPPORTED                                       syscall.Errno = 326
+	ERROR_OFFSET_ALIGNMENT_VIOLATION                                          syscall.Errno = 327
+	ERROR_INVALID_FIELD_IN_PARAMETER_LIST                                     syscall.Errno = 328
+	ERROR_OPERATION_IN_PROGRESS                                               syscall.Errno = 329
+	ERROR_BAD_DEVICE_PATH                                                     syscall.Errno = 330
+	ERROR_TOO_MANY_DESCRIPTORS                                                syscall.Errno = 331
+	ERROR_SCRUB_DATA_DISABLED                                                 syscall.Errno = 332
+	ERROR_NOT_REDUNDANT_STORAGE                                               syscall.Errno = 333
+	ERROR_RESIDENT_FILE_NOT_SUPPORTED                                         syscall.Errno = 334
+	ERROR_COMPRESSED_FILE_NOT_SUPPORTED                                       syscall.Errno = 335
+	ERROR_DIRECTORY_NOT_SUPPORTED                                             syscall.Errno = 336
+	ERROR_NOT_READ_FROM_COPY                                                  syscall.Errno = 337
+	ERROR_FT_WRITE_FAILURE                                                    syscall.Errno = 338
+	ERROR_FT_DI_SCAN_REQUIRED                                                 syscall.Errno = 339
+	ERROR_INVALID_KERNEL_INFO_VERSION                                         syscall.Errno = 340
+	ERROR_INVALID_PEP_INFO_VERSION                                            syscall.Errno = 341
+	ERROR_OBJECT_NOT_EXTERNALLY_BACKED                                        syscall.Errno = 342
+	ERROR_EXTERNAL_BACKING_PROVIDER_UNKNOWN                                   syscall.Errno = 343
+	ERROR_COMPRESSION_NOT_BENEFICIAL                                          syscall.Errno = 344
+	ERROR_STORAGE_TOPOLOGY_ID_MISMATCH                                        syscall.Errno = 345
+	ERROR_BLOCKED_BY_PARENTAL_CONTROLS                                        syscall.Errno = 346
+	ERROR_BLOCK_TOO_MANY_REFERENCES                                           syscall.Errno = 347
+	ERROR_MARKED_TO_DISALLOW_WRITES                                           syscall.Errno = 348
+	ERROR_ENCLAVE_FAILURE                                                     syscall.Errno = 349
+	ERROR_FAIL_NOACTION_REBOOT                                                syscall.Errno = 350
+	ERROR_FAIL_SHUTDOWN                                                       syscall.Errno = 351
+	ERROR_FAIL_RESTART                                                        syscall.Errno = 352
+	ERROR_MAX_SESSIONS_REACHED                                                syscall.Errno = 353
+	ERROR_NETWORK_ACCESS_DENIED_EDP                                           syscall.Errno = 354
+	ERROR_DEVICE_HINT_NAME_BUFFER_TOO_SMALL                                   syscall.Errno = 355
+	ERROR_EDP_POLICY_DENIES_OPERATION                                         syscall.Errno = 356
+	ERROR_EDP_DPL_POLICY_CANT_BE_SATISFIED                                    syscall.Errno = 357
+	ERROR_CLOUD_FILE_SYNC_ROOT_METADATA_CORRUPT                               syscall.Errno = 358
+	ERROR_DEVICE_IN_MAINTENANCE                                               syscall.Errno = 359
+	ERROR_NOT_SUPPORTED_ON_DAX                                                syscall.Errno = 360
+	ERROR_DAX_MAPPING_EXISTS                                                  syscall.Errno = 361
+	ERROR_CLOUD_FILE_PROVIDER_NOT_RUNNING                                     syscall.Errno = 362
+	ERROR_CLOUD_FILE_METADATA_CORRUPT                                         syscall.Errno = 363
+	ERROR_CLOUD_FILE_METADATA_TOO_LARGE                                       syscall.Errno = 364
+	ERROR_CLOUD_FILE_PROPERTY_BLOB_TOO_LARGE                                  syscall.Errno = 365
+	ERROR_CLOUD_FILE_PROPERTY_BLOB_CHECKSUM_MISMATCH                          syscall.Errno = 366
+	ERROR_CHILD_PROCESS_BLOCKED                                               syscall.Errno = 367
+	ERROR_STORAGE_LOST_DATA_PERSISTENCE                                       syscall.Errno = 368
+	ERROR_FILE_SYSTEM_VIRTUALIZATION_UNAVAILABLE                              syscall.Errno = 369
+	ERROR_FILE_SYSTEM_VIRTUALIZATION_METADATA_CORRUPT                         syscall.Errno = 370
+	ERROR_FILE_SYSTEM_VIRTUALIZATION_BUSY                                     syscall.Errno = 371
+	ERROR_FILE_SYSTEM_VIRTUALIZATION_PROVIDER_UNKNOWN                         syscall.Errno = 372
+	ERROR_GDI_HANDLE_LEAK                                                     syscall.Errno = 373
+	ERROR_CLOUD_FILE_TOO_MANY_PROPERTY_BLOBS                                  syscall.Errno = 374
+	ERROR_CLOUD_FILE_PROPERTY_VERSION_NOT_SUPPORTED                           syscall.Errno = 375
+	ERROR_NOT_A_CLOUD_FILE                                                    syscall.Errno = 376
+	ERROR_CLOUD_FILE_NOT_IN_SYNC                                              syscall.Errno = 377
+	ERROR_CLOUD_FILE_ALREADY_CONNECTED                                        syscall.Errno = 378
+	ERROR_CLOUD_FILE_NOT_SUPPORTED                                            syscall.Errno = 379
+	ERROR_CLOUD_FILE_INVALID_REQUEST                                          syscall.Errno = 380
+	ERROR_CLOUD_FILE_READ_ONLY_VOLUME                                         syscall.Errno = 381
+	ERROR_CLOUD_FILE_CONNECTED_PROVIDER_ONLY                                  syscall.Errno = 382
+	ERROR_CLOUD_FILE_VALIDATION_FAILED                                        syscall.Errno = 383
+	ERROR_SMB1_NOT_AVAILABLE                                                  syscall.Errno = 384
+	ERROR_FILE_SYSTEM_VIRTUALIZATION_INVALID_OPERATION                        syscall.Errno = 385
+	ERROR_CLOUD_FILE_AUTHENTICATION_FAILED                                    syscall.Errno = 386
+	ERROR_CLOUD_FILE_INSUFFICIENT_RESOURCES                                   syscall.Errno = 387
+	ERROR_CLOUD_FILE_NETWORK_UNAVAILABLE                                      syscall.Errno = 388
+	ERROR_CLOUD_FILE_UNSUCCESSFUL                                             syscall.Errno = 389
+	ERROR_CLOUD_FILE_NOT_UNDER_SYNC_ROOT                                      syscall.Errno = 390
+	ERROR_CLOUD_FILE_IN_USE                                                   syscall.Errno = 391
+	ERROR_CLOUD_FILE_PINNED                                                   syscall.Errno = 392
+	ERROR_CLOUD_FILE_REQUEST_ABORTED                                          syscall.Errno = 393
+	ERROR_CLOUD_FILE_PROPERTY_CORRUPT                                         syscall.Errno = 394
+	ERROR_CLOUD_FILE_ACCESS_DENIED                                            syscall.Errno = 395
+	ERROR_CLOUD_FILE_INCOMPATIBLE_HARDLINKS                                   syscall.Errno = 396
+	ERROR_CLOUD_FILE_PROPERTY_LOCK_CONFLICT                                   syscall.Errno = 397
+	ERROR_CLOUD_FILE_REQUEST_CANCELED                                         syscall.Errno = 398
+	ERROR_EXTERNAL_SYSKEY_NOT_SUPPORTED                                       syscall.Errno = 399
+	ERROR_THREAD_MODE_ALREADY_BACKGROUND                                      syscall.Errno = 400
+	ERROR_THREAD_MODE_NOT_BACKGROUND                                          syscall.Errno = 401
+	ERROR_PROCESS_MODE_ALREADY_BACKGROUND                                     syscall.Errno = 402
+	ERROR_PROCESS_MODE_NOT_BACKGROUND                                         syscall.Errno = 403
+	ERROR_CLOUD_FILE_PROVIDER_TERMINATED                                      syscall.Errno = 404
+	ERROR_NOT_A_CLOUD_SYNC_ROOT                                               syscall.Errno = 405
+	ERROR_FILE_PROTECTED_UNDER_DPL                                            syscall.Errno = 406
+	ERROR_VOLUME_NOT_CLUSTER_ALIGNED                                          syscall.Errno = 407
+	ERROR_NO_PHYSICALLY_ALIGNED_FREE_SPACE_FOUND                              syscall.Errno = 408
+	ERROR_APPX_FILE_NOT_ENCRYPTED                                             syscall.Errno = 409
+	ERROR_RWRAW_ENCRYPTED_FILE_NOT_ENCRYPTED                                  syscall.Errno = 410
+	ERROR_RWRAW_ENCRYPTED_INVALID_EDATAINFO_FILEOFFSET                        syscall.Errno = 411
+	ERROR_RWRAW_ENCRYPTED_INVALID_EDATAINFO_FILERANGE                         syscall.Errno = 412
+	ERROR_RWRAW_ENCRYPTED_INVALID_EDATAINFO_PARAMETER                         syscall.Errno = 413
+	ERROR_LINUX_SUBSYSTEM_NOT_PRESENT                                         syscall.Errno = 414
+	ERROR_FT_READ_FAILURE                                                     syscall.Errno = 415
+	ERROR_STORAGE_RESERVE_ID_INVALID                                          syscall.Errno = 416
+	ERROR_STORAGE_RESERVE_DOES_NOT_EXIST                                      syscall.Errno = 417
+	ERROR_STORAGE_RESERVE_ALREADY_EXISTS                                      syscall.Errno = 418
+	ERROR_STORAGE_RESERVE_NOT_EMPTY                                           syscall.Errno = 419
+	ERROR_NOT_A_DAX_VOLUME                                                    syscall.Errno = 420
+	ERROR_NOT_DAX_MAPPABLE                                                    syscall.Errno = 421
+	ERROR_TIME_CRITICAL_THREAD                                                syscall.Errno = 422
+	ERROR_DPL_NOT_SUPPORTED_FOR_USER                                          syscall.Errno = 423
+	ERROR_CASE_DIFFERING_NAMES_IN_DIR                                         syscall.Errno = 424
+	ERROR_CAPAUTHZ_NOT_DEVUNLOCKED                                            syscall.Errno = 450
+	ERROR_CAPAUTHZ_CHANGE_TYPE                                                syscall.Errno = 451
+	ERROR_CAPAUTHZ_NOT_PROVISIONED                                            syscall.Errno = 452
+	ERROR_CAPAUTHZ_NOT_AUTHORIZED                                             syscall.Errno = 453
+	ERROR_CAPAUTHZ_NO_POLICY                                                  syscall.Errno = 454
+	ERROR_CAPAUTHZ_DB_CORRUPTED                                               syscall.Errno = 455
+	ERROR_CAPAUTHZ_SCCD_INVALID_CATALOG                                       syscall.Errno = 456
+	ERROR_CAPAUTHZ_SCCD_NO_AUTH_ENTITY                                        syscall.Errno = 457
+	ERROR_CAPAUTHZ_SCCD_PARSE_ERROR                                           syscall.Errno = 458
+	ERROR_CAPAUTHZ_SCCD_DEV_MODE_REQUIRED                                     syscall.Errno = 459
+	ERROR_CAPAUTHZ_SCCD_NO_CAPABILITY_MATCH                                   syscall.Errno = 460
+	ERROR_PNP_QUERY_REMOVE_DEVICE_TIMEOUT                                     syscall.Errno = 480
+	ERROR_PNP_QUERY_REMOVE_RELATED_DEVICE_TIMEOUT                             syscall.Errno = 481
+	ERROR_PNP_QUERY_REMOVE_UNRELATED_DEVICE_TIMEOUT                           syscall.Errno = 482
+	ERROR_DEVICE_HARDWARE_ERROR                                               syscall.Errno = 483
+	ERROR_INVALID_ADDRESS                                                     syscall.Errno = 487
+	ERROR_VRF_CFG_ENABLED                                                     syscall.Errno = 1183
+	ERROR_PARTITION_TERMINATING                                               syscall.Errno = 1184
+	ERROR_USER_PROFILE_LOAD                                                   syscall.Errno = 500
+	ERROR_ARITHMETIC_OVERFLOW                                                 syscall.Errno = 534
+	ERROR_PIPE_CONNECTED                                                      syscall.Errno = 535
+	ERROR_PIPE_LISTENING                                                      syscall.Errno = 536
+	ERROR_VERIFIER_STOP                                                       syscall.Errno = 537
+	ERROR_ABIOS_ERROR                                                         syscall.Errno = 538
+	ERROR_WX86_WARNING                                                        syscall.Errno = 539
+	ERROR_WX86_ERROR                                                          syscall.Errno = 540
+	ERROR_TIMER_NOT_CANCELED                                                  syscall.Errno = 541
+	ERROR_UNWIND                                                              syscall.Errno = 542
+	ERROR_BAD_STACK                                                           syscall.Errno = 543
+	ERROR_INVALID_UNWIND_TARGET                                               syscall.Errno = 544
+	ERROR_INVALID_PORT_ATTRIBUTES                                             syscall.Errno = 545
+	ERROR_PORT_MESSAGE_TOO_LONG                                               syscall.Errno = 546
+	ERROR_INVALID_QUOTA_LOWER                                                 syscall.Errno = 547
+	ERROR_DEVICE_ALREADY_ATTACHED                                             syscall.Errno = 548
+	ERROR_INSTRUCTION_MISALIGNMENT                                            syscall.Errno = 549
+	ERROR_PROFILING_NOT_STARTED                                               syscall.Errno = 550
+	ERROR_PROFILING_NOT_STOPPED                                               syscall.Errno = 551
+	ERROR_COULD_NOT_INTERPRET                                                 syscall.Errno = 552
+	ERROR_PROFILING_AT_LIMIT                                                  syscall.Errno = 553
+	ERROR_CANT_WAIT                                                           syscall.Errno = 554
+	ERROR_CANT_TERMINATE_SELF                                                 syscall.Errno = 555
+	ERROR_UNEXPECTED_MM_CREATE_ERR                                            syscall.Errno = 556
+	ERROR_UNEXPECTED_MM_MAP_ERROR                                             syscall.Errno = 557
+	ERROR_UNEXPECTED_MM_EXTEND_ERR                                            syscall.Errno = 558
+	ERROR_BAD_FUNCTION_TABLE                                                  syscall.Errno = 559
+	ERROR_NO_GUID_TRANSLATION                                                 syscall.Errno = 560
+	ERROR_INVALID_LDT_SIZE                                                    syscall.Errno = 561
+	ERROR_INVALID_LDT_OFFSET                                                  syscall.Errno = 563
+	ERROR_INVALID_LDT_DESCRIPTOR                                              syscall.Errno = 564
+	ERROR_TOO_MANY_THREADS                                                    syscall.Errno = 565
+	ERROR_THREAD_NOT_IN_PROCESS                                               syscall.Errno = 566
+	ERROR_PAGEFILE_QUOTA_EXCEEDED                                             syscall.Errno = 567
+	ERROR_LOGON_SERVER_CONFLICT                                               syscall.Errno = 568
+	ERROR_SYNCHRONIZATION_REQUIRED                                            syscall.Errno = 569
+	ERROR_NET_OPEN_FAILED                                                     syscall.Errno = 570
+	ERROR_IO_PRIVILEGE_FAILED                                                 syscall.Errno = 571
+	ERROR_CONTROL_C_EXIT                                                      syscall.Errno = 572
+	ERROR_MISSING_SYSTEMFILE                                                  syscall.Errno = 573
+	ERROR_UNHANDLED_EXCEPTION                                                 syscall.Errno = 574
+	ERROR_APP_INIT_FAILURE                                                    syscall.Errno = 575
+	ERROR_PAGEFILE_CREATE_FAILED                                              syscall.Errno = 576
+	ERROR_INVALID_IMAGE_HASH                                                  syscall.Errno = 577
+	ERROR_NO_PAGEFILE                                                         syscall.Errno = 578
+	ERROR_ILLEGAL_FLOAT_CONTEXT                                               syscall.Errno = 579
+	ERROR_NO_EVENT_PAIR                                                       syscall.Errno = 580
+	ERROR_DOMAIN_CTRLR_CONFIG_ERROR                                           syscall.Errno = 581
+	ERROR_ILLEGAL_CHARACTER                                                   syscall.Errno = 582
+	ERROR_UNDEFINED_CHARACTER                                                 syscall.Errno = 583
+	ERROR_FLOPPY_VOLUME                                                       syscall.Errno = 584
+	ERROR_BIOS_FAILED_TO_CONNECT_INTERRUPT                                    syscall.Errno = 585
+	ERROR_BACKUP_CONTROLLER                                                   syscall.Errno = 586
+	ERROR_MUTANT_LIMIT_EXCEEDED                                               syscall.Errno = 587
+	ERROR_FS_DRIVER_REQUIRED                                                  syscall.Errno = 588
+	ERROR_CANNOT_LOAD_REGISTRY_FILE                                           syscall.Errno = 589
+	ERROR_DEBUG_ATTACH_FAILED                                                 syscall.Errno = 590
+	ERROR_SYSTEM_PROCESS_TERMINATED                                           syscall.Errno = 591
+	ERROR_DATA_NOT_ACCEPTED                                                   syscall.Errno = 592
+	ERROR_VDM_HARD_ERROR                                                      syscall.Errno = 593
+	ERROR_DRIVER_CANCEL_TIMEOUT                                               syscall.Errno = 594
+	ERROR_REPLY_MESSAGE_MISMATCH                                              syscall.Errno = 595
+	ERROR_LOST_WRITEBEHIND_DATA                                               syscall.Errno = 596
+	ERROR_CLIENT_SERVER_PARAMETERS_INVALID                                    syscall.Errno = 597
+	ERROR_NOT_TINY_STREAM                                                     syscall.Errno = 598
+	ERROR_STACK_OVERFLOW_READ                                                 syscall.Errno = 599
+	ERROR_CONVERT_TO_LARGE                                                    syscall.Errno = 600
+	ERROR_FOUND_OUT_OF_SCOPE                                                  syscall.Errno = 601
+	ERROR_ALLOCATE_BUCKET                                                     syscall.Errno = 602
+	ERROR_MARSHALL_OVERFLOW                                                   syscall.Errno = 603
+	ERROR_INVALID_VARIANT                                                     syscall.Errno = 604
+	ERROR_BAD_COMPRESSION_BUFFER                                              syscall.Errno = 605
+	ERROR_AUDIT_FAILED                                                        syscall.Errno = 606
+	ERROR_TIMER_RESOLUTION_NOT_SET                                            syscall.Errno = 607
+	ERROR_INSUFFICIENT_LOGON_INFO                                             syscall.Errno = 608
+	ERROR_BAD_DLL_ENTRYPOINT                                                  syscall.Errno = 609
+	ERROR_BAD_SERVICE_ENTRYPOINT                                              syscall.Errno = 610
+	ERROR_IP_ADDRESS_CONFLICT1                                                syscall.Errno = 611
+	ERROR_IP_ADDRESS_CONFLICT2                                                syscall.Errno = 612
+	ERROR_REGISTRY_QUOTA_LIMIT                                                syscall.Errno = 613
+	ERROR_NO_CALLBACK_ACTIVE                                                  syscall.Errno = 614
+	ERROR_PWD_TOO_SHORT                                                       syscall.Errno = 615
+	ERROR_PWD_TOO_RECENT                                                      syscall.Errno = 616
+	ERROR_PWD_HISTORY_CONFLICT                                                syscall.Errno = 617
+	ERROR_UNSUPPORTED_COMPRESSION                                             syscall.Errno = 618
+	ERROR_INVALID_HW_PROFILE                                                  syscall.Errno = 619
+	ERROR_INVALID_PLUGPLAY_DEVICE_PATH                                        syscall.Errno = 620
+	ERROR_QUOTA_LIST_INCONSISTENT                                             syscall.Errno = 621
+	ERROR_EVALUATION_EXPIRATION                                               syscall.Errno = 622
+	ERROR_ILLEGAL_DLL_RELOCATION                                              syscall.Errno = 623
+	ERROR_DLL_INIT_FAILED_LOGOFF                                              syscall.Errno = 624
+	ERROR_VALIDATE_CONTINUE                                                   syscall.Errno = 625
+	ERROR_NO_MORE_MATCHES                                                     syscall.Errno = 626
+	ERROR_RANGE_LIST_CONFLICT                                                 syscall.Errno = 627
+	ERROR_SERVER_SID_MISMATCH                                                 syscall.Errno = 628
+	ERROR_CANT_ENABLE_DENY_ONLY                                               syscall.Errno = 629
+	ERROR_FLOAT_MULTIPLE_FAULTS                                               syscall.Errno = 630
+	ERROR_FLOAT_MULTIPLE_TRAPS                                                syscall.Errno = 631
+	ERROR_NOINTERFACE                                                         syscall.Errno = 632
+	ERROR_DRIVER_FAILED_SLEEP                                                 syscall.Errno = 633
+	ERROR_CORRUPT_SYSTEM_FILE                                                 syscall.Errno = 634
+	ERROR_COMMITMENT_MINIMUM                                                  syscall.Errno = 635
+	ERROR_PNP_RESTART_ENUMERATION                                             syscall.Errno = 636
+	ERROR_SYSTEM_IMAGE_BAD_SIGNATURE                                          syscall.Errno = 637
+	ERROR_PNP_REBOOT_REQUIRED                                                 syscall.Errno = 638
+	ERROR_INSUFFICIENT_POWER                                                  syscall.Errno = 639
+	ERROR_MULTIPLE_FAULT_VIOLATION                                            syscall.Errno = 640
+	ERROR_SYSTEM_SHUTDOWN                                                     syscall.Errno = 641
+	ERROR_PORT_NOT_SET                                                        syscall.Errno = 642
+	ERROR_DS_VERSION_CHECK_FAILURE                                            syscall.Errno = 643
+	ERROR_RANGE_NOT_FOUND                                                     syscall.Errno = 644
+	ERROR_NOT_SAFE_MODE_DRIVER                                                syscall.Errno = 646
+	ERROR_FAILED_DRIVER_ENTRY                                                 syscall.Errno = 647
+	ERROR_DEVICE_ENUMERATION_ERROR                                            syscall.Errno = 648
+	ERROR_MOUNT_POINT_NOT_RESOLVED                                            syscall.Errno = 649
+	ERROR_INVALID_DEVICE_OBJECT_PARAMETER                                     syscall.Errno = 650
+	ERROR_MCA_OCCURED                                                         syscall.Errno = 651
+	ERROR_DRIVER_DATABASE_ERROR                                               syscall.Errno = 652
+	ERROR_SYSTEM_HIVE_TOO_LARGE                                               syscall.Errno = 653
+	ERROR_DRIVER_FAILED_PRIOR_UNLOAD                                          syscall.Errno = 654
+	ERROR_VOLSNAP_PREPARE_HIBERNATE                                           syscall.Errno = 655
+	ERROR_HIBERNATION_FAILURE                                                 syscall.Errno = 656
+	ERROR_PWD_TOO_LONG                                                        syscall.Errno = 657
+	ERROR_FILE_SYSTEM_LIMITATION                                              syscall.Errno = 665
+	ERROR_ASSERTION_FAILURE                                                   syscall.Errno = 668
+	ERROR_ACPI_ERROR                                                          syscall.Errno = 669
+	ERROR_WOW_ASSERTION                                                       syscall.Errno = 670
+	ERROR_PNP_BAD_MPS_TABLE                                                   syscall.Errno = 671
+	ERROR_PNP_TRANSLATION_FAILED                                              syscall.Errno = 672
+	ERROR_PNP_IRQ_TRANSLATION_FAILED                                          syscall.Errno = 673
+	ERROR_PNP_INVALID_ID                                                      syscall.Errno = 674
+	ERROR_WAKE_SYSTEM_DEBUGGER                                                syscall.Errno = 675
+	ERROR_HANDLES_CLOSED                                                      syscall.Errno = 676
+	ERROR_EXTRANEOUS_INFORMATION                                              syscall.Errno = 677
+	ERROR_RXACT_COMMIT_NECESSARY                                              syscall.Errno = 678
+	ERROR_MEDIA_CHECK                                                         syscall.Errno = 679
+	ERROR_GUID_SUBSTITUTION_MADE                                              syscall.Errno = 680
+	ERROR_STOPPED_ON_SYMLINK                                                  syscall.Errno = 681
+	ERROR_LONGJUMP                                                            syscall.Errno = 682
+	ERROR_PLUGPLAY_QUERY_VETOED                                               syscall.Errno = 683
+	ERROR_UNWIND_CONSOLIDATE                                                  syscall.Errno = 684
+	ERROR_REGISTRY_HIVE_RECOVERED                                             syscall.Errno = 685
+	ERROR_DLL_MIGHT_BE_INSECURE                                               syscall.Errno = 686
+	ERROR_DLL_MIGHT_BE_INCOMPATIBLE                                           syscall.Errno = 687
+	ERROR_DBG_EXCEPTION_NOT_HANDLED                                           syscall.Errno = 688
+	ERROR_DBG_REPLY_LATER                                                     syscall.Errno = 689
+	ERROR_DBG_UNABLE_TO_PROVIDE_HANDLE                                        syscall.Errno = 690
+	ERROR_DBG_TERMINATE_THREAD                                                syscall.Errno = 691
+	ERROR_DBG_TERMINATE_PROCESS                                               syscall.Errno = 692
+	ERROR_DBG_CONTROL_C                                                       syscall.Errno = 693
+	ERROR_DBG_PRINTEXCEPTION_C                                                syscall.Errno = 694
+	ERROR_DBG_RIPEXCEPTION                                                    syscall.Errno = 695
+	ERROR_DBG_CONTROL_BREAK                                                   syscall.Errno = 696
+	ERROR_DBG_COMMAND_EXCEPTION                                               syscall.Errno = 697
+	ERROR_OBJECT_NAME_EXISTS                                                  syscall.Errno = 698
+	ERROR_THREAD_WAS_SUSPENDED                                                syscall.Errno = 699
+	ERROR_IMAGE_NOT_AT_BASE                                                   syscall.Errno = 700
+	ERROR_RXACT_STATE_CREATED                                                 syscall.Errno = 701
+	ERROR_SEGMENT_NOTIFICATION                                                syscall.Errno = 702
+	ERROR_BAD_CURRENT_DIRECTORY                                               syscall.Errno = 703
+	ERROR_FT_READ_RECOVERY_FROM_BACKUP                                        syscall.Errno = 704
+	ERROR_FT_WRITE_RECOVERY                                                   syscall.Errno = 705
+	ERROR_IMAGE_MACHINE_TYPE_MISMATCH                                         syscall.Errno = 706
+	ERROR_RECEIVE_PARTIAL                                                     syscall.Errno = 707
+	ERROR_RECEIVE_EXPEDITED                                                   syscall.Errno = 708
+	ERROR_RECEIVE_PARTIAL_EXPEDITED                                           syscall.Errno = 709
+	ERROR_EVENT_DONE                                                          syscall.Errno = 710
+	ERROR_EVENT_PENDING                                                       syscall.Errno = 711
+	ERROR_CHECKING_FILE_SYSTEM                                                syscall.Errno = 712
+	ERROR_FATAL_APP_EXIT                                                      syscall.Errno = 713
+	ERROR_PREDEFINED_HANDLE                                                   syscall.Errno = 714
+	ERROR_WAS_UNLOCKED                                                        syscall.Errno = 715
+	ERROR_SERVICE_NOTIFICATION                                                syscall.Errno = 716
+	ERROR_WAS_LOCKED                                                          syscall.Errno = 717
+	ERROR_LOG_HARD_ERROR                                                      syscall.Errno = 718
+	ERROR_ALREADY_WIN32                                                       syscall.Errno = 719
+	ERROR_IMAGE_MACHINE_TYPE_MISMATCH_EXE                                     syscall.Errno = 720
+	ERROR_NO_YIELD_PERFORMED                                                  syscall.Errno = 721
+	ERROR_TIMER_RESUME_IGNORED                                                syscall.Errno = 722
+	ERROR_ARBITRATION_UNHANDLED                                               syscall.Errno = 723
+	ERROR_CARDBUS_NOT_SUPPORTED                                               syscall.Errno = 724
+	ERROR_MP_PROCESSOR_MISMATCH                                               syscall.Errno = 725
+	ERROR_HIBERNATED                                                          syscall.Errno = 726
+	ERROR_RESUME_HIBERNATION                                                  syscall.Errno = 727
+	ERROR_FIRMWARE_UPDATED                                                    syscall.Errno = 728
+	ERROR_DRIVERS_LEAKING_LOCKED_PAGES                                        syscall.Errno = 729
+	ERROR_WAKE_SYSTEM                                                         syscall.Errno = 730
+	ERROR_WAIT_1                                                              syscall.Errno = 731
+	ERROR_WAIT_2                                                              syscall.Errno = 732
+	ERROR_WAIT_3                                                              syscall.Errno = 733
+	ERROR_WAIT_63                                                             syscall.Errno = 734
+	ERROR_ABANDONED_WAIT_0                                                    syscall.Errno = 735
+	ERROR_ABANDONED_WAIT_63                                                   syscall.Errno = 736
+	ERROR_USER_APC                                                            syscall.Errno = 737
+	ERROR_KERNEL_APC                                                          syscall.Errno = 738
+	ERROR_ALERTED                                                             syscall.Errno = 739
+	ERROR_ELEVATION_REQUIRED                                                  syscall.Errno = 740
+	ERROR_REPARSE                                                             syscall.Errno = 741
+	ERROR_OPLOCK_BREAK_IN_PROGRESS                                            syscall.Errno = 742
+	ERROR_VOLUME_MOUNTED                                                      syscall.Errno = 743
+	ERROR_RXACT_COMMITTED                                                     syscall.Errno = 744
+	ERROR_NOTIFY_CLEANUP                                                      syscall.Errno = 745
+	ERROR_PRIMARY_TRANSPORT_CONNECT_FAILED                                    syscall.Errno = 746
+	ERROR_PAGE_FAULT_TRANSITION                                               syscall.Errno = 747
+	ERROR_PAGE_FAULT_DEMAND_ZERO                                              syscall.Errno = 748
+	ERROR_PAGE_FAULT_COPY_ON_WRITE                                            syscall.Errno = 749
+	ERROR_PAGE_FAULT_GUARD_PAGE                                               syscall.Errno = 750
+	ERROR_PAGE_FAULT_PAGING_FILE                                              syscall.Errno = 751
+	ERROR_CACHE_PAGE_LOCKED                                                   syscall.Errno = 752
+	ERROR_CRASH_DUMP                                                          syscall.Errno = 753
+	ERROR_BUFFER_ALL_ZEROS                                                    syscall.Errno = 754
+	ERROR_REPARSE_OBJECT                                                      syscall.Errno = 755
+	ERROR_RESOURCE_REQUIREMENTS_CHANGED                                       syscall.Errno = 756
+	ERROR_TRANSLATION_COMPLETE                                                syscall.Errno = 757
+	ERROR_NOTHING_TO_TERMINATE                                                syscall.Errno = 758
+	ERROR_PROCESS_NOT_IN_JOB                                                  syscall.Errno = 759
+	ERROR_PROCESS_IN_JOB                                                      syscall.Errno = 760
+	ERROR_VOLSNAP_HIBERNATE_READY                                             syscall.Errno = 761
+	ERROR_FSFILTER_OP_COMPLETED_SUCCESSFULLY                                  syscall.Errno = 762
+	ERROR_INTERRUPT_VECTOR_ALREADY_CONNECTED                                  syscall.Errno = 763
+	ERROR_INTERRUPT_STILL_CONNECTED                                           syscall.Errno = 764
+	ERROR_WAIT_FOR_OPLOCK                                                     syscall.Errno = 765
+	ERROR_DBG_EXCEPTION_HANDLED                                               syscall.Errno = 766
+	ERROR_DBG_CONTINUE                                                        syscall.Errno = 767
+	ERROR_CALLBACK_POP_STACK                                                  syscall.Errno = 768
+	ERROR_COMPRESSION_DISABLED                                                syscall.Errno = 769
+	ERROR_CANTFETCHBACKWARDS                                                  syscall.Errno = 770
+	ERROR_CANTSCROLLBACKWARDS                                                 syscall.Errno = 771
+	ERROR_ROWSNOTRELEASED                                                     syscall.Errno = 772
+	ERROR_BAD_ACCESSOR_FLAGS                                                  syscall.Errno = 773
+	ERROR_ERRORS_ENCOUNTERED                                                  syscall.Errno = 774
+	ERROR_NOT_CAPABLE                                                         syscall.Errno = 775
+	ERROR_REQUEST_OUT_OF_SEQUENCE                                             syscall.Errno = 776
+	ERROR_VERSION_PARSE_ERROR                                                 syscall.Errno = 777
+	ERROR_BADSTARTPOSITION                                                    syscall.Errno = 778
+	ERROR_MEMORY_HARDWARE                                                     syscall.Errno = 779
+	ERROR_DISK_REPAIR_DISABLED                                                syscall.Errno = 780
+	ERROR_INSUFFICIENT_RESOURCE_FOR_SPECIFIED_SHARED_SECTION_SIZE             syscall.Errno = 781
+	ERROR_SYSTEM_POWERSTATE_TRANSITION                                        syscall.Errno = 782
+	ERROR_SYSTEM_POWERSTATE_COMPLEX_TRANSITION                                syscall.Errno = 783
+	ERROR_MCA_EXCEPTION                                                       syscall.Errno = 784
+	ERROR_ACCESS_AUDIT_BY_POLICY                                              syscall.Errno = 785
+	ERROR_ACCESS_DISABLED_NO_SAFER_UI_BY_POLICY                               syscall.Errno = 786
+	ERROR_ABANDON_HIBERFILE                                                   syscall.Errno = 787
+	ERROR_LOST_WRITEBEHIND_DATA_NETWORK_DISCONNECTED                          syscall.Errno = 788
+	ERROR_LOST_WRITEBEHIND_DATA_NETWORK_SERVER_ERROR                          syscall.Errno = 789
+	ERROR_LOST_WRITEBEHIND_DATA_LOCAL_DISK_ERROR                              syscall.Errno = 790
+	ERROR_BAD_MCFG_TABLE                                                      syscall.Errno = 791
+	ERROR_DISK_REPAIR_REDIRECTED                                              syscall.Errno = 792
+	ERROR_DISK_REPAIR_UNSUCCESSFUL                                            syscall.Errno = 793
+	ERROR_CORRUPT_LOG_OVERFULL                                                syscall.Errno = 794
+	ERROR_CORRUPT_LOG_CORRUPTED                                               syscall.Errno = 795
+	ERROR_CORRUPT_LOG_UNAVAILABLE                                             syscall.Errno = 796
+	ERROR_CORRUPT_LOG_DELETED_FULL                                            syscall.Errno = 797
+	ERROR_CORRUPT_LOG_CLEARED                                                 syscall.Errno = 798
+	ERROR_ORPHAN_NAME_EXHAUSTED                                               syscall.Errno = 799
+	ERROR_OPLOCK_SWITCHED_TO_NEW_HANDLE                                       syscall.Errno = 800
+	ERROR_CANNOT_GRANT_REQUESTED_OPLOCK                                       syscall.Errno = 801
+	ERROR_CANNOT_BREAK_OPLOCK                                                 syscall.Errno = 802
+	ERROR_OPLOCK_HANDLE_CLOSED                                                syscall.Errno = 803
+	ERROR_NO_ACE_CONDITION                                                    syscall.Errno = 804
+	ERROR_INVALID_ACE_CONDITION                                               syscall.Errno = 805
+	ERROR_FILE_HANDLE_REVOKED                                                 syscall.Errno = 806
+	ERROR_IMAGE_AT_DIFFERENT_BASE                                             syscall.Errno = 807
+	ERROR_ENCRYPTED_IO_NOT_POSSIBLE                                           syscall.Errno = 808
+	ERROR_FILE_METADATA_OPTIMIZATION_IN_PROGRESS                              syscall.Errno = 809
+	ERROR_QUOTA_ACTIVITY                                                      syscall.Errno = 810
+	ERROR_HANDLE_REVOKED                                                      syscall.Errno = 811
+	ERROR_CALLBACK_INVOKE_INLINE                                              syscall.Errno = 812
+	ERROR_CPU_SET_INVALID                                                     syscall.Errno = 813
+	ERROR_ENCLAVE_NOT_TERMINATED                                              syscall.Errno = 814
+	ERROR_ENCLAVE_VIOLATION                                                   syscall.Errno = 815
+	ERROR_EA_ACCESS_DENIED                                                    syscall.Errno = 994
+	ERROR_OPERATION_ABORTED                                                   syscall.Errno = 995
+	ERROR_IO_INCOMPLETE                                                       syscall.Errno = 996
+	ERROR_IO_PENDING                                                          syscall.Errno = 997
+	ERROR_NOACCESS                                                            syscall.Errno = 998
+	ERROR_SWAPERROR                                                           syscall.Errno = 999
+	ERROR_STACK_OVERFLOW                                                      syscall.Errno = 1001
+	ERROR_INVALID_MESSAGE                                                     syscall.Errno = 1002
+	ERROR_CAN_NOT_COMPLETE                                                    syscall.Errno = 1003
+	ERROR_INVALID_FLAGS                                                       syscall.Errno = 1004
+	ERROR_UNRECOGNIZED_VOLUME                                                 syscall.Errno = 1005
+	ERROR_FILE_INVALID                                                        syscall.Errno = 1006
+	ERROR_FULLSCREEN_MODE                                                     syscall.Errno = 1007
+	ERROR_NO_TOKEN                                                            syscall.Errno = 1008
+	ERROR_BADDB                                                               syscall.Errno = 1009
+	ERROR_BADKEY                                                              syscall.Errno = 1010
+	ERROR_CANTOPEN                                                            syscall.Errno = 1011
+	ERROR_CANTREAD                                                            syscall.Errno = 1012
+	ERROR_CANTWRITE                                                           syscall.Errno = 1013
+	ERROR_REGISTRY_RECOVERED                                                  syscall.Errno = 1014
+	ERROR_REGISTRY_CORRUPT                                                    syscall.Errno = 1015
+	ERROR_REGISTRY_IO_FAILED                                                  syscall.Errno = 1016
+	ERROR_NOT_REGISTRY_FILE                                                   syscall.Errno = 1017
+	ERROR_KEY_DELETED                                                         syscall.Errno = 1018
+	ERROR_NO_LOG_SPACE                                                        syscall.Errno = 1019
+	ERROR_KEY_HAS_CHILDREN                                                    syscall.Errno = 1020
+	ERROR_CHILD_MUST_BE_VOLATILE                                              syscall.Errno = 1021
+	ERROR_NOTIFY_ENUM_DIR                                                     syscall.Errno = 1022
+	ERROR_DEPENDENT_SERVICES_RUNNING                                          syscall.Errno = 1051
+	ERROR_INVALID_SERVICE_CONTROL                                             syscall.Errno = 1052
+	ERROR_SERVICE_REQUEST_TIMEOUT                                             syscall.Errno = 1053
+	ERROR_SERVICE_NO_THREAD                                                   syscall.Errno = 1054
+	ERROR_SERVICE_DATABASE_LOCKED                                             syscall.Errno = 1055
+	ERROR_SERVICE_ALREADY_RUNNING                                             syscall.Errno = 1056
+	ERROR_INVALID_SERVICE_ACCOUNT                                             syscall.Errno = 1057
+	ERROR_SERVICE_DISABLED                                                    syscall.Errno = 1058
+	ERROR_CIRCULAR_DEPENDENCY                                                 syscall.Errno = 1059
+	ERROR_SERVICE_DOES_NOT_EXIST                                              syscall.Errno = 1060
+	ERROR_SERVICE_CANNOT_ACCEPT_CTRL                                          syscall.Errno = 1061
+	ERROR_SERVICE_NOT_ACTIVE                                                  syscall.Errno = 1062
+	ERROR_FAILED_SERVICE_CONTROLLER_CONNECT                                   syscall.Errno = 1063
+	ERROR_EXCEPTION_IN_SERVICE                                                syscall.Errno = 1064
+	ERROR_DATABASE_DOES_NOT_EXIST                                             syscall.Errno = 1065
+	ERROR_SERVICE_SPECIFIC_ERROR                                              syscall.Errno = 1066
+	ERROR_PROCESS_ABORTED                                                     syscall.Errno = 1067
+	ERROR_SERVICE_DEPENDENCY_FAIL                                             syscall.Errno = 1068
+	ERROR_SERVICE_LOGON_FAILED                                                syscall.Errno = 1069
+	ERROR_SERVICE_START_HANG                                                  syscall.Errno = 1070
+	ERROR_INVALID_SERVICE_LOCK                                                syscall.Errno = 1071
+	ERROR_SERVICE_MARKED_FOR_DELETE                                           syscall.Errno = 1072
+	ERROR_SERVICE_EXISTS                                                      syscall.Errno = 1073
+	ERROR_ALREADY_RUNNING_LKG                                                 syscall.Errno = 1074
+	ERROR_SERVICE_DEPENDENCY_DELETED                                          syscall.Errno = 1075
+	ERROR_BOOT_ALREADY_ACCEPTED                                               syscall.Errno = 1076
+	ERROR_SERVICE_NEVER_STARTED                                               syscall.Errno = 1077
+	ERROR_DUPLICATE_SERVICE_NAME                                              syscall.Errno = 1078
+	ERROR_DIFFERENT_SERVICE_ACCOUNT                                           syscall.Errno = 1079
+	ERROR_CANNOT_DETECT_DRIVER_FAILURE                                        syscall.Errno = 1080
+	ERROR_CANNOT_DETECT_PROCESS_ABORT                                         syscall.Errno = 1081
+	ERROR_NO_RECOVERY_PROGRAM                                                 syscall.Errno = 1082
+	ERROR_SERVICE_NOT_IN_EXE                                                  syscall.Errno = 1083
+	ERROR_NOT_SAFEBOOT_SERVICE                                                syscall.Errno = 1084
+	ERROR_END_OF_MEDIA                                                        syscall.Errno = 1100
+	ERROR_FILEMARK_DETECTED                                                   syscall.Errno = 1101
+	ERROR_BEGINNING_OF_MEDIA                                                  syscall.Errno = 1102
+	ERROR_SETMARK_DETECTED                                                    syscall.Errno = 1103
+	ERROR_NO_DATA_DETECTED                                                    syscall.Errno = 1104
+	ERROR_PARTITION_FAILURE                                                   syscall.Errno = 1105
+	ERROR_INVALID_BLOCK_LENGTH                                                syscall.Errno = 1106
+	ERROR_DEVICE_NOT_PARTITIONED                                              syscall.Errno = 1107
+	ERROR_UNABLE_TO_LOCK_MEDIA                                                syscall.Errno = 1108
+	ERROR_UNABLE_TO_UNLOAD_MEDIA                                              syscall.Errno = 1109
+	ERROR_MEDIA_CHANGED                                                       syscall.Errno = 1110
+	ERROR_BUS_RESET                                                           syscall.Errno = 1111
+	ERROR_NO_MEDIA_IN_DRIVE                                                   syscall.Errno = 1112
+	ERROR_NO_UNICODE_TRANSLATION                                              syscall.Errno = 1113
+	ERROR_DLL_INIT_FAILED                                                     syscall.Errno = 1114
+	ERROR_SHUTDOWN_IN_PROGRESS                                                syscall.Errno = 1115
+	ERROR_NO_SHUTDOWN_IN_PROGRESS                                             syscall.Errno = 1116
+	ERROR_IO_DEVICE                                                           syscall.Errno = 1117
+	ERROR_SERIAL_NO_DEVICE                                                    syscall.Errno = 1118
+	ERROR_IRQ_BUSY                                                            syscall.Errno = 1119
+	ERROR_MORE_WRITES                                                         syscall.Errno = 1120
+	ERROR_COUNTER_TIMEOUT                                                     syscall.Errno = 1121
+	ERROR_FLOPPY_ID_MARK_NOT_FOUND                                            syscall.Errno = 1122
+	ERROR_FLOPPY_WRONG_CYLINDER                                               syscall.Errno = 1123
+	ERROR_FLOPPY_UNKNOWN_ERROR                                                syscall.Errno = 1124
+	ERROR_FLOPPY_BAD_REGISTERS                                                syscall.Errno = 1125
+	ERROR_DISK_RECALIBRATE_FAILED                                             syscall.Errno = 1126
+	ERROR_DISK_OPERATION_FAILED                                               syscall.Errno = 1127
+	ERROR_DISK_RESET_FAILED                                                   syscall.Errno = 1128
+	ERROR_EOM_OVERFLOW                                                        syscall.Errno = 1129
+	ERROR_NOT_ENOUGH_SERVER_MEMORY                                            syscall.Errno = 1130
+	ERROR_POSSIBLE_DEADLOCK                                                   syscall.Errno = 1131
+	ERROR_MAPPED_ALIGNMENT                                                    syscall.Errno = 1132
+	ERROR_SET_POWER_STATE_VETOED                                              syscall.Errno = 1140
+	ERROR_SET_POWER_STATE_FAILED                                              syscall.Errno = 1141
+	ERROR_TOO_MANY_LINKS                                                      syscall.Errno = 1142
+	ERROR_OLD_WIN_VERSION                                                     syscall.Errno = 1150
+	ERROR_APP_WRONG_OS                                                        syscall.Errno = 1151
+	ERROR_SINGLE_INSTANCE_APP                                                 syscall.Errno = 1152
+	ERROR_RMODE_APP                                                           syscall.Errno = 1153
+	ERROR_INVALID_DLL                                                         syscall.Errno = 1154
+	ERROR_NO_ASSOCIATION                                                      syscall.Errno = 1155
+	ERROR_DDE_FAIL                                                            syscall.Errno = 1156
+	ERROR_DLL_NOT_FOUND                                                       syscall.Errno = 1157
+	ERROR_NO_MORE_USER_HANDLES                                                syscall.Errno = 1158
+	ERROR_MESSAGE_SYNC_ONLY                                                   syscall.Errno = 1159
+	ERROR_SOURCE_ELEMENT_EMPTY                                                syscall.Errno = 1160
+	ERROR_DESTINATION_ELEMENT_FULL                                            syscall.Errno = 1161
+	ERROR_ILLEGAL_ELEMENT_ADDRESS                                             syscall.Errno = 1162
+	ERROR_MAGAZINE_NOT_PRESENT                                                syscall.Errno = 1163
+	ERROR_DEVICE_REINITIALIZATION_NEEDED                                      syscall.Errno = 1164
+	ERROR_DEVICE_REQUIRES_CLEANING                                            syscall.Errno = 1165
+	ERROR_DEVICE_DOOR_OPEN                                                    syscall.Errno = 1166
+	ERROR_DEVICE_NOT_CONNECTED                                                syscall.Errno = 1167
+	ERROR_NOT_FOUND                                                           syscall.Errno = 1168
+	ERROR_NO_MATCH                                                            syscall.Errno = 1169
+	ERROR_SET_NOT_FOUND                                                       syscall.Errno = 1170
+	ERROR_POINT_NOT_FOUND                                                     syscall.Errno = 1171
+	ERROR_NO_TRACKING_SERVICE                                                 syscall.Errno = 1172
+	ERROR_NO_VOLUME_ID                                                        syscall.Errno = 1173
+	ERROR_UNABLE_TO_REMOVE_REPLACED                                           syscall.Errno = 1175
+	ERROR_UNABLE_TO_MOVE_REPLACEMENT                                          syscall.Errno = 1176
+	ERROR_UNABLE_TO_MOVE_REPLACEMENT_2                                        syscall.Errno = 1177
+	ERROR_JOURNAL_DELETE_IN_PROGRESS                                          syscall.Errno = 1178
+	ERROR_JOURNAL_NOT_ACTIVE                                                  syscall.Errno = 1179
+	ERROR_POTENTIAL_FILE_FOUND                                                syscall.Errno = 1180
+	ERROR_JOURNAL_ENTRY_DELETED                                               syscall.Errno = 1181
+	ERROR_SHUTDOWN_IS_SCHEDULED                                               syscall.Errno = 1190
+	ERROR_SHUTDOWN_USERS_LOGGED_ON                                            syscall.Errno = 1191
+	ERROR_BAD_DEVICE                                                          syscall.Errno = 1200
+	ERROR_CONNECTION_UNAVAIL                                                  syscall.Errno = 1201
+	ERROR_DEVICE_ALREADY_REMEMBERED                                           syscall.Errno = 1202
+	ERROR_NO_NET_OR_BAD_PATH                                                  syscall.Errno = 1203
+	ERROR_BAD_PROVIDER                                                        syscall.Errno = 1204
+	ERROR_CANNOT_OPEN_PROFILE                                                 syscall.Errno = 1205
+	ERROR_BAD_PROFILE                                                         syscall.Errno = 1206
+	ERROR_NOT_CONTAINER                                                       syscall.Errno = 1207
+	ERROR_EXTENDED_ERROR                                                      syscall.Errno = 1208
+	ERROR_INVALID_GROUPNAME                                                   syscall.Errno = 1209
+	ERROR_INVALID_COMPUTERNAME                                                syscall.Errno = 1210
+	ERROR_INVALID_EVENTNAME                                                   syscall.Errno = 1211
+	ERROR_INVALID_DOMAINNAME                                                  syscall.Errno = 1212
+	ERROR_INVALID_SERVICENAME                                                 syscall.Errno = 1213
+	ERROR_INVALID_NETNAME                                                     syscall.Errno = 1214
+	ERROR_INVALID_SHARENAME                                                   syscall.Errno = 1215
+	ERROR_INVALID_PASSWORDNAME                                                syscall.Errno = 1216
+	ERROR_INVALID_MESSAGENAME                                                 syscall.Errno = 1217
+	ERROR_INVALID_MESSAGEDEST                                                 syscall.Errno = 1218
+	ERROR_SESSION_CREDENTIAL_CONFLICT                                         syscall.Errno = 1219
+	ERROR_REMOTE_SESSION_LIMIT_EXCEEDED                                       syscall.Errno = 1220
+	ERROR_DUP_DOMAINNAME                                                      syscall.Errno = 1221
+	ERROR_NO_NETWORK                                                          syscall.Errno = 1222
+	ERROR_CANCELLED                                                           syscall.Errno = 1223
+	ERROR_USER_MAPPED_FILE                                                    syscall.Errno = 1224
+	ERROR_CONNECTION_REFUSED                                                  syscall.Errno = 1225
+	ERROR_GRACEFUL_DISCONNECT                                                 syscall.Errno = 1226
+	ERROR_ADDRESS_ALREADY_ASSOCIATED                                          syscall.Errno = 1227
+	ERROR_ADDRESS_NOT_ASSOCIATED                                              syscall.Errno = 1228
+	ERROR_CONNECTION_INVALID                                                  syscall.Errno = 1229
+	ERROR_CONNECTION_ACTIVE                                                   syscall.Errno = 1230
+	ERROR_NETWORK_UNREACHABLE                                                 syscall.Errno = 1231
+	ERROR_HOST_UNREACHABLE                                                    syscall.Errno = 1232
+	ERROR_PROTOCOL_UNREACHABLE                                                syscall.Errno = 1233
+	ERROR_PORT_UNREACHABLE                                                    syscall.Errno = 1234
+	ERROR_REQUEST_ABORTED                                                     syscall.Errno = 1235
+	ERROR_CONNECTION_ABORTED                                                  syscall.Errno = 1236
+	ERROR_RETRY                                                               syscall.Errno = 1237
+	ERROR_CONNECTION_COUNT_LIMIT                                              syscall.Errno = 1238
+	ERROR_LOGIN_TIME_RESTRICTION                                              syscall.Errno = 1239
+	ERROR_LOGIN_WKSTA_RESTRICTION                                             syscall.Errno = 1240
+	ERROR_INCORRECT_ADDRESS                                                   syscall.Errno = 1241
+	ERROR_ALREADY_REGISTERED                                                  syscall.Errno = 1242
+	ERROR_SERVICE_NOT_FOUND                                                   syscall.Errno = 1243
+	ERROR_NOT_AUTHENTICATED                                                   syscall.Errno = 1244
+	ERROR_NOT_LOGGED_ON                                                       syscall.Errno = 1245
+	ERROR_CONTINUE                                                            syscall.Errno = 1246
+	ERROR_ALREADY_INITIALIZED                                                 syscall.Errno = 1247
+	ERROR_NO_MORE_DEVICES                                                     syscall.Errno = 1248
+	ERROR_NO_SUCH_SITE                                                        syscall.Errno = 1249
+	ERROR_DOMAIN_CONTROLLER_EXISTS                                            syscall.Errno = 1250
+	ERROR_ONLY_IF_CONNECTED                                                   syscall.Errno = 1251
+	ERROR_OVERRIDE_NOCHANGES                                                  syscall.Errno = 1252
+	ERROR_BAD_USER_PROFILE                                                    syscall.Errno = 1253
+	ERROR_NOT_SUPPORTED_ON_SBS                                                syscall.Errno = 1254
+	ERROR_SERVER_SHUTDOWN_IN_PROGRESS                                         syscall.Errno = 1255
+	ERROR_HOST_DOWN                                                           syscall.Errno = 1256
+	ERROR_NON_ACCOUNT_SID                                                     syscall.Errno = 1257
+	ERROR_NON_DOMAIN_SID                                                      syscall.Errno = 1258
+	ERROR_APPHELP_BLOCK                                                       syscall.Errno = 1259
+	ERROR_ACCESS_DISABLED_BY_POLICY                                           syscall.Errno = 1260
+	ERROR_REG_NAT_CONSUMPTION                                                 syscall.Errno = 1261
+	ERROR_CSCSHARE_OFFLINE                                                    syscall.Errno = 1262
+	ERROR_PKINIT_FAILURE                                                      syscall.Errno = 1263
+	ERROR_SMARTCARD_SUBSYSTEM_FAILURE                                         syscall.Errno = 1264
+	ERROR_DOWNGRADE_DETECTED                                                  syscall.Errno = 1265
+	ERROR_MACHINE_LOCKED                                                      syscall.Errno = 1271
+	ERROR_SMB_GUEST_LOGON_BLOCKED                                             syscall.Errno = 1272
+	ERROR_CALLBACK_SUPPLIED_INVALID_DATA                                      syscall.Errno = 1273
+	ERROR_SYNC_FOREGROUND_REFRESH_REQUIRED                                    syscall.Errno = 1274
+	ERROR_DRIVER_BLOCKED                                                      syscall.Errno = 1275
+	ERROR_INVALID_IMPORT_OF_NON_DLL                                           syscall.Errno = 1276
+	ERROR_ACCESS_DISABLED_WEBBLADE                                            syscall.Errno = 1277
+	ERROR_ACCESS_DISABLED_WEBBLADE_TAMPER                                     syscall.Errno = 1278
+	ERROR_RECOVERY_FAILURE                                                    syscall.Errno = 1279
+	ERROR_ALREADY_FIBER                                                       syscall.Errno = 1280
+	ERROR_ALREADY_THREAD                                                      syscall.Errno = 1281
+	ERROR_STACK_BUFFER_OVERRUN                                                syscall.Errno = 1282
+	ERROR_PARAMETER_QUOTA_EXCEEDED                                            syscall.Errno = 1283
+	ERROR_DEBUGGER_INACTIVE                                                   syscall.Errno = 1284
+	ERROR_DELAY_LOAD_FAILED                                                   syscall.Errno = 1285
+	ERROR_VDM_DISALLOWED                                                      syscall.Errno = 1286
+	ERROR_UNIDENTIFIED_ERROR                                                  syscall.Errno = 1287
+	ERROR_INVALID_CRUNTIME_PARAMETER                                          syscall.Errno = 1288
+	ERROR_BEYOND_VDL                                                          syscall.Errno = 1289
+	ERROR_INCOMPATIBLE_SERVICE_SID_TYPE                                       syscall.Errno = 1290
+	ERROR_DRIVER_PROCESS_TERMINATED                                           syscall.Errno = 1291
+	ERROR_IMPLEMENTATION_LIMIT                                                syscall.Errno = 1292
+	ERROR_PROCESS_IS_PROTECTED                                                syscall.Errno = 1293
+	ERROR_SERVICE_NOTIFY_CLIENT_LAGGING                                       syscall.Errno = 1294
+	ERROR_DISK_QUOTA_EXCEEDED                                                 syscall.Errno = 1295
+	ERROR_CONTENT_BLOCKED                                                     syscall.Errno = 1296
+	ERROR_INCOMPATIBLE_SERVICE_PRIVILEGE                                      syscall.Errno = 1297
+	ERROR_APP_HANG                                                            syscall.Errno = 1298
+	ERROR_INVALID_LABEL                                                       syscall.Errno = 1299
+	ERROR_NOT_ALL_ASSIGNED                                                    syscall.Errno = 1300
+	ERROR_SOME_NOT_MAPPED                                                     syscall.Errno = 1301
+	ERROR_NO_QUOTAS_FOR_ACCOUNT                                               syscall.Errno = 1302
+	ERROR_LOCAL_USER_SESSION_KEY                                              syscall.Errno = 1303
+	ERROR_NULL_LM_PASSWORD                                                    syscall.Errno = 1304
+	ERROR_UNKNOWN_REVISION                                                    syscall.Errno = 1305
+	ERROR_REVISION_MISMATCH                                                   syscall.Errno = 1306
+	ERROR_INVALID_OWNER                                                       syscall.Errno = 1307
+	ERROR_INVALID_PRIMARY_GROUP                                               syscall.Errno = 1308
+	ERROR_NO_IMPERSONATION_TOKEN                                              syscall.Errno = 1309
+	ERROR_CANT_DISABLE_MANDATORY                                              syscall.Errno = 1310
+	ERROR_NO_LOGON_SERVERS                                                    syscall.Errno = 1311
+	ERROR_NO_SUCH_LOGON_SESSION                                               syscall.Errno = 1312
+	ERROR_NO_SUCH_PRIVILEGE                                                   syscall.Errno = 1313
+	ERROR_PRIVILEGE_NOT_HELD                                                  syscall.Errno = 1314
+	ERROR_INVALID_ACCOUNT_NAME                                                syscall.Errno = 1315
+	ERROR_USER_EXISTS                                                         syscall.Errno = 1316
+	ERROR_NO_SUCH_USER                                                        syscall.Errno = 1317
+	ERROR_GROUP_EXISTS                                                        syscall.Errno = 1318
+	ERROR_NO_SUCH_GROUP                                                       syscall.Errno = 1319
+	ERROR_MEMBER_IN_GROUP                                                     syscall.Errno = 1320
+	ERROR_MEMBER_NOT_IN_GROUP                                                 syscall.Errno = 1321
+	ERROR_LAST_ADMIN                                                          syscall.Errno = 1322
+	ERROR_WRONG_PASSWORD                                                      syscall.Errno = 1323
+	ERROR_ILL_FORMED_PASSWORD                                                 syscall.Errno = 1324
+	ERROR_PASSWORD_RESTRICTION                                                syscall.Errno = 1325
+	ERROR_LOGON_FAILURE                                                       syscall.Errno = 1326
+	ERROR_ACCOUNT_RESTRICTION                                                 syscall.Errno = 1327
+	ERROR_INVALID_LOGON_HOURS                                                 syscall.Errno = 1328
+	ERROR_INVALID_WORKSTATION                                                 syscall.Errno = 1329
+	ERROR_PASSWORD_EXPIRED                                                    syscall.Errno = 1330
+	ERROR_ACCOUNT_DISABLED                                                    syscall.Errno = 1331
+	ERROR_NONE_MAPPED                                                         syscall.Errno = 1332
+	ERROR_TOO_MANY_LUIDS_REQUESTED                                            syscall.Errno = 1333
+	ERROR_LUIDS_EXHAUSTED                                                     syscall.Errno = 1334
+	ERROR_INVALID_SUB_AUTHORITY                                               syscall.Errno = 1335
+	ERROR_INVALID_ACL                                                         syscall.Errno = 1336
+	ERROR_INVALID_SID                                                         syscall.Errno = 1337
+	ERROR_INVALID_SECURITY_DESCR                                              syscall.Errno = 1338
+	ERROR_BAD_INHERITANCE_ACL                                                 syscall.Errno = 1340
+	ERROR_SERVER_DISABLED                                                     syscall.Errno = 1341
+	ERROR_SERVER_NOT_DISABLED                                                 syscall.Errno = 1342
+	ERROR_INVALID_ID_AUTHORITY                                                syscall.Errno = 1343
+	ERROR_ALLOTTED_SPACE_EXCEEDED                                             syscall.Errno = 1344
+	ERROR_INVALID_GROUP_ATTRIBUTES                                            syscall.Errno = 1345
+	ERROR_BAD_IMPERSONATION_LEVEL                                             syscall.Errno = 1346
+	ERROR_CANT_OPEN_ANONYMOUS                                                 syscall.Errno = 1347
+	ERROR_BAD_VALIDATION_CLASS                                                syscall.Errno = 1348
+	ERROR_BAD_TOKEN_TYPE                                                      syscall.Errno = 1349
+	ERROR_NO_SECURITY_ON_OBJECT                                               syscall.Errno = 1350
+	ERROR_CANT_ACCESS_DOMAIN_INFO                                             syscall.Errno = 1351
+	ERROR_INVALID_SERVER_STATE                                                syscall.Errno = 1352
+	ERROR_INVALID_DOMAIN_STATE                                                syscall.Errno = 1353
+	ERROR_INVALID_DOMAIN_ROLE                                                 syscall.Errno = 1354
+	ERROR_NO_SUCH_DOMAIN                                                      syscall.Errno = 1355
+	ERROR_DOMAIN_EXISTS                                                       syscall.Errno = 1356
+	ERROR_DOMAIN_LIMIT_EXCEEDED                                               syscall.Errno = 1357
+	ERROR_INTERNAL_DB_CORRUPTION                                              syscall.Errno = 1358
+	ERROR_INTERNAL_ERROR                                                      syscall.Errno = 1359
+	ERROR_GENERIC_NOT_MAPPED                                                  syscall.Errno = 1360
+	ERROR_BAD_DESCRIPTOR_FORMAT                                               syscall.Errno = 1361
+	ERROR_NOT_LOGON_PROCESS                                                   syscall.Errno = 1362
+	ERROR_LOGON_SESSION_EXISTS                                                syscall.Errno = 1363
+	ERROR_NO_SUCH_PACKAGE                                                     syscall.Errno = 1364
+	ERROR_BAD_LOGON_SESSION_STATE                                             syscall.Errno = 1365
+	ERROR_LOGON_SESSION_COLLISION                                             syscall.Errno = 1366
+	ERROR_INVALID_LOGON_TYPE                                                  syscall.Errno = 1367
+	ERROR_CANNOT_IMPERSONATE                                                  syscall.Errno = 1368
+	ERROR_RXACT_INVALID_STATE                                                 syscall.Errno = 1369
+	ERROR_RXACT_COMMIT_FAILURE                                                syscall.Errno = 1370
+	ERROR_SPECIAL_ACCOUNT                                                     syscall.Errno = 1371
+	ERROR_SPECIAL_GROUP                                                       syscall.Errno = 1372
+	ERROR_SPECIAL_USER                                                        syscall.Errno = 1373
+	ERROR_MEMBERS_PRIMARY_GROUP                                               syscall.Errno = 1374
+	ERROR_TOKEN_ALREADY_IN_USE                                                syscall.Errno = 1375
+	ERROR_NO_SUCH_ALIAS                                                       syscall.Errno = 1376
+	ERROR_MEMBER_NOT_IN_ALIAS                                                 syscall.Errno = 1377
+	ERROR_MEMBER_IN_ALIAS                                                     syscall.Errno = 1378
+	ERROR_ALIAS_EXISTS                                                        syscall.Errno = 1379
+	ERROR_LOGON_NOT_GRANTED                                                   syscall.Errno = 1380
+	ERROR_TOO_MANY_SECRETS                                                    syscall.Errno = 1381
+	ERROR_SECRET_TOO_LONG                                                     syscall.Errno = 1382
+	ERROR_INTERNAL_DB_ERROR                                                   syscall.Errno = 1383
+	ERROR_TOO_MANY_CONTEXT_IDS                                                syscall.Errno = 1384
+	ERROR_LOGON_TYPE_NOT_GRANTED                                              syscall.Errno = 1385
+	ERROR_NT_CROSS_ENCRYPTION_REQUIRED                                        syscall.Errno = 1386
+	ERROR_NO_SUCH_MEMBER                                                      syscall.Errno = 1387
+	ERROR_INVALID_MEMBER                                                      syscall.Errno = 1388
+	ERROR_TOO_MANY_SIDS                                                       syscall.Errno = 1389
+	ERROR_LM_CROSS_ENCRYPTION_REQUIRED                                        syscall.Errno = 1390
+	ERROR_NO_INHERITANCE                                                      syscall.Errno = 1391
+	ERROR_FILE_CORRUPT                                                        syscall.Errno = 1392
+	ERROR_DISK_CORRUPT                                                        syscall.Errno = 1393
+	ERROR_NO_USER_SESSION_KEY                                                 syscall.Errno = 1394
+	ERROR_LICENSE_QUOTA_EXCEEDED                                              syscall.Errno = 1395
+	ERROR_WRONG_TARGET_NAME                                                   syscall.Errno = 1396
+	ERROR_MUTUAL_AUTH_FAILED                                                  syscall.Errno = 1397
+	ERROR_TIME_SKEW                                                           syscall.Errno = 1398
+	ERROR_CURRENT_DOMAIN_NOT_ALLOWED                                          syscall.Errno = 1399
+	ERROR_INVALID_WINDOW_HANDLE                                               syscall.Errno = 1400
+	ERROR_INVALID_MENU_HANDLE                                                 syscall.Errno = 1401
+	ERROR_INVALID_CURSOR_HANDLE                                               syscall.Errno = 1402
+	ERROR_INVALID_ACCEL_HANDLE                                                syscall.Errno = 1403
+	ERROR_INVALID_HOOK_HANDLE                                                 syscall.Errno = 1404
+	ERROR_INVALID_DWP_HANDLE                                                  syscall.Errno = 1405
+	ERROR_TLW_WITH_WSCHILD                                                    syscall.Errno = 1406
+	ERROR_CANNOT_FIND_WND_CLASS                                               syscall.Errno = 1407
+	ERROR_WINDOW_OF_OTHER_THREAD                                              syscall.Errno = 1408
+	ERROR_HOTKEY_ALREADY_REGISTERED                                           syscall.Errno = 1409
+	ERROR_CLASS_ALREADY_EXISTS                                                syscall.Errno = 1410
+	ERROR_CLASS_DOES_NOT_EXIST                                                syscall.Errno = 1411
+	ERROR_CLASS_HAS_WINDOWS                                                   syscall.Errno = 1412
+	ERROR_INVALID_INDEX                                                       syscall.Errno = 1413
+	ERROR_INVALID_ICON_HANDLE                                                 syscall.Errno = 1414
+	ERROR_PRIVATE_DIALOG_INDEX                                                syscall.Errno = 1415
+	ERROR_LISTBOX_ID_NOT_FOUND                                                syscall.Errno = 1416
+	ERROR_NO_WILDCARD_CHARACTERS                                              syscall.Errno = 1417
+	ERROR_CLIPBOARD_NOT_OPEN                                                  syscall.Errno = 1418
+	ERROR_HOTKEY_NOT_REGISTERED                                               syscall.Errno = 1419
+	ERROR_WINDOW_NOT_DIALOG                                                   syscall.Errno = 1420
+	ERROR_CONTROL_ID_NOT_FOUND                                                syscall.Errno = 1421
+	ERROR_INVALID_COMBOBOX_MESSAGE                                            syscall.Errno = 1422
+	ERROR_WINDOW_NOT_COMBOBOX                                                 syscall.Errno = 1423
+	ERROR_INVALID_EDIT_HEIGHT                                                 syscall.Errno = 1424
+	ERROR_DC_NOT_FOUND                                                        syscall.Errno = 1425
+	ERROR_INVALID_HOOK_FILTER                                                 syscall.Errno = 1426
+	ERROR_INVALID_FILTER_PROC                                                 syscall.Errno = 1427
+	ERROR_HOOK_NEEDS_HMOD                                                     syscall.Errno = 1428
+	ERROR_GLOBAL_ONLY_HOOK                                                    syscall.Errno = 1429
+	ERROR_JOURNAL_HOOK_SET                                                    syscall.Errno = 1430
+	ERROR_HOOK_NOT_INSTALLED                                                  syscall.Errno = 1431
+	ERROR_INVALID_LB_MESSAGE                                                  syscall.Errno = 1432
+	ERROR_SETCOUNT_ON_BAD_LB                                                  syscall.Errno = 1433
+	ERROR_LB_WITHOUT_TABSTOPS                                                 syscall.Errno = 1434
+	ERROR_DESTROY_OBJECT_OF_OTHER_THREAD                                      syscall.Errno = 1435
+	ERROR_CHILD_WINDOW_MENU                                                   syscall.Errno = 1436
+	ERROR_NO_SYSTEM_MENU                                                      syscall.Errno = 1437
+	ERROR_INVALID_MSGBOX_STYLE                                                syscall.Errno = 1438
+	ERROR_INVALID_SPI_VALUE                                                   syscall.Errno = 1439
+	ERROR_SCREEN_ALREADY_LOCKED                                               syscall.Errno = 1440
+	ERROR_HWNDS_HAVE_DIFF_PARENT                                              syscall.Errno = 1441
+	ERROR_NOT_CHILD_WINDOW                                                    syscall.Errno = 1442
+	ERROR_INVALID_GW_COMMAND                                                  syscall.Errno = 1443
+	ERROR_INVALID_THREAD_ID                                                   syscall.Errno = 1444
+	ERROR_NON_MDICHILD_WINDOW                                                 syscall.Errno = 1445
+	ERROR_POPUP_ALREADY_ACTIVE                                                syscall.Errno = 1446
+	ERROR_NO_SCROLLBARS                                                       syscall.Errno = 1447
+	ERROR_INVALID_SCROLLBAR_RANGE                                             syscall.Errno = 1448
+	ERROR_INVALID_SHOWWIN_COMMAND                                             syscall.Errno = 1449
+	ERROR_NO_SYSTEM_RESOURCES                                                 syscall.Errno = 1450
+	ERROR_NONPAGED_SYSTEM_RESOURCES                                           syscall.Errno = 1451
+	ERROR_PAGED_SYSTEM_RESOURCES                                              syscall.Errno = 1452
+	ERROR_WORKING_SET_QUOTA                                                   syscall.Errno = 1453
+	ERROR_PAGEFILE_QUOTA                                                      syscall.Errno = 1454
+	ERROR_COMMITMENT_LIMIT                                                    syscall.Errno = 1455
+	ERROR_MENU_ITEM_NOT_FOUND                                                 syscall.Errno = 1456
+	ERROR_INVALID_KEYBOARD_HANDLE                                             syscall.Errno = 1457
+	ERROR_HOOK_TYPE_NOT_ALLOWED                                               syscall.Errno = 1458
+	ERROR_REQUIRES_INTERACTIVE_WINDOWSTATION                                  syscall.Errno = 1459
+	ERROR_TIMEOUT                                                             syscall.Errno = 1460
+	ERROR_INVALID_MONITOR_HANDLE                                              syscall.Errno = 1461
+	ERROR_INCORRECT_SIZE                                                      syscall.Errno = 1462
+	ERROR_SYMLINK_CLASS_DISABLED                                              syscall.Errno = 1463
+	ERROR_SYMLINK_NOT_SUPPORTED                                               syscall.Errno = 1464
+	ERROR_XML_PARSE_ERROR                                                     syscall.Errno = 1465
+	ERROR_XMLDSIG_ERROR                                                       syscall.Errno = 1466
+	ERROR_RESTART_APPLICATION                                                 syscall.Errno = 1467
+	ERROR_WRONG_COMPARTMENT                                                   syscall.Errno = 1468
+	ERROR_AUTHIP_FAILURE                                                      syscall.Errno = 1469
+	ERROR_NO_NVRAM_RESOURCES                                                  syscall.Errno = 1470
+	ERROR_NOT_GUI_PROCESS                                                     syscall.Errno = 1471
+	ERROR_EVENTLOG_FILE_CORRUPT                                               syscall.Errno = 1500
+	ERROR_EVENTLOG_CANT_START                                                 syscall.Errno = 1501
+	ERROR_LOG_FILE_FULL                                                       syscall.Errno = 1502
+	ERROR_EVENTLOG_FILE_CHANGED                                               syscall.Errno = 1503
+	ERROR_CONTAINER_ASSIGNED                                                  syscall.Errno = 1504
+	ERROR_JOB_NO_CONTAINER                                                    syscall.Errno = 1505
+	ERROR_INVALID_TASK_NAME                                                   syscall.Errno = 1550
+	ERROR_INVALID_TASK_INDEX                                                  syscall.Errno = 1551
+	ERROR_THREAD_ALREADY_IN_TASK                                              syscall.Errno = 1552
+	ERROR_INSTALL_SERVICE_FAILURE                                             syscall.Errno = 1601
+	ERROR_INSTALL_USEREXIT                                                    syscall.Errno = 1602
+	ERROR_INSTALL_FAILURE                                                     syscall.Errno = 1603
+	ERROR_INSTALL_SUSPEND                                                     syscall.Errno = 1604
+	ERROR_UNKNOWN_PRODUCT                                                     syscall.Errno = 1605
+	ERROR_UNKNOWN_FEATURE                                                     syscall.Errno = 1606
+	ERROR_UNKNOWN_COMPONENT                                                   syscall.Errno = 1607
+	ERROR_UNKNOWN_PROPERTY                                                    syscall.Errno = 1608
+	ERROR_INVALID_HANDLE_STATE                                                syscall.Errno = 1609
+	ERROR_BAD_CONFIGURATION                                                   syscall.Errno = 1610
+	ERROR_INDEX_ABSENT                                                        syscall.Errno = 1611
+	ERROR_INSTALL_SOURCE_ABSENT                                               syscall.Errno = 1612
+	ERROR_INSTALL_PACKAGE_VERSION                                             syscall.Errno = 1613
+	ERROR_PRODUCT_UNINSTALLED                                                 syscall.Errno = 1614
+	ERROR_BAD_QUERY_SYNTAX                                                    syscall.Errno = 1615
+	ERROR_INVALID_FIELD                                                       syscall.Errno = 1616
+	ERROR_DEVICE_REMOVED                                                      syscall.Errno = 1617
+	ERROR_INSTALL_ALREADY_RUNNING                                             syscall.Errno = 1618
+	ERROR_INSTALL_PACKAGE_OPEN_FAILED                                         syscall.Errno = 1619
+	ERROR_INSTALL_PACKAGE_INVALID                                             syscall.Errno = 1620
+	ERROR_INSTALL_UI_FAILURE                                                  syscall.Errno = 1621
+	ERROR_INSTALL_LOG_FAILURE                                                 syscall.Errno = 1622
+	ERROR_INSTALL_LANGUAGE_UNSUPPORTED                                        syscall.Errno = 1623
+	ERROR_INSTALL_TRANSFORM_FAILURE                                           syscall.Errno = 1624
+	ERROR_INSTALL_PACKAGE_REJECTED                                            syscall.Errno = 1625
+	ERROR_FUNCTION_NOT_CALLED                                                 syscall.Errno = 1626
+	ERROR_FUNCTION_FAILED                                                     syscall.Errno = 1627
+	ERROR_INVALID_TABLE                                                       syscall.Errno = 1628
+	ERROR_DATATYPE_MISMATCH                                                   syscall.Errno = 1629
+	ERROR_UNSUPPORTED_TYPE                                                    syscall.Errno = 1630
+	ERROR_CREATE_FAILED                                                       syscall.Errno = 1631
+	ERROR_INSTALL_TEMP_UNWRITABLE                                             syscall.Errno = 1632
+	ERROR_INSTALL_PLATFORM_UNSUPPORTED                                        syscall.Errno = 1633
+	ERROR_INSTALL_NOTUSED                                                     syscall.Errno = 1634
+	ERROR_PATCH_PACKAGE_OPEN_FAILED                                           syscall.Errno = 1635
+	ERROR_PATCH_PACKAGE_INVALID                                               syscall.Errno = 1636
+	ERROR_PATCH_PACKAGE_UNSUPPORTED                                           syscall.Errno = 1637
+	ERROR_PRODUCT_VERSION                                                     syscall.Errno = 1638
+	ERROR_INVALID_COMMAND_LINE                                                syscall.Errno = 1639
+	ERROR_INSTALL_REMOTE_DISALLOWED                                           syscall.Errno = 1640
+	ERROR_SUCCESS_REBOOT_INITIATED                                            syscall.Errno = 1641
+	ERROR_PATCH_TARGET_NOT_FOUND                                              syscall.Errno = 1642
+	ERROR_PATCH_PACKAGE_REJECTED                                              syscall.Errno = 1643
+	ERROR_INSTALL_TRANSFORM_REJECTED                                          syscall.Errno = 1644
+	ERROR_INSTALL_REMOTE_PROHIBITED                                           syscall.Errno = 1645
+	ERROR_PATCH_REMOVAL_UNSUPPORTED                                           syscall.Errno = 1646
+	ERROR_UNKNOWN_PATCH                                                       syscall.Errno = 1647
+	ERROR_PATCH_NO_SEQUENCE                                                   syscall.Errno = 1648
+	ERROR_PATCH_REMOVAL_DISALLOWED                                            syscall.Errno = 1649
+	ERROR_INVALID_PATCH_XML                                                   syscall.Errno = 1650
+	ERROR_PATCH_MANAGED_ADVERTISED_PRODUCT                                    syscall.Errno = 1651
+	ERROR_INSTALL_SERVICE_SAFEBOOT                                            syscall.Errno = 1652
+	ERROR_FAIL_FAST_EXCEPTION                                                 syscall.Errno = 1653
+	ERROR_INSTALL_REJECTED                                                    syscall.Errno = 1654
+	ERROR_DYNAMIC_CODE_BLOCKED                                                syscall.Errno = 1655
+	ERROR_NOT_SAME_OBJECT                                                     syscall.Errno = 1656
+	ERROR_STRICT_CFG_VIOLATION                                                syscall.Errno = 1657
+	ERROR_SET_CONTEXT_DENIED                                                  syscall.Errno = 1660
+	ERROR_CROSS_PARTITION_VIOLATION                                           syscall.Errno = 1661
+	RPC_S_INVALID_STRING_BINDING                                              syscall.Errno = 1700
+	RPC_S_WRONG_KIND_OF_BINDING                                               syscall.Errno = 1701
+	RPC_S_INVALID_BINDING                                                     syscall.Errno = 1702
+	RPC_S_PROTSEQ_NOT_SUPPORTED                                               syscall.Errno = 1703
+	RPC_S_INVALID_RPC_PROTSEQ                                                 syscall.Errno = 1704
+	RPC_S_INVALID_STRING_UUID                                                 syscall.Errno = 1705
+	RPC_S_INVALID_ENDPOINT_FORMAT                                             syscall.Errno = 1706
+	RPC_S_INVALID_NET_ADDR                                                    syscall.Errno = 1707
+	RPC_S_NO_ENDPOINT_FOUND                                                   syscall.Errno = 1708
+	RPC_S_INVALID_TIMEOUT                                                     syscall.Errno = 1709
+	RPC_S_OBJECT_NOT_FOUND                                                    syscall.Errno = 1710
+	RPC_S_ALREADY_REGISTERED                                                  syscall.Errno = 1711
+	RPC_S_TYPE_ALREADY_REGISTERED                                             syscall.Errno = 1712
+	RPC_S_ALREADY_LISTENING                                                   syscall.Errno = 1713
+	RPC_S_NO_PROTSEQS_REGISTERED                                              syscall.Errno = 1714
+	RPC_S_NOT_LISTENING                                                       syscall.Errno = 1715
+	RPC_S_UNKNOWN_MGR_TYPE                                                    syscall.Errno = 1716
+	RPC_S_UNKNOWN_IF                                                          syscall.Errno = 1717
+	RPC_S_NO_BINDINGS                                                         syscall.Errno = 1718
+	RPC_S_NO_PROTSEQS                                                         syscall.Errno = 1719
+	RPC_S_CANT_CREATE_ENDPOINT                                                syscall.Errno = 1720
+	RPC_S_OUT_OF_RESOURCES                                                    syscall.Errno = 1721
+	RPC_S_SERVER_UNAVAILABLE                                                  syscall.Errno = 1722
+	RPC_S_SERVER_TOO_BUSY                                                     syscall.Errno = 1723
+	RPC_S_INVALID_NETWORK_OPTIONS                                             syscall.Errno = 1724
+	RPC_S_NO_CALL_ACTIVE                                                      syscall.Errno = 1725
+	RPC_S_CALL_FAILED                                                         syscall.Errno = 1726
+	RPC_S_CALL_FAILED_DNE                                                     syscall.Errno = 1727
+	RPC_S_PROTOCOL_ERROR                                                      syscall.Errno = 1728
+	RPC_S_PROXY_ACCESS_DENIED                                                 syscall.Errno = 1729
+	RPC_S_UNSUPPORTED_TRANS_SYN                                               syscall.Errno = 1730
+	RPC_S_UNSUPPORTED_TYPE                                                    syscall.Errno = 1732
+	RPC_S_INVALID_TAG                                                         syscall.Errno = 1733
+	RPC_S_INVALID_BOUND                                                       syscall.Errno = 1734
+	RPC_S_NO_ENTRY_NAME                                                       syscall.Errno = 1735
+	RPC_S_INVALID_NAME_SYNTAX                                                 syscall.Errno = 1736
+	RPC_S_UNSUPPORTED_NAME_SYNTAX                                             syscall.Errno = 1737
+	RPC_S_UUID_NO_ADDRESS                                                     syscall.Errno = 1739
+	RPC_S_DUPLICATE_ENDPOINT                                                  syscall.Errno = 1740
+	RPC_S_UNKNOWN_AUTHN_TYPE                                                  syscall.Errno = 1741
+	RPC_S_MAX_CALLS_TOO_SMALL                                                 syscall.Errno = 1742
+	RPC_S_STRING_TOO_LONG                                                     syscall.Errno = 1743
+	RPC_S_PROTSEQ_NOT_FOUND                                                   syscall.Errno = 1744
+	RPC_S_PROCNUM_OUT_OF_RANGE                                                syscall.Errno = 1745
+	RPC_S_BINDING_HAS_NO_AUTH                                                 syscall.Errno = 1746
+	RPC_S_UNKNOWN_AUTHN_SERVICE                                               syscall.Errno = 1747
+	RPC_S_UNKNOWN_AUTHN_LEVEL                                                 syscall.Errno = 1748
+	RPC_S_INVALID_AUTH_IDENTITY                                               syscall.Errno = 1749
+	RPC_S_UNKNOWN_AUTHZ_SERVICE                                               syscall.Errno = 1750
+	EPT_S_INVALID_ENTRY                                                       syscall.Errno = 1751
+	EPT_S_CANT_PERFORM_OP                                                     syscall.Errno = 1752
+	EPT_S_NOT_REGISTERED                                                      syscall.Errno = 1753
+	RPC_S_NOTHING_TO_EXPORT                                                   syscall.Errno = 1754
+	RPC_S_INCOMPLETE_NAME                                                     syscall.Errno = 1755
+	RPC_S_INVALID_VERS_OPTION                                                 syscall.Errno = 1756
+	RPC_S_NO_MORE_MEMBERS                                                     syscall.Errno = 1757
+	RPC_S_NOT_ALL_OBJS_UNEXPORTED                                             syscall.Errno = 1758
+	RPC_S_INTERFACE_NOT_FOUND                                                 syscall.Errno = 1759
+	RPC_S_ENTRY_ALREADY_EXISTS                                                syscall.Errno = 1760
+	RPC_S_ENTRY_NOT_FOUND                                                     syscall.Errno = 1761
+	RPC_S_NAME_SERVICE_UNAVAILABLE                                            syscall.Errno = 1762
+	RPC_S_INVALID_NAF_ID                                                      syscall.Errno = 1763
+	RPC_S_CANNOT_SUPPORT                                                      syscall.Errno = 1764
+	RPC_S_NO_CONTEXT_AVAILABLE                                                syscall.Errno = 1765
+	RPC_S_INTERNAL_ERROR                                                      syscall.Errno = 1766
+	RPC_S_ZERO_DIVIDE                                                         syscall.Errno = 1767
+	RPC_S_ADDRESS_ERROR                                                       syscall.Errno = 1768
+	RPC_S_FP_DIV_ZERO                                                         syscall.Errno = 1769
+	RPC_S_FP_UNDERFLOW                                                        syscall.Errno = 1770
+	RPC_S_FP_OVERFLOW                                                         syscall.Errno = 1771
+	RPC_X_NO_MORE_ENTRIES                                                     syscall.Errno = 1772
+	RPC_X_SS_CHAR_TRANS_OPEN_FAIL                                             syscall.Errno = 1773
+	RPC_X_SS_CHAR_TRANS_SHORT_FILE                                            syscall.Errno = 1774
+	RPC_X_SS_IN_NULL_CONTEXT                                                  syscall.Errno = 1775
+	RPC_X_SS_CONTEXT_DAMAGED                                                  syscall.Errno = 1777
+	RPC_X_SS_HANDLES_MISMATCH                                                 syscall.Errno = 1778
+	RPC_X_SS_CANNOT_GET_CALL_HANDLE                                           syscall.Errno = 1779
+	RPC_X_NULL_REF_POINTER                                                    syscall.Errno = 1780
+	RPC_X_ENUM_VALUE_OUT_OF_RANGE                                             syscall.Errno = 1781
+	RPC_X_BYTE_COUNT_TOO_SMALL                                                syscall.Errno = 1782
+	RPC_X_BAD_STUB_DATA                                                       syscall.Errno = 1783
+	ERROR_INVALID_USER_BUFFER                                                 syscall.Errno = 1784
+	ERROR_UNRECOGNIZED_MEDIA                                                  syscall.Errno = 1785
+	ERROR_NO_TRUST_LSA_SECRET                                                 syscall.Errno = 1786
+	ERROR_NO_TRUST_SAM_ACCOUNT                                                syscall.Errno = 1787
+	ERROR_TRUSTED_DOMAIN_FAILURE                                              syscall.Errno = 1788
+	ERROR_TRUSTED_RELATIONSHIP_FAILURE                                        syscall.Errno = 1789
+	ERROR_TRUST_FAILURE                                                       syscall.Errno = 1790
+	RPC_S_CALL_IN_PROGRESS                                                    syscall.Errno = 1791
+	ERROR_NETLOGON_NOT_STARTED                                                syscall.Errno = 1792
+	ERROR_ACCOUNT_EXPIRED                                                     syscall.Errno = 1793
+	ERROR_REDIRECTOR_HAS_OPEN_HANDLES                                         syscall.Errno = 1794
+	ERROR_PRINTER_DRIVER_ALREADY_INSTALLED                                    syscall.Errno = 1795
+	ERROR_UNKNOWN_PORT                                                        syscall.Errno = 1796
+	ERROR_UNKNOWN_PRINTER_DRIVER                                              syscall.Errno = 1797
+	ERROR_UNKNOWN_PRINTPROCESSOR                                              syscall.Errno = 1798
+	ERROR_INVALID_SEPARATOR_FILE                                              syscall.Errno = 1799
+	ERROR_INVALID_PRIORITY                                                    syscall.Errno = 1800
+	ERROR_INVALID_PRINTER_NAME                                                syscall.Errno = 1801
+	ERROR_PRINTER_ALREADY_EXISTS                                              syscall.Errno = 1802
+	ERROR_INVALID_PRINTER_COMMAND                                             syscall.Errno = 1803
+	ERROR_INVALID_DATATYPE                                                    syscall.Errno = 1804
+	ERROR_INVALID_ENVIRONMENT                                                 syscall.Errno = 1805
+	RPC_S_NO_MORE_BINDINGS                                                    syscall.Errno = 1806
+	ERROR_NOLOGON_INTERDOMAIN_TRUST_ACCOUNT                                   syscall.Errno = 1807
+	ERROR_NOLOGON_WORKSTATION_TRUST_ACCOUNT                                   syscall.Errno = 1808
+	ERROR_NOLOGON_SERVER_TRUST_ACCOUNT                                        syscall.Errno = 1809
+	ERROR_DOMAIN_TRUST_INCONSISTENT                                           syscall.Errno = 1810
+	ERROR_SERVER_HAS_OPEN_HANDLES                                             syscall.Errno = 1811
+	ERROR_RESOURCE_DATA_NOT_FOUND                                             syscall.Errno = 1812
+	ERROR_RESOURCE_TYPE_NOT_FOUND                                             syscall.Errno = 1813
+	ERROR_RESOURCE_NAME_NOT_FOUND                                             syscall.Errno = 1814
+	ERROR_RESOURCE_LANG_NOT_FOUND                                             syscall.Errno = 1815
+	ERROR_NOT_ENOUGH_QUOTA                                                    syscall.Errno = 1816
+	RPC_S_NO_INTERFACES                                                       syscall.Errno = 1817
+	RPC_S_CALL_CANCELLED                                                      syscall.Errno = 1818
+	RPC_S_BINDING_INCOMPLETE                                                  syscall.Errno = 1819
+	RPC_S_COMM_FAILURE                                                        syscall.Errno = 1820
+	RPC_S_UNSUPPORTED_AUTHN_LEVEL                                             syscall.Errno = 1821
+	RPC_S_NO_PRINC_NAME                                                       syscall.Errno = 1822
+	RPC_S_NOT_RPC_ERROR                                                       syscall.Errno = 1823
+	RPC_S_UUID_LOCAL_ONLY                                                     syscall.Errno = 1824
+	RPC_S_SEC_PKG_ERROR                                                       syscall.Errno = 1825
+	RPC_S_NOT_CANCELLED                                                       syscall.Errno = 1826
+	RPC_X_INVALID_ES_ACTION                                                   syscall.Errno = 1827
+	RPC_X_WRONG_ES_VERSION                                                    syscall.Errno = 1828
+	RPC_X_WRONG_STUB_VERSION                                                  syscall.Errno = 1829
+	RPC_X_INVALID_PIPE_OBJECT                                                 syscall.Errno = 1830
+	RPC_X_WRONG_PIPE_ORDER                                                    syscall.Errno = 1831
+	RPC_X_WRONG_PIPE_VERSION                                                  syscall.Errno = 1832
+	RPC_S_COOKIE_AUTH_FAILED                                                  syscall.Errno = 1833
+	RPC_S_DO_NOT_DISTURB                                                      syscall.Errno = 1834
+	RPC_S_SYSTEM_HANDLE_COUNT_EXCEEDED                                        syscall.Errno = 1835
+	RPC_S_SYSTEM_HANDLE_TYPE_MISMATCH                                         syscall.Errno = 1836
+	RPC_S_GROUP_MEMBER_NOT_FOUND                                              syscall.Errno = 1898
+	EPT_S_CANT_CREATE                                                         syscall.Errno = 1899
+	RPC_S_INVALID_OBJECT                                                      syscall.Errno = 1900
+	ERROR_INVALID_TIME                                                        syscall.Errno = 1901
+	ERROR_INVALID_FORM_NAME                                                   syscall.Errno = 1902
+	ERROR_INVALID_FORM_SIZE                                                   syscall.Errno = 1903
+	ERROR_ALREADY_WAITING                                                     syscall.Errno = 1904
+	ERROR_PRINTER_DELETED                                                     syscall.Errno = 1905
+	ERROR_INVALID_PRINTER_STATE                                               syscall.Errno = 1906
+	ERROR_PASSWORD_MUST_CHANGE                                                syscall.Errno = 1907
+	ERROR_DOMAIN_CONTROLLER_NOT_FOUND                                         syscall.Errno = 1908
+	ERROR_ACCOUNT_LOCKED_OUT                                                  syscall.Errno = 1909
+	OR_INVALID_OXID                                                           syscall.Errno = 1910
+	OR_INVALID_OID                                                            syscall.Errno = 1911
+	OR_INVALID_SET                                                            syscall.Errno = 1912
+	RPC_S_SEND_INCOMPLETE                                                     syscall.Errno = 1913
+	RPC_S_INVALID_ASYNC_HANDLE                                                syscall.Errno = 1914
+	RPC_S_INVALID_ASYNC_CALL                                                  syscall.Errno = 1915
+	RPC_X_PIPE_CLOSED                                                         syscall.Errno = 1916
+	RPC_X_PIPE_DISCIPLINE_ERROR                                               syscall.Errno = 1917
+	RPC_X_PIPE_EMPTY                                                          syscall.Errno = 1918
+	ERROR_NO_SITENAME                                                         syscall.Errno = 1919
+	ERROR_CANT_ACCESS_FILE                                                    syscall.Errno = 1920
+	ERROR_CANT_RESOLVE_FILENAME                                               syscall.Errno = 1921
+	RPC_S_ENTRY_TYPE_MISMATCH                                                 syscall.Errno = 1922
+	RPC_S_NOT_ALL_OBJS_EXPORTED                                               syscall.Errno = 1923
+	RPC_S_INTERFACE_NOT_EXPORTED                                              syscall.Errno = 1924
+	RPC_S_PROFILE_NOT_ADDED                                                   syscall.Errno = 1925
+	RPC_S_PRF_ELT_NOT_ADDED                                                   syscall.Errno = 1926
+	RPC_S_PRF_ELT_NOT_REMOVED                                                 syscall.Errno = 1927
+	RPC_S_GRP_ELT_NOT_ADDED                                                   syscall.Errno = 1928
+	RPC_S_GRP_ELT_NOT_REMOVED                                                 syscall.Errno = 1929
+	ERROR_KM_DRIVER_BLOCKED                                                   syscall.Errno = 1930
+	ERROR_CONTEXT_EXPIRED                                                     syscall.Errno = 1931
+	ERROR_PER_USER_TRUST_QUOTA_EXCEEDED                                       syscall.Errno = 1932
+	ERROR_ALL_USER_TRUST_QUOTA_EXCEEDED                                       syscall.Errno = 1933
+	ERROR_USER_DELETE_TRUST_QUOTA_EXCEEDED                                    syscall.Errno = 1934
+	ERROR_AUTHENTICATION_FIREWALL_FAILED                                      syscall.Errno = 1935
+	ERROR_REMOTE_PRINT_CONNECTIONS_BLOCKED                                    syscall.Errno = 1936
+	ERROR_NTLM_BLOCKED                                                        syscall.Errno = 1937
+	ERROR_PASSWORD_CHANGE_REQUIRED                                            syscall.Errno = 1938
+	ERROR_LOST_MODE_LOGON_RESTRICTION                                         syscall.Errno = 1939
+	ERROR_INVALID_PIXEL_FORMAT                                                syscall.Errno = 2000
+	ERROR_BAD_DRIVER                                                          syscall.Errno = 2001
+	ERROR_INVALID_WINDOW_STYLE                                                syscall.Errno = 2002
+	ERROR_METAFILE_NOT_SUPPORTED                                              syscall.Errno = 2003
+	ERROR_TRANSFORM_NOT_SUPPORTED                                             syscall.Errno = 2004
+	ERROR_CLIPPING_NOT_SUPPORTED                                              syscall.Errno = 2005
+	ERROR_INVALID_CMM                                                         syscall.Errno = 2010
+	ERROR_INVALID_PROFILE                                                     syscall.Errno = 2011
+	ERROR_TAG_NOT_FOUND                                                       syscall.Errno = 2012
+	ERROR_TAG_NOT_PRESENT                                                     syscall.Errno = 2013
+	ERROR_DUPLICATE_TAG                                                       syscall.Errno = 2014
+	ERROR_PROFILE_NOT_ASSOCIATED_WITH_DEVICE                                  syscall.Errno = 2015
+	ERROR_PROFILE_NOT_FOUND                                                   syscall.Errno = 2016
+	ERROR_INVALID_COLORSPACE                                                  syscall.Errno = 2017
+	ERROR_ICM_NOT_ENABLED                                                     syscall.Errno = 2018
+	ERROR_DELETING_ICM_XFORM                                                  syscall.Errno = 2019
+	ERROR_INVALID_TRANSFORM                                                   syscall.Errno = 2020
+	ERROR_COLORSPACE_MISMATCH                                                 syscall.Errno = 2021
+	ERROR_INVALID_COLORINDEX                                                  syscall.Errno = 2022
+	ERROR_PROFILE_DOES_NOT_MATCH_DEVICE                                       syscall.Errno = 2023
+	ERROR_CONNECTED_OTHER_PASSWORD                                            syscall.Errno = 2108
+	ERROR_CONNECTED_OTHER_PASSWORD_DEFAULT                                    syscall.Errno = 2109
+	ERROR_BAD_USERNAME                                                        syscall.Errno = 2202
+	ERROR_NOT_CONNECTED                                                       syscall.Errno = 2250
+	ERROR_OPEN_FILES                                                          syscall.Errno = 2401
+	ERROR_ACTIVE_CONNECTIONS                                                  syscall.Errno = 2402
+	ERROR_DEVICE_IN_USE                                                       syscall.Errno = 2404
+	ERROR_UNKNOWN_PRINT_MONITOR                                               syscall.Errno = 3000
+	ERROR_PRINTER_DRIVER_IN_USE                                               syscall.Errno = 3001
+	ERROR_SPOOL_FILE_NOT_FOUND                                                syscall.Errno = 3002
+	ERROR_SPL_NO_STARTDOC                                                     syscall.Errno = 3003
+	ERROR_SPL_NO_ADDJOB                                                       syscall.Errno = 3004
+	ERROR_PRINT_PROCESSOR_ALREADY_INSTALLED                                   syscall.Errno = 3005
+	ERROR_PRINT_MONITOR_ALREADY_INSTALLED                                     syscall.Errno = 3006
+	ERROR_INVALID_PRINT_MONITOR                                               syscall.Errno = 3007
+	ERROR_PRINT_MONITOR_IN_USE                                                syscall.Errno = 3008
+	ERROR_PRINTER_HAS_JOBS_QUEUED                                             syscall.Errno = 3009
+	ERROR_SUCCESS_REBOOT_REQUIRED                                             syscall.Errno = 3010
+	ERROR_SUCCESS_RESTART_REQUIRED                                            syscall.Errno = 3011
+	ERROR_PRINTER_NOT_FOUND                                                   syscall.Errno = 3012
+	ERROR_PRINTER_DRIVER_WARNED                                               syscall.Errno = 3013
+	ERROR_PRINTER_DRIVER_BLOCKED                                              syscall.Errno = 3014
+	ERROR_PRINTER_DRIVER_PACKAGE_IN_USE                                       syscall.Errno = 3015
+	ERROR_CORE_DRIVER_PACKAGE_NOT_FOUND                                       syscall.Errno = 3016
+	ERROR_FAIL_REBOOT_REQUIRED                                                syscall.Errno = 3017
+	ERROR_FAIL_REBOOT_INITIATED                                               syscall.Errno = 3018
+	ERROR_PRINTER_DRIVER_DOWNLOAD_NEEDED                                      syscall.Errno = 3019
+	ERROR_PRINT_JOB_RESTART_REQUIRED                                          syscall.Errno = 3020
+	ERROR_INVALID_PRINTER_DRIVER_MANIFEST                                     syscall.Errno = 3021
+	ERROR_PRINTER_NOT_SHAREABLE                                               syscall.Errno = 3022
+	ERROR_REQUEST_PAUSED                                                      syscall.Errno = 3050
+	ERROR_APPEXEC_CONDITION_NOT_SATISFIED                                     syscall.Errno = 3060
+	ERROR_APPEXEC_HANDLE_INVALIDATED                                          syscall.Errno = 3061
+	ERROR_APPEXEC_INVALID_HOST_GENERATION                                     syscall.Errno = 3062
+	ERROR_APPEXEC_UNEXPECTED_PROCESS_REGISTRATION                             syscall.Errno = 3063
+	ERROR_APPEXEC_INVALID_HOST_STATE                                          syscall.Errno = 3064
+	ERROR_APPEXEC_NO_DONOR                                                    syscall.Errno = 3065
+	ERROR_APPEXEC_HOST_ID_MISMATCH                                            syscall.Errno = 3066
+	ERROR_APPEXEC_UNKNOWN_USER                                                syscall.Errno = 3067
+	ERROR_IO_REISSUE_AS_CACHED                                                syscall.Errno = 3950
+	ERROR_WINS_INTERNAL                                                       syscall.Errno = 4000
+	ERROR_CAN_NOT_DEL_LOCAL_WINS                                              syscall.Errno = 4001
+	ERROR_STATIC_INIT                                                         syscall.Errno = 4002
+	ERROR_INC_BACKUP                                                          syscall.Errno = 4003
+	ERROR_FULL_BACKUP                                                         syscall.Errno = 4004
+	ERROR_REC_NON_EXISTENT                                                    syscall.Errno = 4005
+	ERROR_RPL_NOT_ALLOWED                                                     syscall.Errno = 4006
+	PEERDIST_ERROR_CONTENTINFO_VERSION_UNSUPPORTED                            syscall.Errno = 4050
+	PEERDIST_ERROR_CANNOT_PARSE_CONTENTINFO                                   syscall.Errno = 4051
+	PEERDIST_ERROR_MISSING_DATA                                               syscall.Errno = 4052
+	PEERDIST_ERROR_NO_MORE                                                    syscall.Errno = 4053
+	PEERDIST_ERROR_NOT_INITIALIZED                                            syscall.Errno = 4054
+	PEERDIST_ERROR_ALREADY_INITIALIZED                                        syscall.Errno = 4055
+	PEERDIST_ERROR_SHUTDOWN_IN_PROGRESS                                       syscall.Errno = 4056
+	PEERDIST_ERROR_INVALIDATED                                                syscall.Errno = 4057
+	PEERDIST_ERROR_ALREADY_EXISTS                                             syscall.Errno = 4058
+	PEERDIST_ERROR_OPERATION_NOTFOUND                                         syscall.Errno = 4059
+	PEERDIST_ERROR_ALREADY_COMPLETED                                          syscall.Errno = 4060
+	PEERDIST_ERROR_OUT_OF_BOUNDS                                              syscall.Errno = 4061
+	PEERDIST_ERROR_VERSION_UNSUPPORTED                                        syscall.Errno = 4062
+	PEERDIST_ERROR_INVALID_CONFIGURATION                                      syscall.Errno = 4063
+	PEERDIST_ERROR_NOT_LICENSED                                               syscall.Errno = 4064
+	PEERDIST_ERROR_SERVICE_UNAVAILABLE                                        syscall.Errno = 4065
+	PEERDIST_ERROR_TRUST_FAILURE                                              syscall.Errno = 4066
+	ERROR_DHCP_ADDRESS_CONFLICT                                               syscall.Errno = 4100
+	ERROR_WMI_GUID_NOT_FOUND                                                  syscall.Errno = 4200
+	ERROR_WMI_INSTANCE_NOT_FOUND                                              syscall.Errno = 4201
+	ERROR_WMI_ITEMID_NOT_FOUND                                                syscall.Errno = 4202
+	ERROR_WMI_TRY_AGAIN                                                       syscall.Errno = 4203
+	ERROR_WMI_DP_NOT_FOUND                                                    syscall.Errno = 4204
+	ERROR_WMI_UNRESOLVED_INSTANCE_REF                                         syscall.Errno = 4205
+	ERROR_WMI_ALREADY_ENABLED                                                 syscall.Errno = 4206
+	ERROR_WMI_GUID_DISCONNECTED                                               syscall.Errno = 4207
+	ERROR_WMI_SERVER_UNAVAILABLE                                              syscall.Errno = 4208
+	ERROR_WMI_DP_FAILED                                                       syscall.Errno = 4209
+	ERROR_WMI_INVALID_MOF                                                     syscall.Errno = 4210
+	ERROR_WMI_INVALID_REGINFO                                                 syscall.Errno = 4211
+	ERROR_WMI_ALREADY_DISABLED                                                syscall.Errno = 4212
+	ERROR_WMI_READ_ONLY                                                       syscall.Errno = 4213
+	ERROR_WMI_SET_FAILURE                                                     syscall.Errno = 4214
+	ERROR_NOT_APPCONTAINER                                                    syscall.Errno = 4250
+	ERROR_APPCONTAINER_REQUIRED                                               syscall.Errno = 4251
+	ERROR_NOT_SUPPORTED_IN_APPCONTAINER                                       syscall.Errno = 4252
+	ERROR_INVALID_PACKAGE_SID_LENGTH                                          syscall.Errno = 4253
+	ERROR_INVALID_MEDIA                                                       syscall.Errno = 4300
+	ERROR_INVALID_LIBRARY                                                     syscall.Errno = 4301
+	ERROR_INVALID_MEDIA_POOL                                                  syscall.Errno = 4302
+	ERROR_DRIVE_MEDIA_MISMATCH                                                syscall.Errno = 4303
+	ERROR_MEDIA_OFFLINE                                                       syscall.Errno = 4304
+	ERROR_LIBRARY_OFFLINE                                                     syscall.Errno = 4305
+	ERROR_EMPTY                                                               syscall.Errno = 4306
+	ERROR_NOT_EMPTY                                                           syscall.Errno = 4307
+	ERROR_MEDIA_UNAVAILABLE                                                   syscall.Errno = 4308
+	ERROR_RESOURCE_DISABLED                                                   syscall.Errno = 4309
+	ERROR_INVALID_CLEANER                                                     syscall.Errno = 4310
+	ERROR_UNABLE_TO_CLEAN                                                     syscall.Errno = 4311
+	ERROR_OBJECT_NOT_FOUND                                                    syscall.Errno = 4312
+	ERROR_DATABASE_FAILURE                                                    syscall.Errno = 4313
+	ERROR_DATABASE_FULL                                                       syscall.Errno = 4314
+	ERROR_MEDIA_INCOMPATIBLE                                                  syscall.Errno = 4315
+	ERROR_RESOURCE_NOT_PRESENT                                                syscall.Errno = 4316
+	ERROR_INVALID_OPERATION                                                   syscall.Errno = 4317
+	ERROR_MEDIA_NOT_AVAILABLE                                                 syscall.Errno = 4318
+	ERROR_DEVICE_NOT_AVAILABLE                                                syscall.Errno = 4319
+	ERROR_REQUEST_REFUSED                                                     syscall.Errno = 4320
+	ERROR_INVALID_DRIVE_OBJECT                                                syscall.Errno = 4321
+	ERROR_LIBRARY_FULL                                                        syscall.Errno = 4322
+	ERROR_MEDIUM_NOT_ACCESSIBLE                                               syscall.Errno = 4323
+	ERROR_UNABLE_TO_LOAD_MEDIUM                                               syscall.Errno = 4324
+	ERROR_UNABLE_TO_INVENTORY_DRIVE                                           syscall.Errno = 4325
+	ERROR_UNABLE_TO_INVENTORY_SLOT                                            syscall.Errno = 4326
+	ERROR_UNABLE_TO_INVENTORY_TRANSPORT                                       syscall.Errno = 4327
+	ERROR_TRANSPORT_FULL                                                      syscall.Errno = 4328
+	ERROR_CONTROLLING_IEPORT                                                  syscall.Errno = 4329
+	ERROR_UNABLE_TO_EJECT_MOUNTED_MEDIA                                       syscall.Errno = 4330
+	ERROR_CLEANER_SLOT_SET                                                    syscall.Errno = 4331
+	ERROR_CLEANER_SLOT_NOT_SET                                                syscall.Errno = 4332
+	ERROR_CLEANER_CARTRIDGE_SPENT                                             syscall.Errno = 4333
+	ERROR_UNEXPECTED_OMID                                                     syscall.Errno = 4334
+	ERROR_CANT_DELETE_LAST_ITEM                                               syscall.Errno = 4335
+	ERROR_MESSAGE_EXCEEDS_MAX_SIZE                                            syscall.Errno = 4336
+	ERROR_VOLUME_CONTAINS_SYS_FILES                                           syscall.Errno = 4337
+	ERROR_INDIGENOUS_TYPE                                                     syscall.Errno = 4338
+	ERROR_NO_SUPPORTING_DRIVES                                                syscall.Errno = 4339
+	ERROR_CLEANER_CARTRIDGE_INSTALLED                                         syscall.Errno = 4340
+	ERROR_IEPORT_FULL                                                         syscall.Errno = 4341
+	ERROR_FILE_OFFLINE                                                        syscall.Errno = 4350
+	ERROR_REMOTE_STORAGE_NOT_ACTIVE                                           syscall.Errno = 4351
+	ERROR_REMOTE_STORAGE_MEDIA_ERROR                                          syscall.Errno = 4352
+	ERROR_NOT_A_REPARSE_POINT                                                 syscall.Errno = 4390
+	ERROR_REPARSE_ATTRIBUTE_CONFLICT                                          syscall.Errno = 4391
+	ERROR_INVALID_REPARSE_DATA                                                syscall.Errno = 4392
+	ERROR_REPARSE_TAG_INVALID                                                 syscall.Errno = 4393
+	ERROR_REPARSE_TAG_MISMATCH                                                syscall.Errno = 4394
+	ERROR_REPARSE_POINT_ENCOUNTERED                                           syscall.Errno = 4395
+	ERROR_APP_DATA_NOT_FOUND                                                  syscall.Errno = 4400
+	ERROR_APP_DATA_EXPIRED                                                    syscall.Errno = 4401
+	ERROR_APP_DATA_CORRUPT                                                    syscall.Errno = 4402
+	ERROR_APP_DATA_LIMIT_EXCEEDED                                             syscall.Errno = 4403
+	ERROR_APP_DATA_REBOOT_REQUIRED                                            syscall.Errno = 4404
+	ERROR_SECUREBOOT_ROLLBACK_DETECTED                                        syscall.Errno = 4420
+	ERROR_SECUREBOOT_POLICY_VIOLATION                                         syscall.Errno = 4421
+	ERROR_SECUREBOOT_INVALID_POLICY                                           syscall.Errno = 4422
+	ERROR_SECUREBOOT_POLICY_PUBLISHER_NOT_FOUND                               syscall.Errno = 4423
+	ERROR_SECUREBOOT_POLICY_NOT_SIGNED                                        syscall.Errno = 4424
+	ERROR_SECUREBOOT_NOT_ENABLED                                              syscall.Errno = 4425
+	ERROR_SECUREBOOT_FILE_REPLACED                                            syscall.Errno = 4426
+	ERROR_SECUREBOOT_POLICY_NOT_AUTHORIZED                                    syscall.Errno = 4427
+	ERROR_SECUREBOOT_POLICY_UNKNOWN                                           syscall.Errno = 4428
+	ERROR_SECUREBOOT_POLICY_MISSING_ANTIROLLBACKVERSION                       syscall.Errno = 4429
+	ERROR_SECUREBOOT_PLATFORM_ID_MISMATCH                                     syscall.Errno = 4430
+	ERROR_SECUREBOOT_POLICY_ROLLBACK_DETECTED                                 syscall.Errno = 4431
+	ERROR_SECUREBOOT_POLICY_UPGRADE_MISMATCH                                  syscall.Errno = 4432
+	ERROR_SECUREBOOT_REQUIRED_POLICY_FILE_MISSING                             syscall.Errno = 4433
+	ERROR_SECUREBOOT_NOT_BASE_POLICY                                          syscall.Errno = 4434
+	ERROR_SECUREBOOT_NOT_SUPPLEMENTAL_POLICY                                  syscall.Errno = 4435
+	ERROR_OFFLOAD_READ_FLT_NOT_SUPPORTED                                      syscall.Errno = 4440
+	ERROR_OFFLOAD_WRITE_FLT_NOT_SUPPORTED                                     syscall.Errno = 4441
+	ERROR_OFFLOAD_READ_FILE_NOT_SUPPORTED                                     syscall.Errno = 4442
+	ERROR_OFFLOAD_WRITE_FILE_NOT_SUPPORTED                                    syscall.Errno = 4443
+	ERROR_ALREADY_HAS_STREAM_ID                                               syscall.Errno = 4444
+	ERROR_SMR_GARBAGE_COLLECTION_REQUIRED                                     syscall.Errno = 4445
+	ERROR_WOF_WIM_HEADER_CORRUPT                                              syscall.Errno = 4446
+	ERROR_WOF_WIM_RESOURCE_TABLE_CORRUPT                                      syscall.Errno = 4447
+	ERROR_WOF_FILE_RESOURCE_TABLE_CORRUPT                                     syscall.Errno = 4448
+	ERROR_VOLUME_NOT_SIS_ENABLED                                              syscall.Errno = 4500
+	ERROR_SYSTEM_INTEGRITY_ROLLBACK_DETECTED                                  syscall.Errno = 4550
+	ERROR_SYSTEM_INTEGRITY_POLICY_VIOLATION                                   syscall.Errno = 4551
+	ERROR_SYSTEM_INTEGRITY_INVALID_POLICY                                     syscall.Errno = 4552
+	ERROR_SYSTEM_INTEGRITY_POLICY_NOT_SIGNED                                  syscall.Errno = 4553
+	ERROR_VSM_NOT_INITIALIZED                                                 syscall.Errno = 4560
+	ERROR_VSM_DMA_PROTECTION_NOT_IN_USE                                       syscall.Errno = 4561
+	ERROR_PLATFORM_MANIFEST_NOT_AUTHORIZED                                    syscall.Errno = 4570
+	ERROR_PLATFORM_MANIFEST_INVALID                                           syscall.Errno = 4571
+	ERROR_PLATFORM_MANIFEST_FILE_NOT_AUTHORIZED                               syscall.Errno = 4572
+	ERROR_PLATFORM_MANIFEST_CATALOG_NOT_AUTHORIZED                            syscall.Errno = 4573
+	ERROR_PLATFORM_MANIFEST_BINARY_ID_NOT_FOUND                               syscall.Errno = 4574
+	ERROR_PLATFORM_MANIFEST_NOT_ACTIVE                                        syscall.Errno = 4575
+	ERROR_PLATFORM_MANIFEST_NOT_SIGNED                                        syscall.Errno = 4576
+	ERROR_DEPENDENT_RESOURCE_EXISTS                                           syscall.Errno = 5001
+	ERROR_DEPENDENCY_NOT_FOUND                                                syscall.Errno = 5002
+	ERROR_DEPENDENCY_ALREADY_EXISTS                                           syscall.Errno = 5003
+	ERROR_RESOURCE_NOT_ONLINE                                                 syscall.Errno = 5004
+	ERROR_HOST_NODE_NOT_AVAILABLE                                             syscall.Errno = 5005
+	ERROR_RESOURCE_NOT_AVAILABLE                                              syscall.Errno = 5006
+	ERROR_RESOURCE_NOT_FOUND                                                  syscall.Errno = 5007
+	ERROR_SHUTDOWN_CLUSTER                                                    syscall.Errno = 5008
+	ERROR_CANT_EVICT_ACTIVE_NODE                                              syscall.Errno = 5009
+	ERROR_OBJECT_ALREADY_EXISTS                                               syscall.Errno = 5010
+	ERROR_OBJECT_IN_LIST                                                      syscall.Errno = 5011
+	ERROR_GROUP_NOT_AVAILABLE                                                 syscall.Errno = 5012
+	ERROR_GROUP_NOT_FOUND                                                     syscall.Errno = 5013
+	ERROR_GROUP_NOT_ONLINE                                                    syscall.Errno = 5014
+	ERROR_HOST_NODE_NOT_RESOURCE_OWNER                                        syscall.Errno = 5015
+	ERROR_HOST_NODE_NOT_GROUP_OWNER                                           syscall.Errno = 5016
+	ERROR_RESMON_CREATE_FAILED                                                syscall.Errno = 5017
+	ERROR_RESMON_ONLINE_FAILED                                                syscall.Errno = 5018
+	ERROR_RESOURCE_ONLINE                                                     syscall.Errno = 5019
+	ERROR_QUORUM_RESOURCE                                                     syscall.Errno = 5020
+	ERROR_NOT_QUORUM_CAPABLE                                                  syscall.Errno = 5021
+	ERROR_CLUSTER_SHUTTING_DOWN                                               syscall.Errno = 5022
+	ERROR_INVALID_STATE                                                       syscall.Errno = 5023
+	ERROR_RESOURCE_PROPERTIES_STORED                                          syscall.Errno = 5024
+	ERROR_NOT_QUORUM_CLASS                                                    syscall.Errno = 5025
+	ERROR_CORE_RESOURCE                                                       syscall.Errno = 5026
+	ERROR_QUORUM_RESOURCE_ONLINE_FAILED                                       syscall.Errno = 5027
+	ERROR_QUORUMLOG_OPEN_FAILED                                               syscall.Errno = 5028
+	ERROR_CLUSTERLOG_CORRUPT                                                  syscall.Errno = 5029
+	ERROR_CLUSTERLOG_RECORD_EXCEEDS_MAXSIZE                                   syscall.Errno = 5030
+	ERROR_CLUSTERLOG_EXCEEDS_MAXSIZE                                          syscall.Errno = 5031
+	ERROR_CLUSTERLOG_CHKPOINT_NOT_FOUND                                       syscall.Errno = 5032
+	ERROR_CLUSTERLOG_NOT_ENOUGH_SPACE                                         syscall.Errno = 5033
+	ERROR_QUORUM_OWNER_ALIVE                                                  syscall.Errno = 5034
+	ERROR_NETWORK_NOT_AVAILABLE                                               syscall.Errno = 5035
+	ERROR_NODE_NOT_AVAILABLE                                                  syscall.Errno = 5036
+	ERROR_ALL_NODES_NOT_AVAILABLE                                             syscall.Errno = 5037
+	ERROR_RESOURCE_FAILED                                                     syscall.Errno = 5038
+	ERROR_CLUSTER_INVALID_NODE                                                syscall.Errno = 5039
+	ERROR_CLUSTER_NODE_EXISTS                                                 syscall.Errno = 5040
+	ERROR_CLUSTER_JOIN_IN_PROGRESS                                            syscall.Errno = 5041
+	ERROR_CLUSTER_NODE_NOT_FOUND                                              syscall.Errno = 5042
+	ERROR_CLUSTER_LOCAL_NODE_NOT_FOUND                                        syscall.Errno = 5043
+	ERROR_CLUSTER_NETWORK_EXISTS                                              syscall.Errno = 5044
+	ERROR_CLUSTER_NETWORK_NOT_FOUND                                           syscall.Errno = 5045
+	ERROR_CLUSTER_NETINTERFACE_EXISTS                                         syscall.Errno = 5046
+	ERROR_CLUSTER_NETINTERFACE_NOT_FOUND                                      syscall.Errno = 5047
+	ERROR_CLUSTER_INVALID_REQUEST                                             syscall.Errno = 5048
+	ERROR_CLUSTER_INVALID_NETWORK_PROVIDER                                    syscall.Errno = 5049
+	ERROR_CLUSTER_NODE_DOWN                                                   syscall.Errno = 5050
+	ERROR_CLUSTER_NODE_UNREACHABLE                                            syscall.Errno = 5051
+	ERROR_CLUSTER_NODE_NOT_MEMBER                                             syscall.Errno = 5052
+	ERROR_CLUSTER_JOIN_NOT_IN_PROGRESS                                        syscall.Errno = 5053
+	ERROR_CLUSTER_INVALID_NETWORK                                             syscall.Errno = 5054
+	ERROR_CLUSTER_NODE_UP                                                     syscall.Errno = 5056
+	ERROR_CLUSTER_IPADDR_IN_USE                                               syscall.Errno = 5057
+	ERROR_CLUSTER_NODE_NOT_PAUSED                                             syscall.Errno = 5058
+	ERROR_CLUSTER_NO_SECURITY_CONTEXT                                         syscall.Errno = 5059
+	ERROR_CLUSTER_NETWORK_NOT_INTERNAL                                        syscall.Errno = 5060
+	ERROR_CLUSTER_NODE_ALREADY_UP                                             syscall.Errno = 5061
+	ERROR_CLUSTER_NODE_ALREADY_DOWN                                           syscall.Errno = 5062
+	ERROR_CLUSTER_NETWORK_ALREADY_ONLINE                                      syscall.Errno = 5063
+	ERROR_CLUSTER_NETWORK_ALREADY_OFFLINE                                     syscall.Errno = 5064
+	ERROR_CLUSTER_NODE_ALREADY_MEMBER                                         syscall.Errno = 5065
+	ERROR_CLUSTER_LAST_INTERNAL_NETWORK                                       syscall.Errno = 5066
+	ERROR_CLUSTER_NETWORK_HAS_DEPENDENTS                                      syscall.Errno = 5067
+	ERROR_INVALID_OPERATION_ON_QUORUM                                         syscall.Errno = 5068
+	ERROR_DEPENDENCY_NOT_ALLOWED                                              syscall.Errno = 5069
+	ERROR_CLUSTER_NODE_PAUSED                                                 syscall.Errno = 5070
+	ERROR_NODE_CANT_HOST_RESOURCE                                             syscall.Errno = 5071
+	ERROR_CLUSTER_NODE_NOT_READY                                              syscall.Errno = 5072
+	ERROR_CLUSTER_NODE_SHUTTING_DOWN                                          syscall.Errno = 5073
+	ERROR_CLUSTER_JOIN_ABORTED                                                syscall.Errno = 5074
+	ERROR_CLUSTER_INCOMPATIBLE_VERSIONS                                       syscall.Errno = 5075
+	ERROR_CLUSTER_MAXNUM_OF_RESOURCES_EXCEEDED                                syscall.Errno = 5076
+	ERROR_CLUSTER_SYSTEM_CONFIG_CHANGED                                       syscall.Errno = 5077
+	ERROR_CLUSTER_RESOURCE_TYPE_NOT_FOUND                                     syscall.Errno = 5078
+	ERROR_CLUSTER_RESTYPE_NOT_SUPPORTED                                       syscall.Errno = 5079
+	ERROR_CLUSTER_RESNAME_NOT_FOUND                                           syscall.Errno = 5080
+	ERROR_CLUSTER_NO_RPC_PACKAGES_REGISTERED                                  syscall.Errno = 5081
+	ERROR_CLUSTER_OWNER_NOT_IN_PREFLIST                                       syscall.Errno = 5082
+	ERROR_CLUSTER_DATABASE_SEQMISMATCH                                        syscall.Errno = 5083
+	ERROR_RESMON_INVALID_STATE                                                syscall.Errno = 5084
+	ERROR_CLUSTER_GUM_NOT_LOCKER                                              syscall.Errno = 5085
+	ERROR_QUORUM_DISK_NOT_FOUND                                               syscall.Errno = 5086
+	ERROR_DATABASE_BACKUP_CORRUPT                                             syscall.Errno = 5087
+	ERROR_CLUSTER_NODE_ALREADY_HAS_DFS_ROOT                                   syscall.Errno = 5088
+	ERROR_RESOURCE_PROPERTY_UNCHANGEABLE                                      syscall.Errno = 5089
+	ERROR_NO_ADMIN_ACCESS_POINT                                               syscall.Errno = 5090
+	ERROR_CLUSTER_MEMBERSHIP_INVALID_STATE                                    syscall.Errno = 5890
+	ERROR_CLUSTER_QUORUMLOG_NOT_FOUND                                         syscall.Errno = 5891
+	ERROR_CLUSTER_MEMBERSHIP_HALT                                             syscall.Errno = 5892
+	ERROR_CLUSTER_INSTANCE_ID_MISMATCH                                        syscall.Errno = 5893
+	ERROR_CLUSTER_NETWORK_NOT_FOUND_FOR_IP                                    syscall.Errno = 5894
+	ERROR_CLUSTER_PROPERTY_DATA_TYPE_MISMATCH                                 syscall.Errno = 5895
+	ERROR_CLUSTER_EVICT_WITHOUT_CLEANUP                                       syscall.Errno = 5896
+	ERROR_CLUSTER_PARAMETER_MISMATCH                                          syscall.Errno = 5897
+	ERROR_NODE_CANNOT_BE_CLUSTERED                                            syscall.Errno = 5898
+	ERROR_CLUSTER_WRONG_OS_VERSION                                            syscall.Errno = 5899
+	ERROR_CLUSTER_CANT_CREATE_DUP_CLUSTER_NAME                                syscall.Errno = 5900
+	ERROR_CLUSCFG_ALREADY_COMMITTED                                           syscall.Errno = 5901
+	ERROR_CLUSCFG_ROLLBACK_FAILED                                             syscall.Errno = 5902
+	ERROR_CLUSCFG_SYSTEM_DISK_DRIVE_LETTER_CONFLICT                           syscall.Errno = 5903
+	ERROR_CLUSTER_OLD_VERSION                                                 syscall.Errno = 5904
+	ERROR_CLUSTER_MISMATCHED_COMPUTER_ACCT_NAME                               syscall.Errno = 5905
+	ERROR_CLUSTER_NO_NET_ADAPTERS                                             syscall.Errno = 5906
+	ERROR_CLUSTER_POISONED                                                    syscall.Errno = 5907
+	ERROR_CLUSTER_GROUP_MOVING                                                syscall.Errno = 5908
+	ERROR_CLUSTER_RESOURCE_TYPE_BUSY                                          syscall.Errno = 5909
+	ERROR_RESOURCE_CALL_TIMED_OUT                                             syscall.Errno = 5910
+	ERROR_INVALID_CLUSTER_IPV6_ADDRESS                                        syscall.Errno = 5911
+	ERROR_CLUSTER_INTERNAL_INVALID_FUNCTION                                   syscall.Errno = 5912
+	ERROR_CLUSTER_PARAMETER_OUT_OF_BOUNDS                                     syscall.Errno = 5913
+	ERROR_CLUSTER_PARTIAL_SEND                                                syscall.Errno = 5914
+	ERROR_CLUSTER_REGISTRY_INVALID_FUNCTION                                   syscall.Errno = 5915
+	ERROR_CLUSTER_INVALID_STRING_TERMINATION                                  syscall.Errno = 5916
+	ERROR_CLUSTER_INVALID_STRING_FORMAT                                       syscall.Errno = 5917
+	ERROR_CLUSTER_DATABASE_TRANSACTION_IN_PROGRESS                            syscall.Errno = 5918
+	ERROR_CLUSTER_DATABASE_TRANSACTION_NOT_IN_PROGRESS                        syscall.Errno = 5919
+	ERROR_CLUSTER_NULL_DATA                                                   syscall.Errno = 5920
+	ERROR_CLUSTER_PARTIAL_READ                                                syscall.Errno = 5921
+	ERROR_CLUSTER_PARTIAL_WRITE                                               syscall.Errno = 5922
+	ERROR_CLUSTER_CANT_DESERIALIZE_DATA                                       syscall.Errno = 5923
+	ERROR_DEPENDENT_RESOURCE_PROPERTY_CONFLICT                                syscall.Errno = 5924
+	ERROR_CLUSTER_NO_QUORUM                                                   syscall.Errno = 5925
+	ERROR_CLUSTER_INVALID_IPV6_NETWORK                                        syscall.Errno = 5926
+	ERROR_CLUSTER_INVALID_IPV6_TUNNEL_NETWORK                                 syscall.Errno = 5927
+	ERROR_QUORUM_NOT_ALLOWED_IN_THIS_GROUP                                    syscall.Errno = 5928
+	ERROR_DEPENDENCY_TREE_TOO_COMPLEX                                         syscall.Errno = 5929
+	ERROR_EXCEPTION_IN_RESOURCE_CALL                                          syscall.Errno = 5930
+	ERROR_CLUSTER_RHS_FAILED_INITIALIZATION                                   syscall.Errno = 5931
+	ERROR_CLUSTER_NOT_INSTALLED                                               syscall.Errno = 5932
+	ERROR_CLUSTER_RESOURCES_MUST_BE_ONLINE_ON_THE_SAME_NODE                   syscall.Errno = 5933
+	ERROR_CLUSTER_MAX_NODES_IN_CLUSTER                                        syscall.Errno = 5934
+	ERROR_CLUSTER_TOO_MANY_NODES                                              syscall.Errno = 5935
+	ERROR_CLUSTER_OBJECT_ALREADY_USED                                         syscall.Errno = 5936
+	ERROR_NONCORE_GROUPS_FOUND                                                syscall.Errno = 5937
+	ERROR_FILE_SHARE_RESOURCE_CONFLICT                                        syscall.Errno = 5938
+	ERROR_CLUSTER_EVICT_INVALID_REQUEST                                       syscall.Errno = 5939
+	ERROR_CLUSTER_SINGLETON_RESOURCE                                          syscall.Errno = 5940
+	ERROR_CLUSTER_GROUP_SINGLETON_RESOURCE                                    syscall.Errno = 5941
+	ERROR_CLUSTER_RESOURCE_PROVIDER_FAILED                                    syscall.Errno = 5942
+	ERROR_CLUSTER_RESOURCE_CONFIGURATION_ERROR                                syscall.Errno = 5943
+	ERROR_CLUSTER_GROUP_BUSY                                                  syscall.Errno = 5944
+	ERROR_CLUSTER_NOT_SHARED_VOLUME                                           syscall.Errno = 5945
+	ERROR_CLUSTER_INVALID_SECURITY_DESCRIPTOR                                 syscall.Errno = 5946
+	ERROR_CLUSTER_SHARED_VOLUMES_IN_USE                                       syscall.Errno = 5947
+	ERROR_CLUSTER_USE_SHARED_VOLUMES_API                                      syscall.Errno = 5948
+	ERROR_CLUSTER_BACKUP_IN_PROGRESS                                          syscall.Errno = 5949
+	ERROR_NON_CSV_PATH                                                        syscall.Errno = 5950
+	ERROR_CSV_VOLUME_NOT_LOCAL                                                syscall.Errno = 5951
+	ERROR_CLUSTER_WATCHDOG_TERMINATING                                        syscall.Errno = 5952
+	ERROR_CLUSTER_RESOURCE_VETOED_MOVE_INCOMPATIBLE_NODES                     syscall.Errno = 5953
+	ERROR_CLUSTER_INVALID_NODE_WEIGHT                                         syscall.Errno = 5954
+	ERROR_CLUSTER_RESOURCE_VETOED_CALL                                        syscall.Errno = 5955
+	ERROR_RESMON_SYSTEM_RESOURCES_LACKING                                     syscall.Errno = 5956
+	ERROR_CLUSTER_RESOURCE_VETOED_MOVE_NOT_ENOUGH_RESOURCES_ON_DESTINATION    syscall.Errno = 5957
+	ERROR_CLUSTER_RESOURCE_VETOED_MOVE_NOT_ENOUGH_RESOURCES_ON_SOURCE         syscall.Errno = 5958
+	ERROR_CLUSTER_GROUP_QUEUED                                                syscall.Errno = 5959
+	ERROR_CLUSTER_RESOURCE_LOCKED_STATUS                                      syscall.Errno = 5960
+	ERROR_CLUSTER_SHARED_VOLUME_FAILOVER_NOT_ALLOWED                          syscall.Errno = 5961
+	ERROR_CLUSTER_NODE_DRAIN_IN_PROGRESS                                      syscall.Errno = 5962
+	ERROR_CLUSTER_DISK_NOT_CONNECTED                                          syscall.Errno = 5963
+	ERROR_DISK_NOT_CSV_CAPABLE                                                syscall.Errno = 5964
+	ERROR_RESOURCE_NOT_IN_AVAILABLE_STORAGE                                   syscall.Errno = 5965
+	ERROR_CLUSTER_SHARED_VOLUME_REDIRECTED                                    syscall.Errno = 5966
+	ERROR_CLUSTER_SHARED_VOLUME_NOT_REDIRECTED                                syscall.Errno = 5967
+	ERROR_CLUSTER_CANNOT_RETURN_PROPERTIES                                    syscall.Errno = 5968
+	ERROR_CLUSTER_RESOURCE_CONTAINS_UNSUPPORTED_DIFF_AREA_FOR_SHARED_VOLUMES  syscall.Errno = 5969
+	ERROR_CLUSTER_RESOURCE_IS_IN_MAINTENANCE_MODE                             syscall.Errno = 5970
+	ERROR_CLUSTER_AFFINITY_CONFLICT                                           syscall.Errno = 5971
+	ERROR_CLUSTER_RESOURCE_IS_REPLICA_VIRTUAL_MACHINE                         syscall.Errno = 5972
+	ERROR_CLUSTER_UPGRADE_INCOMPATIBLE_VERSIONS                               syscall.Errno = 5973
+	ERROR_CLUSTER_UPGRADE_FIX_QUORUM_NOT_SUPPORTED                            syscall.Errno = 5974
+	ERROR_CLUSTER_UPGRADE_RESTART_REQUIRED                                    syscall.Errno = 5975
+	ERROR_CLUSTER_UPGRADE_IN_PROGRESS                                         syscall.Errno = 5976
+	ERROR_CLUSTER_UPGRADE_INCOMPLETE                                          syscall.Errno = 5977
+	ERROR_CLUSTER_NODE_IN_GRACE_PERIOD                                        syscall.Errno = 5978
+	ERROR_CLUSTER_CSV_IO_PAUSE_TIMEOUT                                        syscall.Errno = 5979
+	ERROR_NODE_NOT_ACTIVE_CLUSTER_MEMBER                                      syscall.Errno = 5980
+	ERROR_CLUSTER_RESOURCE_NOT_MONITORED                                      syscall.Errno = 5981
+	ERROR_CLUSTER_RESOURCE_DOES_NOT_SUPPORT_UNMONITORED                       syscall.Errno = 5982
+	ERROR_CLUSTER_RESOURCE_IS_REPLICATED                                      syscall.Errno = 5983
+	ERROR_CLUSTER_NODE_ISOLATED                                               syscall.Errno = 5984
+	ERROR_CLUSTER_NODE_QUARANTINED                                            syscall.Errno = 5985
+	ERROR_CLUSTER_DATABASE_UPDATE_CONDITION_FAILED                            syscall.Errno = 5986
+	ERROR_CLUSTER_SPACE_DEGRADED                                              syscall.Errno = 5987
+	ERROR_CLUSTER_TOKEN_DELEGATION_NOT_SUPPORTED                              syscall.Errno = 5988
+	ERROR_CLUSTER_CSV_INVALID_HANDLE                                          syscall.Errno = 5989
+	ERROR_CLUSTER_CSV_SUPPORTED_ONLY_ON_COORDINATOR                           syscall.Errno = 5990
+	ERROR_GROUPSET_NOT_AVAILABLE                                              syscall.Errno = 5991
+	ERROR_GROUPSET_NOT_FOUND                                                  syscall.Errno = 5992
+	ERROR_GROUPSET_CANT_PROVIDE                                               syscall.Errno = 5993
+	ERROR_CLUSTER_FAULT_DOMAIN_PARENT_NOT_FOUND                               syscall.Errno = 5994
+	ERROR_CLUSTER_FAULT_DOMAIN_INVALID_HIERARCHY                              syscall.Errno = 5995
+	ERROR_CLUSTER_FAULT_DOMAIN_FAILED_S2D_VALIDATION                          syscall.Errno = 5996
+	ERROR_CLUSTER_FAULT_DOMAIN_S2D_CONNECTIVITY_LOSS                          syscall.Errno = 5997
+	ERROR_CLUSTER_INVALID_INFRASTRUCTURE_FILESERVER_NAME                      syscall.Errno = 5998
+	ERROR_CLUSTERSET_MANAGEMENT_CLUSTER_UNREACHABLE                           syscall.Errno = 5999
+	ERROR_ENCRYPTION_FAILED                                                   syscall.Errno = 6000
+	ERROR_DECRYPTION_FAILED                                                   syscall.Errno = 6001
+	ERROR_FILE_ENCRYPTED                                                      syscall.Errno = 6002
+	ERROR_NO_RECOVERY_POLICY                                                  syscall.Errno = 6003
+	ERROR_NO_EFS                                                              syscall.Errno = 6004
+	ERROR_WRONG_EFS                                                           syscall.Errno = 6005
+	ERROR_NO_USER_KEYS                                                        syscall.Errno = 6006
+	ERROR_FILE_NOT_ENCRYPTED                                                  syscall.Errno = 6007
+	ERROR_NOT_EXPORT_FORMAT                                                   syscall.Errno = 6008
+	ERROR_FILE_READ_ONLY                                                      syscall.Errno = 6009
+	ERROR_DIR_EFS_DISALLOWED                                                  syscall.Errno = 6010
+	ERROR_EFS_SERVER_NOT_TRUSTED                                              syscall.Errno = 6011
+	ERROR_BAD_RECOVERY_POLICY                                                 syscall.Errno = 6012
+	ERROR_EFS_ALG_BLOB_TOO_BIG                                                syscall.Errno = 6013
+	ERROR_VOLUME_NOT_SUPPORT_EFS                                              syscall.Errno = 6014
+	ERROR_EFS_DISABLED                                                        syscall.Errno = 6015
+	ERROR_EFS_VERSION_NOT_SUPPORT                                             syscall.Errno = 6016
+	ERROR_CS_ENCRYPTION_INVALID_SERVER_RESPONSE                               syscall.Errno = 6017
+	ERROR_CS_ENCRYPTION_UNSUPPORTED_SERVER                                    syscall.Errno = 6018
+	ERROR_CS_ENCRYPTION_EXISTING_ENCRYPTED_FILE                               syscall.Errno = 6019
+	ERROR_CS_ENCRYPTION_NEW_ENCRYPTED_FILE                                    syscall.Errno = 6020
+	ERROR_CS_ENCRYPTION_FILE_NOT_CSE                                          syscall.Errno = 6021
+	ERROR_ENCRYPTION_POLICY_DENIES_OPERATION                                  syscall.Errno = 6022
+	ERROR_NO_BROWSER_SERVERS_FOUND                                            syscall.Errno = 6118
+	SCHED_E_SERVICE_NOT_LOCALSYSTEM                                           syscall.Errno = 6200
+	ERROR_LOG_SECTOR_INVALID                                                  syscall.Errno = 6600
+	ERROR_LOG_SECTOR_PARITY_INVALID                                           syscall.Errno = 6601
+	ERROR_LOG_SECTOR_REMAPPED                                                 syscall.Errno = 6602
+	ERROR_LOG_BLOCK_INCOMPLETE                                                syscall.Errno = 6603
+	ERROR_LOG_INVALID_RANGE                                                   syscall.Errno = 6604
+	ERROR_LOG_BLOCKS_EXHAUSTED                                                syscall.Errno = 6605
+	ERROR_LOG_READ_CONTEXT_INVALID                                            syscall.Errno = 6606
+	ERROR_LOG_RESTART_INVALID                                                 syscall.Errno = 6607
+	ERROR_LOG_BLOCK_VERSION                                                   syscall.Errno = 6608
+	ERROR_LOG_BLOCK_INVALID                                                   syscall.Errno = 6609
+	ERROR_LOG_READ_MODE_INVALID                                               syscall.Errno = 6610
+	ERROR_LOG_NO_RESTART                                                      syscall.Errno = 6611
+	ERROR_LOG_METADATA_CORRUPT                                                syscall.Errno = 6612
+	ERROR_LOG_METADATA_INVALID                                                syscall.Errno = 6613
+	ERROR_LOG_METADATA_INCONSISTENT                                           syscall.Errno = 6614
+	ERROR_LOG_RESERVATION_INVALID                                             syscall.Errno = 6615
+	ERROR_LOG_CANT_DELETE                                                     syscall.Errno = 6616
+	ERROR_LOG_CONTAINER_LIMIT_EXCEEDED                                        syscall.Errno = 6617
+	ERROR_LOG_START_OF_LOG                                                    syscall.Errno = 6618
+	ERROR_LOG_POLICY_ALREADY_INSTALLED                                        syscall.Errno = 6619
+	ERROR_LOG_POLICY_NOT_INSTALLED                                            syscall.Errno = 6620
+	ERROR_LOG_POLICY_INVALID                                                  syscall.Errno = 6621
+	ERROR_LOG_POLICY_CONFLICT                                                 syscall.Errno = 6622
+	ERROR_LOG_PINNED_ARCHIVE_TAIL                                             syscall.Errno = 6623
+	ERROR_LOG_RECORD_NONEXISTENT                                              syscall.Errno = 6624
+	ERROR_LOG_RECORDS_RESERVED_INVALID                                        syscall.Errno = 6625
+	ERROR_LOG_SPACE_RESERVED_INVALID                                          syscall.Errno = 6626
+	ERROR_LOG_TAIL_INVALID                                                    syscall.Errno = 6627
+	ERROR_LOG_FULL                                                            syscall.Errno = 6628
+	ERROR_COULD_NOT_RESIZE_LOG                                                syscall.Errno = 6629
+	ERROR_LOG_MULTIPLEXED                                                     syscall.Errno = 6630
+	ERROR_LOG_DEDICATED                                                       syscall.Errno = 6631
+	ERROR_LOG_ARCHIVE_NOT_IN_PROGRESS                                         syscall.Errno = 6632
+	ERROR_LOG_ARCHIVE_IN_PROGRESS                                             syscall.Errno = 6633
+	ERROR_LOG_EPHEMERAL                                                       syscall.Errno = 6634
+	ERROR_LOG_NOT_ENOUGH_CONTAINERS                                           syscall.Errno = 6635
+	ERROR_LOG_CLIENT_ALREADY_REGISTERED                                       syscall.Errno = 6636
+	ERROR_LOG_CLIENT_NOT_REGISTERED                                           syscall.Errno = 6637
+	ERROR_LOG_FULL_HANDLER_IN_PROGRESS                                        syscall.Errno = 6638
+	ERROR_LOG_CONTAINER_READ_FAILED                                           syscall.Errno = 6639
+	ERROR_LOG_CONTAINER_WRITE_FAILED                                          syscall.Errno = 6640
+	ERROR_LOG_CONTAINER_OPEN_FAILED                                           syscall.Errno = 6641
+	ERROR_LOG_CONTAINER_STATE_INVALID                                         syscall.Errno = 6642
+	ERROR_LOG_STATE_INVALID                                                   syscall.Errno = 6643
+	ERROR_LOG_PINNED                                                          syscall.Errno = 6644
+	ERROR_LOG_METADATA_FLUSH_FAILED                                           syscall.Errno = 6645
+	ERROR_LOG_INCONSISTENT_SECURITY                                           syscall.Errno = 6646
+	ERROR_LOG_APPENDED_FLUSH_FAILED                                           syscall.Errno = 6647
+	ERROR_LOG_PINNED_RESERVATION                                              syscall.Errno = 6648
+	ERROR_INVALID_TRANSACTION                                                 syscall.Errno = 6700
+	ERROR_TRANSACTION_NOT_ACTIVE                                              syscall.Errno = 6701
+	ERROR_TRANSACTION_REQUEST_NOT_VALID                                       syscall.Errno = 6702
+	ERROR_TRANSACTION_NOT_REQUESTED                                           syscall.Errno = 6703
+	ERROR_TRANSACTION_ALREADY_ABORTED                                         syscall.Errno = 6704
+	ERROR_TRANSACTION_ALREADY_COMMITTED                                       syscall.Errno = 6705
+	ERROR_TM_INITIALIZATION_FAILED                                            syscall.Errno = 6706
+	ERROR_RESOURCEMANAGER_READ_ONLY                                           syscall.Errno = 6707
+	ERROR_TRANSACTION_NOT_JOINED                                              syscall.Errno = 6708
+	ERROR_TRANSACTION_SUPERIOR_EXISTS                                         syscall.Errno = 6709
+	ERROR_CRM_PROTOCOL_ALREADY_EXISTS                                         syscall.Errno = 6710
+	ERROR_TRANSACTION_PROPAGATION_FAILED                                      syscall.Errno = 6711
+	ERROR_CRM_PROTOCOL_NOT_FOUND                                              syscall.Errno = 6712
+	ERROR_TRANSACTION_INVALID_MARSHALL_BUFFER                                 syscall.Errno = 6713
+	ERROR_CURRENT_TRANSACTION_NOT_VALID                                       syscall.Errno = 6714
+	ERROR_TRANSACTION_NOT_FOUND                                               syscall.Errno = 6715
+	ERROR_RESOURCEMANAGER_NOT_FOUND                                           syscall.Errno = 6716
+	ERROR_ENLISTMENT_NOT_FOUND                                                syscall.Errno = 6717
+	ERROR_TRANSACTIONMANAGER_NOT_FOUND                                        syscall.Errno = 6718
+	ERROR_TRANSACTIONMANAGER_NOT_ONLINE                                       syscall.Errno = 6719
+	ERROR_TRANSACTIONMANAGER_RECOVERY_NAME_COLLISION                          syscall.Errno = 6720
+	ERROR_TRANSACTION_NOT_ROOT                                                syscall.Errno = 6721
+	ERROR_TRANSACTION_OBJECT_EXPIRED                                          syscall.Errno = 6722
+	ERROR_TRANSACTION_RESPONSE_NOT_ENLISTED                                   syscall.Errno = 6723
+	ERROR_TRANSACTION_RECORD_TOO_LONG                                         syscall.Errno = 6724
+	ERROR_IMPLICIT_TRANSACTION_NOT_SUPPORTED                                  syscall.Errno = 6725
+	ERROR_TRANSACTION_INTEGRITY_VIOLATED                                      syscall.Errno = 6726
+	ERROR_TRANSACTIONMANAGER_IDENTITY_MISMATCH                                syscall.Errno = 6727
+	ERROR_RM_CANNOT_BE_FROZEN_FOR_SNAPSHOT                                    syscall.Errno = 6728
+	ERROR_TRANSACTION_MUST_WRITETHROUGH                                       syscall.Errno = 6729
+	ERROR_TRANSACTION_NO_SUPERIOR                                             syscall.Errno = 6730
+	ERROR_HEURISTIC_DAMAGE_POSSIBLE                                           syscall.Errno = 6731
+	ERROR_TRANSACTIONAL_CONFLICT                                              syscall.Errno = 6800
+	ERROR_RM_NOT_ACTIVE                                                       syscall.Errno = 6801
+	ERROR_RM_METADATA_CORRUPT                                                 syscall.Errno = 6802
+	ERROR_DIRECTORY_NOT_RM                                                    syscall.Errno = 6803
+	ERROR_TRANSACTIONS_UNSUPPORTED_REMOTE                                     syscall.Errno = 6805
+	ERROR_LOG_RESIZE_INVALID_SIZE                                             syscall.Errno = 6806
+	ERROR_OBJECT_NO_LONGER_EXISTS                                             syscall.Errno = 6807
+	ERROR_STREAM_MINIVERSION_NOT_FOUND                                        syscall.Errno = 6808
+	ERROR_STREAM_MINIVERSION_NOT_VALID                                        syscall.Errno = 6809
+	ERROR_MINIVERSION_INACCESSIBLE_FROM_SPECIFIED_TRANSACTION                 syscall.Errno = 6810
+	ERROR_CANT_OPEN_MINIVERSION_WITH_MODIFY_INTENT                            syscall.Errno = 6811
+	ERROR_CANT_CREATE_MORE_STREAM_MINIVERSIONS                                syscall.Errno = 6812
+	ERROR_REMOTE_FILE_VERSION_MISMATCH                                        syscall.Errno = 6814
+	ERROR_HANDLE_NO_LONGER_VALID                                              syscall.Errno = 6815
+	ERROR_NO_TXF_METADATA                                                     syscall.Errno = 6816
+	ERROR_LOG_CORRUPTION_DETECTED                                             syscall.Errno = 6817
+	ERROR_CANT_RECOVER_WITH_HANDLE_OPEN                                       syscall.Errno = 6818
+	ERROR_RM_DISCONNECTED                                                     syscall.Errno = 6819
+	ERROR_ENLISTMENT_NOT_SUPERIOR                                             syscall.Errno = 6820
+	ERROR_RECOVERY_NOT_NEEDED                                                 syscall.Errno = 6821
+	ERROR_RM_ALREADY_STARTED                                                  syscall.Errno = 6822
+	ERROR_FILE_IDENTITY_NOT_PERSISTENT                                        syscall.Errno = 6823
+	ERROR_CANT_BREAK_TRANSACTIONAL_DEPENDENCY                                 syscall.Errno = 6824
+	ERROR_CANT_CROSS_RM_BOUNDARY                                              syscall.Errno = 6825
+	ERROR_TXF_DIR_NOT_EMPTY                                                   syscall.Errno = 6826
+	ERROR_INDOUBT_TRANSACTIONS_EXIST                                          syscall.Errno = 6827
+	ERROR_TM_VOLATILE                                                         syscall.Errno = 6828
+	ERROR_ROLLBACK_TIMER_EXPIRED                                              syscall.Errno = 6829
+	ERROR_TXF_ATTRIBUTE_CORRUPT                                               syscall.Errno = 6830
+	ERROR_EFS_NOT_ALLOWED_IN_TRANSACTION                                      syscall.Errno = 6831
+	ERROR_TRANSACTIONAL_OPEN_NOT_ALLOWED                                      syscall.Errno = 6832
+	ERROR_LOG_GROWTH_FAILED                                                   syscall.Errno = 6833
+	ERROR_TRANSACTED_MAPPING_UNSUPPORTED_REMOTE                               syscall.Errno = 6834
+	ERROR_TXF_METADATA_ALREADY_PRESENT                                        syscall.Errno = 6835
+	ERROR_TRANSACTION_SCOPE_CALLBACKS_NOT_SET                                 syscall.Errno = 6836
+	ERROR_TRANSACTION_REQUIRED_PROMOTION                                      syscall.Errno = 6837
+	ERROR_CANNOT_EXECUTE_FILE_IN_TRANSACTION                                  syscall.Errno = 6838
+	ERROR_TRANSACTIONS_NOT_FROZEN                                             syscall.Errno = 6839
+	ERROR_TRANSACTION_FREEZE_IN_PROGRESS                                      syscall.Errno = 6840
+	ERROR_NOT_SNAPSHOT_VOLUME                                                 syscall.Errno = 6841
+	ERROR_NO_SAVEPOINT_WITH_OPEN_FILES                                        syscall.Errno = 6842
+	ERROR_DATA_LOST_REPAIR                                                    syscall.Errno = 6843
+	ERROR_SPARSE_NOT_ALLOWED_IN_TRANSACTION                                   syscall.Errno = 6844
+	ERROR_TM_IDENTITY_MISMATCH                                                syscall.Errno = 6845
+	ERROR_FLOATED_SECTION                                                     syscall.Errno = 6846
+	ERROR_CANNOT_ACCEPT_TRANSACTED_WORK                                       syscall.Errno = 6847
+	ERROR_CANNOT_ABORT_TRANSACTIONS                                           syscall.Errno = 6848
+	ERROR_BAD_CLUSTERS                                                        syscall.Errno = 6849
+	ERROR_COMPRESSION_NOT_ALLOWED_IN_TRANSACTION                              syscall.Errno = 6850
+	ERROR_VOLUME_DIRTY                                                        syscall.Errno = 6851
+	ERROR_NO_LINK_TRACKING_IN_TRANSACTION                                     syscall.Errno = 6852
+	ERROR_OPERATION_NOT_SUPPORTED_IN_TRANSACTION                              syscall.Errno = 6853
+	ERROR_EXPIRED_HANDLE                                                      syscall.Errno = 6854
+	ERROR_TRANSACTION_NOT_ENLISTED                                            syscall.Errno = 6855
+	ERROR_CTX_WINSTATION_NAME_INVALID                                         syscall.Errno = 7001
+	ERROR_CTX_INVALID_PD                                                      syscall.Errno = 7002
+	ERROR_CTX_PD_NOT_FOUND                                                    syscall.Errno = 7003
+	ERROR_CTX_WD_NOT_FOUND                                                    syscall.Errno = 7004
+	ERROR_CTX_CANNOT_MAKE_EVENTLOG_ENTRY                                      syscall.Errno = 7005
+	ERROR_CTX_SERVICE_NAME_COLLISION                                          syscall.Errno = 7006
+	ERROR_CTX_CLOSE_PENDING                                                   syscall.Errno = 7007
+	ERROR_CTX_NO_OUTBUF                                                       syscall.Errno = 7008
+	ERROR_CTX_MODEM_INF_NOT_FOUND                                             syscall.Errno = 7009
+	ERROR_CTX_INVALID_MODEMNAME                                               syscall.Errno = 7010
+	ERROR_CTX_MODEM_RESPONSE_ERROR                                            syscall.Errno = 7011
+	ERROR_CTX_MODEM_RESPONSE_TIMEOUT                                          syscall.Errno = 7012
+	ERROR_CTX_MODEM_RESPONSE_NO_CARRIER                                       syscall.Errno = 7013
+	ERROR_CTX_MODEM_RESPONSE_NO_DIALTONE                                      syscall.Errno = 7014
+	ERROR_CTX_MODEM_RESPONSE_BUSY                                             syscall.Errno = 7015
+	ERROR_CTX_MODEM_RESPONSE_VOICE                                            syscall.Errno = 7016
+	ERROR_CTX_TD_ERROR                                                        syscall.Errno = 7017
+	ERROR_CTX_WINSTATION_NOT_FOUND                                            syscall.Errno = 7022
+	ERROR_CTX_WINSTATION_ALREADY_EXISTS                                       syscall.Errno = 7023
+	ERROR_CTX_WINSTATION_BUSY                                                 syscall.Errno = 7024
+	ERROR_CTX_BAD_VIDEO_MODE                                                  syscall.Errno = 7025
+	ERROR_CTX_GRAPHICS_INVALID                                                syscall.Errno = 7035
+	ERROR_CTX_LOGON_DISABLED                                                  syscall.Errno = 7037
+	ERROR_CTX_NOT_CONSOLE                                                     syscall.Errno = 7038
+	ERROR_CTX_CLIENT_QUERY_TIMEOUT                                            syscall.Errno = 7040
+	ERROR_CTX_CONSOLE_DISCONNECT                                              syscall.Errno = 7041
+	ERROR_CTX_CONSOLE_CONNECT                                                 syscall.Errno = 7042
+	ERROR_CTX_SHADOW_DENIED                                                   syscall.Errno = 7044
+	ERROR_CTX_WINSTATION_ACCESS_DENIED                                        syscall.Errno = 7045
+	ERROR_CTX_INVALID_WD                                                      syscall.Errno = 7049
+	ERROR_CTX_SHADOW_INVALID                                                  syscall.Errno = 7050
+	ERROR_CTX_SHADOW_DISABLED                                                 syscall.Errno = 7051
+	ERROR_CTX_CLIENT_LICENSE_IN_USE                                           syscall.Errno = 7052
+	ERROR_CTX_CLIENT_LICENSE_NOT_SET                                          syscall.Errno = 7053
+	ERROR_CTX_LICENSE_NOT_AVAILABLE                                           syscall.Errno = 7054
+	ERROR_CTX_LICENSE_CLIENT_INVALID                                          syscall.Errno = 7055
+	ERROR_CTX_LICENSE_EXPIRED                                                 syscall.Errno = 7056
+	ERROR_CTX_SHADOW_NOT_RUNNING                                              syscall.Errno = 7057
+	ERROR_CTX_SHADOW_ENDED_BY_MODE_CHANGE                                     syscall.Errno = 7058
+	ERROR_ACTIVATION_COUNT_EXCEEDED                                           syscall.Errno = 7059
+	ERROR_CTX_WINSTATIONS_DISABLED                                            syscall.Errno = 7060
+	ERROR_CTX_ENCRYPTION_LEVEL_REQUIRED                                       syscall.Errno = 7061
+	ERROR_CTX_SESSION_IN_USE                                                  syscall.Errno = 7062
+	ERROR_CTX_NO_FORCE_LOGOFF                                                 syscall.Errno = 7063
+	ERROR_CTX_ACCOUNT_RESTRICTION                                             syscall.Errno = 7064
+	ERROR_RDP_PROTOCOL_ERROR                                                  syscall.Errno = 7065
+	ERROR_CTX_CDM_CONNECT                                                     syscall.Errno = 7066
+	ERROR_CTX_CDM_DISCONNECT                                                  syscall.Errno = 7067
+	ERROR_CTX_SECURITY_LAYER_ERROR                                            syscall.Errno = 7068
+	ERROR_TS_INCOMPATIBLE_SESSIONS                                            syscall.Errno = 7069
+	ERROR_TS_VIDEO_SUBSYSTEM_ERROR                                            syscall.Errno = 7070
+	FRS_ERR_INVALID_API_SEQUENCE                                              syscall.Errno = 8001
+	FRS_ERR_STARTING_SERVICE                                                  syscall.Errno = 8002
+	FRS_ERR_STOPPING_SERVICE                                                  syscall.Errno = 8003
+	FRS_ERR_INTERNAL_API                                                      syscall.Errno = 8004
+	FRS_ERR_INTERNAL                                                          syscall.Errno = 8005
+	FRS_ERR_SERVICE_COMM                                                      syscall.Errno = 8006
+	FRS_ERR_INSUFFICIENT_PRIV                                                 syscall.Errno = 8007
+	FRS_ERR_AUTHENTICATION                                                    syscall.Errno = 8008
+	FRS_ERR_PARENT_INSUFFICIENT_PRIV                                          syscall.Errno = 8009
+	FRS_ERR_PARENT_AUTHENTICATION                                             syscall.Errno = 8010
+	FRS_ERR_CHILD_TO_PARENT_COMM                                              syscall.Errno = 8011
+	FRS_ERR_PARENT_TO_CHILD_COMM                                              syscall.Errno = 8012
+	FRS_ERR_SYSVOL_POPULATE                                                   syscall.Errno = 8013
+	FRS_ERR_SYSVOL_POPULATE_TIMEOUT                                           syscall.Errno = 8014
+	FRS_ERR_SYSVOL_IS_BUSY                                                    syscall.Errno = 8015
+	FRS_ERR_SYSVOL_DEMOTE                                                     syscall.Errno = 8016
+	FRS_ERR_INVALID_SERVICE_PARAMETER                                         syscall.Errno = 8017
+	DS_S_SUCCESS                                                                            = ERROR_SUCCESS
+	ERROR_DS_NOT_INSTALLED                                                    syscall.Errno = 8200
+	ERROR_DS_MEMBERSHIP_EVALUATED_LOCALLY                                     syscall.Errno = 8201
+	ERROR_DS_NO_ATTRIBUTE_OR_VALUE                                            syscall.Errno = 8202
+	ERROR_DS_INVALID_ATTRIBUTE_SYNTAX                                         syscall.Errno = 8203
+	ERROR_DS_ATTRIBUTE_TYPE_UNDEFINED                                         syscall.Errno = 8204
+	ERROR_DS_ATTRIBUTE_OR_VALUE_EXISTS                                        syscall.Errno = 8205
+	ERROR_DS_BUSY                                                             syscall.Errno = 8206
+	ERROR_DS_UNAVAILABLE                                                      syscall.Errno = 8207
+	ERROR_DS_NO_RIDS_ALLOCATED                                                syscall.Errno = 8208
+	ERROR_DS_NO_MORE_RIDS                                                     syscall.Errno = 8209
+	ERROR_DS_INCORRECT_ROLE_OWNER                                             syscall.Errno = 8210
+	ERROR_DS_RIDMGR_INIT_ERROR                                                syscall.Errno = 8211
+	ERROR_DS_OBJ_CLASS_VIOLATION                                              syscall.Errno = 8212
+	ERROR_DS_CANT_ON_NON_LEAF                                                 syscall.Errno = 8213
+	ERROR_DS_CANT_ON_RDN                                                      syscall.Errno = 8214
+	ERROR_DS_CANT_MOD_OBJ_CLASS                                               syscall.Errno = 8215
+	ERROR_DS_CROSS_DOM_MOVE_ERROR                                             syscall.Errno = 8216
+	ERROR_DS_GC_NOT_AVAILABLE                                                 syscall.Errno = 8217
+	ERROR_SHARED_POLICY                                                       syscall.Errno = 8218
+	ERROR_POLICY_OBJECT_NOT_FOUND                                             syscall.Errno = 8219
+	ERROR_POLICY_ONLY_IN_DS                                                   syscall.Errno = 8220
+	ERROR_PROMOTION_ACTIVE                                                    syscall.Errno = 8221
+	ERROR_NO_PROMOTION_ACTIVE                                                 syscall.Errno = 8222
+	ERROR_DS_OPERATIONS_ERROR                                                 syscall.Errno = 8224
+	ERROR_DS_PROTOCOL_ERROR                                                   syscall.Errno = 8225
+	ERROR_DS_TIMELIMIT_EXCEEDED                                               syscall.Errno = 8226
+	ERROR_DS_SIZELIMIT_EXCEEDED                                               syscall.Errno = 8227
+	ERROR_DS_ADMIN_LIMIT_EXCEEDED                                             syscall.Errno = 8228
+	ERROR_DS_COMPARE_FALSE                                                    syscall.Errno = 8229
+	ERROR_DS_COMPARE_TRUE                                                     syscall.Errno = 8230
+	ERROR_DS_AUTH_METHOD_NOT_SUPPORTED                                        syscall.Errno = 8231
+	ERROR_DS_STRONG_AUTH_REQUIRED                                             syscall.Errno = 8232
+	ERROR_DS_INAPPROPRIATE_AUTH                                               syscall.Errno = 8233
+	ERROR_DS_AUTH_UNKNOWN                                                     syscall.Errno = 8234
+	ERROR_DS_REFERRAL                                                         syscall.Errno = 8235
+	ERROR_DS_UNAVAILABLE_CRIT_EXTENSION                                       syscall.Errno = 8236
+	ERROR_DS_CONFIDENTIALITY_REQUIRED                                         syscall.Errno = 8237
+	ERROR_DS_INAPPROPRIATE_MATCHING                                           syscall.Errno = 8238
+	ERROR_DS_CONSTRAINT_VIOLATION                                             syscall.Errno = 8239
+	ERROR_DS_NO_SUCH_OBJECT                                                   syscall.Errno = 8240
+	ERROR_DS_ALIAS_PROBLEM                                                    syscall.Errno = 8241
+	ERROR_DS_INVALID_DN_SYNTAX                                                syscall.Errno = 8242
+	ERROR_DS_IS_LEAF                                                          syscall.Errno = 8243
+	ERROR_DS_ALIAS_DEREF_PROBLEM                                              syscall.Errno = 8244
+	ERROR_DS_UNWILLING_TO_PERFORM                                             syscall.Errno = 8245
+	ERROR_DS_LOOP_DETECT                                                      syscall.Errno = 8246
+	ERROR_DS_NAMING_VIOLATION                                                 syscall.Errno = 8247
+	ERROR_DS_OBJECT_RESULTS_TOO_LARGE                                         syscall.Errno = 8248
+	ERROR_DS_AFFECTS_MULTIPLE_DSAS                                            syscall.Errno = 8249
+	ERROR_DS_SERVER_DOWN                                                      syscall.Errno = 8250
+	ERROR_DS_LOCAL_ERROR                                                      syscall.Errno = 8251
+	ERROR_DS_ENCODING_ERROR                                                   syscall.Errno = 8252
+	ERROR_DS_DECODING_ERROR                                                   syscall.Errno = 8253
+	ERROR_DS_FILTER_UNKNOWN                                                   syscall.Errno = 8254
+	ERROR_DS_PARAM_ERROR                                                      syscall.Errno = 8255
+	ERROR_DS_NOT_SUPPORTED                                                    syscall.Errno = 8256
+	ERROR_DS_NO_RESULTS_RETURNED                                              syscall.Errno = 8257
+	ERROR_DS_CONTROL_NOT_FOUND                                                syscall.Errno = 8258
+	ERROR_DS_CLIENT_LOOP                                                      syscall.Errno = 8259
+	ERROR_DS_REFERRAL_LIMIT_EXCEEDED                                          syscall.Errno = 8260
+	ERROR_DS_SORT_CONTROL_MISSING                                             syscall.Errno = 8261
+	ERROR_DS_OFFSET_RANGE_ERROR                                               syscall.Errno = 8262
+	ERROR_DS_RIDMGR_DISABLED                                                  syscall.Errno = 8263
+	ERROR_DS_ROOT_MUST_BE_NC                                                  syscall.Errno = 8301
+	ERROR_DS_ADD_REPLICA_INHIBITED                                            syscall.Errno = 8302
+	ERROR_DS_ATT_NOT_DEF_IN_SCHEMA                                            syscall.Errno = 8303
+	ERROR_DS_MAX_OBJ_SIZE_EXCEEDED                                            syscall.Errno = 8304
+	ERROR_DS_OBJ_STRING_NAME_EXISTS                                           syscall.Errno = 8305
+	ERROR_DS_NO_RDN_DEFINED_IN_SCHEMA                                         syscall.Errno = 8306
+	ERROR_DS_RDN_DOESNT_MATCH_SCHEMA                                          syscall.Errno = 8307
+	ERROR_DS_NO_REQUESTED_ATTS_FOUND                                          syscall.Errno = 8308
+	ERROR_DS_USER_BUFFER_TO_SMALL                                             syscall.Errno = 8309
+	ERROR_DS_ATT_IS_NOT_ON_OBJ                                                syscall.Errno = 8310
+	ERROR_DS_ILLEGAL_MOD_OPERATION                                            syscall.Errno = 8311
+	ERROR_DS_OBJ_TOO_LARGE                                                    syscall.Errno = 8312
+	ERROR_DS_BAD_INSTANCE_TYPE                                                syscall.Errno = 8313
+	ERROR_DS_MASTERDSA_REQUIRED                                               syscall.Errno = 8314
+	ERROR_DS_OBJECT_CLASS_REQUIRED                                            syscall.Errno = 8315
+	ERROR_DS_MISSING_REQUIRED_ATT                                             syscall.Errno = 8316
+	ERROR_DS_ATT_NOT_DEF_FOR_CLASS                                            syscall.Errno = 8317
+	ERROR_DS_ATT_ALREADY_EXISTS                                               syscall.Errno = 8318
+	ERROR_DS_CANT_ADD_ATT_VALUES                                              syscall.Errno = 8320
+	ERROR_DS_SINGLE_VALUE_CONSTRAINT                                          syscall.Errno = 8321
+	ERROR_DS_RANGE_CONSTRAINT                                                 syscall.Errno = 8322
+	ERROR_DS_ATT_VAL_ALREADY_EXISTS                                           syscall.Errno = 8323
+	ERROR_DS_CANT_REM_MISSING_ATT                                             syscall.Errno = 8324
+	ERROR_DS_CANT_REM_MISSING_ATT_VAL                                         syscall.Errno = 8325
+	ERROR_DS_ROOT_CANT_BE_SUBREF                                              syscall.Errno = 8326
+	ERROR_DS_NO_CHAINING                                                      syscall.Errno = 8327
+	ERROR_DS_NO_CHAINED_EVAL                                                  syscall.Errno = 8328
+	ERROR_DS_NO_PARENT_OBJECT                                                 syscall.Errno = 8329
+	ERROR_DS_PARENT_IS_AN_ALIAS                                               syscall.Errno = 8330
+	ERROR_DS_CANT_MIX_MASTER_AND_REPS                                         syscall.Errno = 8331
+	ERROR_DS_CHILDREN_EXIST                                                   syscall.Errno = 8332
+	ERROR_DS_OBJ_NOT_FOUND                                                    syscall.Errno = 8333
+	ERROR_DS_ALIASED_OBJ_MISSING                                              syscall.Errno = 8334
+	ERROR_DS_BAD_NAME_SYNTAX                                                  syscall.Errno = 8335
+	ERROR_DS_ALIAS_POINTS_TO_ALIAS                                            syscall.Errno = 8336
+	ERROR_DS_CANT_DEREF_ALIAS                                                 syscall.Errno = 8337
+	ERROR_DS_OUT_OF_SCOPE                                                     syscall.Errno = 8338
+	ERROR_DS_OBJECT_BEING_REMOVED                                             syscall.Errno = 8339
+	ERROR_DS_CANT_DELETE_DSA_OBJ                                              syscall.Errno = 8340
+	ERROR_DS_GENERIC_ERROR                                                    syscall.Errno = 8341
+	ERROR_DS_DSA_MUST_BE_INT_MASTER                                           syscall.Errno = 8342
+	ERROR_DS_CLASS_NOT_DSA                                                    syscall.Errno = 8343
+	ERROR_DS_INSUFF_ACCESS_RIGHTS                                             syscall.Errno = 8344
+	ERROR_DS_ILLEGAL_SUPERIOR                                                 syscall.Errno = 8345
+	ERROR_DS_ATTRIBUTE_OWNED_BY_SAM                                           syscall.Errno = 8346
+	ERROR_DS_NAME_TOO_MANY_PARTS                                              syscall.Errno = 8347
+	ERROR_DS_NAME_TOO_LONG                                                    syscall.Errno = 8348
+	ERROR_DS_NAME_VALUE_TOO_LONG                                              syscall.Errno = 8349
+	ERROR_DS_NAME_UNPARSEABLE                                                 syscall.Errno = 8350
+	ERROR_DS_NAME_TYPE_UNKNOWN                                                syscall.Errno = 8351
+	ERROR_DS_NOT_AN_OBJECT                                                    syscall.Errno = 8352
+	ERROR_DS_SEC_DESC_TOO_SHORT                                               syscall.Errno = 8353
+	ERROR_DS_SEC_DESC_INVALID                                                 syscall.Errno = 8354
+	ERROR_DS_NO_DELETED_NAME                                                  syscall.Errno = 8355
+	ERROR_DS_SUBREF_MUST_HAVE_PARENT                                          syscall.Errno = 8356
+	ERROR_DS_NCNAME_MUST_BE_NC                                                syscall.Errno = 8357
+	ERROR_DS_CANT_ADD_SYSTEM_ONLY                                             syscall.Errno = 8358
+	ERROR_DS_CLASS_MUST_BE_CONCRETE                                           syscall.Errno = 8359
+	ERROR_DS_INVALID_DMD                                                      syscall.Errno = 8360
+	ERROR_DS_OBJ_GUID_EXISTS                                                  syscall.Errno = 8361
+	ERROR_DS_NOT_ON_BACKLINK                                                  syscall.Errno = 8362
+	ERROR_DS_NO_CROSSREF_FOR_NC                                               syscall.Errno = 8363
+	ERROR_DS_SHUTTING_DOWN                                                    syscall.Errno = 8364
+	ERROR_DS_UNKNOWN_OPERATION                                                syscall.Errno = 8365
+	ERROR_DS_INVALID_ROLE_OWNER                                               syscall.Errno = 8366
+	ERROR_DS_COULDNT_CONTACT_FSMO                                             syscall.Errno = 8367
+	ERROR_DS_CROSS_NC_DN_RENAME                                               syscall.Errno = 8368
+	ERROR_DS_CANT_MOD_SYSTEM_ONLY                                             syscall.Errno = 8369
+	ERROR_DS_REPLICATOR_ONLY                                                  syscall.Errno = 8370
+	ERROR_DS_OBJ_CLASS_NOT_DEFINED                                            syscall.Errno = 8371
+	ERROR_DS_OBJ_CLASS_NOT_SUBCLASS                                           syscall.Errno = 8372
+	ERROR_DS_NAME_REFERENCE_INVALID                                           syscall.Errno = 8373
+	ERROR_DS_CROSS_REF_EXISTS                                                 syscall.Errno = 8374
+	ERROR_DS_CANT_DEL_MASTER_CROSSREF                                         syscall.Errno = 8375
+	ERROR_DS_SUBTREE_NOTIFY_NOT_NC_HEAD                                       syscall.Errno = 8376
+	ERROR_DS_NOTIFY_FILTER_TOO_COMPLEX                                        syscall.Errno = 8377
+	ERROR_DS_DUP_RDN                                                          syscall.Errno = 8378
+	ERROR_DS_DUP_OID                                                          syscall.Errno = 8379
+	ERROR_DS_DUP_MAPI_ID                                                      syscall.Errno = 8380
+	ERROR_DS_DUP_SCHEMA_ID_GUID                                               syscall.Errno = 8381
+	ERROR_DS_DUP_LDAP_DISPLAY_NAME                                            syscall.Errno = 8382
+	ERROR_DS_SEMANTIC_ATT_TEST                                                syscall.Errno = 8383
+	ERROR_DS_SYNTAX_MISMATCH                                                  syscall.Errno = 8384
+	ERROR_DS_EXISTS_IN_MUST_HAVE                                              syscall.Errno = 8385
+	ERROR_DS_EXISTS_IN_MAY_HAVE                                               syscall.Errno = 8386
+	ERROR_DS_NONEXISTENT_MAY_HAVE                                             syscall.Errno = 8387
+	ERROR_DS_NONEXISTENT_MUST_HAVE                                            syscall.Errno = 8388
+	ERROR_DS_AUX_CLS_TEST_FAIL                                                syscall.Errno = 8389
+	ERROR_DS_NONEXISTENT_POSS_SUP                                             syscall.Errno = 8390
+	ERROR_DS_SUB_CLS_TEST_FAIL                                                syscall.Errno = 8391
+	ERROR_DS_BAD_RDN_ATT_ID_SYNTAX                                            syscall.Errno = 8392
+	ERROR_DS_EXISTS_IN_AUX_CLS                                                syscall.Errno = 8393
+	ERROR_DS_EXISTS_IN_SUB_CLS                                                syscall.Errno = 8394
+	ERROR_DS_EXISTS_IN_POSS_SUP                                               syscall.Errno = 8395
+	ERROR_DS_RECALCSCHEMA_FAILED                                              syscall.Errno = 8396
+	ERROR_DS_TREE_DELETE_NOT_FINISHED                                         syscall.Errno = 8397
+	ERROR_DS_CANT_DELETE                                                      syscall.Errno = 8398
+	ERROR_DS_ATT_SCHEMA_REQ_ID                                                syscall.Errno = 8399
+	ERROR_DS_BAD_ATT_SCHEMA_SYNTAX                                            syscall.Errno = 8400
+	ERROR_DS_CANT_CACHE_ATT                                                   syscall.Errno = 8401
+	ERROR_DS_CANT_CACHE_CLASS                                                 syscall.Errno = 8402
+	ERROR_DS_CANT_REMOVE_ATT_CACHE                                            syscall.Errno = 8403
+	ERROR_DS_CANT_REMOVE_CLASS_CACHE                                          syscall.Errno = 8404
+	ERROR_DS_CANT_RETRIEVE_DN                                                 syscall.Errno = 8405
+	ERROR_DS_MISSING_SUPREF                                                   syscall.Errno = 8406
+	ERROR_DS_CANT_RETRIEVE_INSTANCE                                           syscall.Errno = 8407
+	ERROR_DS_CODE_INCONSISTENCY                                               syscall.Errno = 8408
+	ERROR_DS_DATABASE_ERROR                                                   syscall.Errno = 8409
+	ERROR_DS_GOVERNSID_MISSING                                                syscall.Errno = 8410
+	ERROR_DS_MISSING_EXPECTED_ATT                                             syscall.Errno = 8411
+	ERROR_DS_NCNAME_MISSING_CR_REF                                            syscall.Errno = 8412
+	ERROR_DS_SECURITY_CHECKING_ERROR                                          syscall.Errno = 8413
+	ERROR_DS_SCHEMA_NOT_LOADED                                                syscall.Errno = 8414
+	ERROR_DS_SCHEMA_ALLOC_FAILED                                              syscall.Errno = 8415
+	ERROR_DS_ATT_SCHEMA_REQ_SYNTAX                                            syscall.Errno = 8416
+	ERROR_DS_GCVERIFY_ERROR                                                   syscall.Errno = 8417
+	ERROR_DS_DRA_SCHEMA_MISMATCH                                              syscall.Errno = 8418
+	ERROR_DS_CANT_FIND_DSA_OBJ                                                syscall.Errno = 8419
+	ERROR_DS_CANT_FIND_EXPECTED_NC                                            syscall.Errno = 8420
+	ERROR_DS_CANT_FIND_NC_IN_CACHE                                            syscall.Errno = 8421
+	ERROR_DS_CANT_RETRIEVE_CHILD                                              syscall.Errno = 8422
+	ERROR_DS_SECURITY_ILLEGAL_MODIFY                                          syscall.Errno = 8423
+	ERROR_DS_CANT_REPLACE_HIDDEN_REC                                          syscall.Errno = 8424
+	ERROR_DS_BAD_HIERARCHY_FILE                                               syscall.Errno = 8425
+	ERROR_DS_BUILD_HIERARCHY_TABLE_FAILED                                     syscall.Errno = 8426
+	ERROR_DS_CONFIG_PARAM_MISSING                                             syscall.Errno = 8427
+	ERROR_DS_COUNTING_AB_INDICES_FAILED                                       syscall.Errno = 8428
+	ERROR_DS_HIERARCHY_TABLE_MALLOC_FAILED                                    syscall.Errno = 8429
+	ERROR_DS_INTERNAL_FAILURE                                                 syscall.Errno = 8430
+	ERROR_DS_UNKNOWN_ERROR                                                    syscall.Errno = 8431
+	ERROR_DS_ROOT_REQUIRES_CLASS_TOP                                          syscall.Errno = 8432
+	ERROR_DS_REFUSING_FSMO_ROLES                                              syscall.Errno = 8433
+	ERROR_DS_MISSING_FSMO_SETTINGS                                            syscall.Errno = 8434
+	ERROR_DS_UNABLE_TO_SURRENDER_ROLES                                        syscall.Errno = 8435
+	ERROR_DS_DRA_GENERIC                                                      syscall.Errno = 8436
+	ERROR_DS_DRA_INVALID_PARAMETER                                            syscall.Errno = 8437
+	ERROR_DS_DRA_BUSY                                                         syscall.Errno = 8438
+	ERROR_DS_DRA_BAD_DN                                                       syscall.Errno = 8439
+	ERROR_DS_DRA_BAD_NC                                                       syscall.Errno = 8440
+	ERROR_DS_DRA_DN_EXISTS                                                    syscall.Errno = 8441
+	ERROR_DS_DRA_INTERNAL_ERROR                                               syscall.Errno = 8442
+	ERROR_DS_DRA_INCONSISTENT_DIT                                             syscall.Errno = 8443
+	ERROR_DS_DRA_CONNECTION_FAILED                                            syscall.Errno = 8444
+	ERROR_DS_DRA_BAD_INSTANCE_TYPE                                            syscall.Errno = 8445
+	ERROR_DS_DRA_OUT_OF_MEM                                                   syscall.Errno = 8446
+	ERROR_DS_DRA_MAIL_PROBLEM                                                 syscall.Errno = 8447
+	ERROR_DS_DRA_REF_ALREADY_EXISTS                                           syscall.Errno = 8448
+	ERROR_DS_DRA_REF_NOT_FOUND                                                syscall.Errno = 8449
+	ERROR_DS_DRA_OBJ_IS_REP_SOURCE                                            syscall.Errno = 8450
+	ERROR_DS_DRA_DB_ERROR                                                     syscall.Errno = 8451
+	ERROR_DS_DRA_NO_REPLICA                                                   syscall.Errno = 8452
+	ERROR_DS_DRA_ACCESS_DENIED                                                syscall.Errno = 8453
+	ERROR_DS_DRA_NOT_SUPPORTED                                                syscall.Errno = 8454
+	ERROR_DS_DRA_RPC_CANCELLED                                                syscall.Errno = 8455
+	ERROR_DS_DRA_SOURCE_DISABLED                                              syscall.Errno = 8456
+	ERROR_DS_DRA_SINK_DISABLED                                                syscall.Errno = 8457
+	ERROR_DS_DRA_NAME_COLLISION                                               syscall.Errno = 8458
+	ERROR_DS_DRA_SOURCE_REINSTALLED                                           syscall.Errno = 8459
+	ERROR_DS_DRA_MISSING_PARENT                                               syscall.Errno = 8460
+	ERROR_DS_DRA_PREEMPTED                                                    syscall.Errno = 8461
+	ERROR_DS_DRA_ABANDON_SYNC                                                 syscall.Errno = 8462
+	ERROR_DS_DRA_SHUTDOWN                                                     syscall.Errno = 8463
+	ERROR_DS_DRA_INCOMPATIBLE_PARTIAL_SET                                     syscall.Errno = 8464
+	ERROR_DS_DRA_SOURCE_IS_PARTIAL_REPLICA                                    syscall.Errno = 8465
+	ERROR_DS_DRA_EXTN_CONNECTION_FAILED                                       syscall.Errno = 8466
+	ERROR_DS_INSTALL_SCHEMA_MISMATCH                                          syscall.Errno = 8467
+	ERROR_DS_DUP_LINK_ID                                                      syscall.Errno = 8468
+	ERROR_DS_NAME_ERROR_RESOLVING                                             syscall.Errno = 8469
+	ERROR_DS_NAME_ERROR_NOT_FOUND                                             syscall.Errno = 8470
+	ERROR_DS_NAME_ERROR_NOT_UNIQUE                                            syscall.Errno = 8471
+	ERROR_DS_NAME_ERROR_NO_MAPPING                                            syscall.Errno = 8472
+	ERROR_DS_NAME_ERROR_DOMAIN_ONLY                                           syscall.Errno = 8473
+	ERROR_DS_NAME_ERROR_NO_SYNTACTICAL_MAPPING                                syscall.Errno = 8474
+	ERROR_DS_CONSTRUCTED_ATT_MOD                                              syscall.Errno = 8475
+	ERROR_DS_WRONG_OM_OBJ_CLASS                                               syscall.Errno = 8476
+	ERROR_DS_DRA_REPL_PENDING                                                 syscall.Errno = 8477
+	ERROR_DS_DS_REQUIRED                                                      syscall.Errno = 8478
+	ERROR_DS_INVALID_LDAP_DISPLAY_NAME                                        syscall.Errno = 8479
+	ERROR_DS_NON_BASE_SEARCH                                                  syscall.Errno = 8480
+	ERROR_DS_CANT_RETRIEVE_ATTS                                               syscall.Errno = 8481
+	ERROR_DS_BACKLINK_WITHOUT_LINK                                            syscall.Errno = 8482
+	ERROR_DS_EPOCH_MISMATCH                                                   syscall.Errno = 8483
+	ERROR_DS_SRC_NAME_MISMATCH                                                syscall.Errno = 8484
+	ERROR_DS_SRC_AND_DST_NC_IDENTICAL                                         syscall.Errno = 8485
+	ERROR_DS_DST_NC_MISMATCH                                                  syscall.Errno = 8486
+	ERROR_DS_NOT_AUTHORITIVE_FOR_DST_NC                                       syscall.Errno = 8487
+	ERROR_DS_SRC_GUID_MISMATCH                                                syscall.Errno = 8488
+	ERROR_DS_CANT_MOVE_DELETED_OBJECT                                         syscall.Errno = 8489
+	ERROR_DS_PDC_OPERATION_IN_PROGRESS                                        syscall.Errno = 8490
+	ERROR_DS_CROSS_DOMAIN_CLEANUP_REQD                                        syscall.Errno = 8491
+	ERROR_DS_ILLEGAL_XDOM_MOVE_OPERATION                                      syscall.Errno = 8492
+	ERROR_DS_CANT_WITH_ACCT_GROUP_MEMBERSHPS                                  syscall.Errno = 8493
+	ERROR_DS_NC_MUST_HAVE_NC_PARENT                                           syscall.Errno = 8494
+	ERROR_DS_CR_IMPOSSIBLE_TO_VALIDATE                                        syscall.Errno = 8495
+	ERROR_DS_DST_DOMAIN_NOT_NATIVE                                            syscall.Errno = 8496
+	ERROR_DS_MISSING_INFRASTRUCTURE_CONTAINER                                 syscall.Errno = 8497
+	ERROR_DS_CANT_MOVE_ACCOUNT_GROUP                                          syscall.Errno = 8498
+	ERROR_DS_CANT_MOVE_RESOURCE_GROUP                                         syscall.Errno = 8499
+	ERROR_DS_INVALID_SEARCH_FLAG                                              syscall.Errno = 8500
+	ERROR_DS_NO_TREE_DELETE_ABOVE_NC                                          syscall.Errno = 8501
+	ERROR_DS_COULDNT_LOCK_TREE_FOR_DELETE                                     syscall.Errno = 8502
+	ERROR_DS_COULDNT_IDENTIFY_OBJECTS_FOR_TREE_DELETE                         syscall.Errno = 8503
+	ERROR_DS_SAM_INIT_FAILURE                                                 syscall.Errno = 8504
+	ERROR_DS_SENSITIVE_GROUP_VIOLATION                                        syscall.Errno = 8505
+	ERROR_DS_CANT_MOD_PRIMARYGROUPID                                          syscall.Errno = 8506
+	ERROR_DS_ILLEGAL_BASE_SCHEMA_MOD                                          syscall.Errno = 8507
+	ERROR_DS_NONSAFE_SCHEMA_CHANGE                                            syscall.Errno = 8508
+	ERROR_DS_SCHEMA_UPDATE_DISALLOWED                                         syscall.Errno = 8509
+	ERROR_DS_CANT_CREATE_UNDER_SCHEMA                                         syscall.Errno = 8510
+	ERROR_DS_INSTALL_NO_SRC_SCH_VERSION                                       syscall.Errno = 8511
+	ERROR_DS_INSTALL_NO_SCH_VERSION_IN_INIFILE                                syscall.Errno = 8512
+	ERROR_DS_INVALID_GROUP_TYPE                                               syscall.Errno = 8513
+	ERROR_DS_NO_NEST_GLOBALGROUP_IN_MIXEDDOMAIN                               syscall.Errno = 8514
+	ERROR_DS_NO_NEST_LOCALGROUP_IN_MIXEDDOMAIN                                syscall.Errno = 8515
+	ERROR_DS_GLOBAL_CANT_HAVE_LOCAL_MEMBER                                    syscall.Errno = 8516
+	ERROR_DS_GLOBAL_CANT_HAVE_UNIVERSAL_MEMBER                                syscall.Errno = 8517
+	ERROR_DS_UNIVERSAL_CANT_HAVE_LOCAL_MEMBER                                 syscall.Errno = 8518
+	ERROR_DS_GLOBAL_CANT_HAVE_CROSSDOMAIN_MEMBER                              syscall.Errno = 8519
+	ERROR_DS_LOCAL_CANT_HAVE_CROSSDOMAIN_LOCAL_MEMBER                         syscall.Errno = 8520
+	ERROR_DS_HAVE_PRIMARY_MEMBERS                                             syscall.Errno = 8521
+	ERROR_DS_STRING_SD_CONVERSION_FAILED                                      syscall.Errno = 8522
+	ERROR_DS_NAMING_MASTER_GC                                                 syscall.Errno = 8523
+	ERROR_DS_DNS_LOOKUP_FAILURE                                               syscall.Errno = 8524
+	ERROR_DS_COULDNT_UPDATE_SPNS                                              syscall.Errno = 8525
+	ERROR_DS_CANT_RETRIEVE_SD                                                 syscall.Errno = 8526
+	ERROR_DS_KEY_NOT_UNIQUE                                                   syscall.Errno = 8527
+	ERROR_DS_WRONG_LINKED_ATT_SYNTAX                                          syscall.Errno = 8528
+	ERROR_DS_SAM_NEED_BOOTKEY_PASSWORD                                        syscall.Errno = 8529
+	ERROR_DS_SAM_NEED_BOOTKEY_FLOPPY                                          syscall.Errno = 8530
+	ERROR_DS_CANT_START                                                       syscall.Errno = 8531
+	ERROR_DS_INIT_FAILURE                                                     syscall.Errno = 8532
+	ERROR_DS_NO_PKT_PRIVACY_ON_CONNECTION                                     syscall.Errno = 8533
+	ERROR_DS_SOURCE_DOMAIN_IN_FOREST                                          syscall.Errno = 8534
+	ERROR_DS_DESTINATION_DOMAIN_NOT_IN_FOREST                                 syscall.Errno = 8535
+	ERROR_DS_DESTINATION_AUDITING_NOT_ENABLED                                 syscall.Errno = 8536
+	ERROR_DS_CANT_FIND_DC_FOR_SRC_DOMAIN                                      syscall.Errno = 8537
+	ERROR_DS_SRC_OBJ_NOT_GROUP_OR_USER                                        syscall.Errno = 8538
+	ERROR_DS_SRC_SID_EXISTS_IN_FOREST                                         syscall.Errno = 8539
+	ERROR_DS_SRC_AND_DST_OBJECT_CLASS_MISMATCH                                syscall.Errno = 8540
+	ERROR_SAM_INIT_FAILURE                                                    syscall.Errno = 8541
+	ERROR_DS_DRA_SCHEMA_INFO_SHIP                                             syscall.Errno = 8542
+	ERROR_DS_DRA_SCHEMA_CONFLICT                                              syscall.Errno = 8543
+	ERROR_DS_DRA_EARLIER_SCHEMA_CONFLICT                                      syscall.Errno = 8544
+	ERROR_DS_DRA_OBJ_NC_MISMATCH                                              syscall.Errno = 8545
+	ERROR_DS_NC_STILL_HAS_DSAS                                                syscall.Errno = 8546
+	ERROR_DS_GC_REQUIRED                                                      syscall.Errno = 8547
+	ERROR_DS_LOCAL_MEMBER_OF_LOCAL_ONLY                                       syscall.Errno = 8548
+	ERROR_DS_NO_FPO_IN_UNIVERSAL_GROUPS                                       syscall.Errno = 8549
+	ERROR_DS_CANT_ADD_TO_GC                                                   syscall.Errno = 8550
+	ERROR_DS_NO_CHECKPOINT_WITH_PDC                                           syscall.Errno = 8551
+	ERROR_DS_SOURCE_AUDITING_NOT_ENABLED                                      syscall.Errno = 8552
+	ERROR_DS_CANT_CREATE_IN_NONDOMAIN_NC                                      syscall.Errno = 8553
+	ERROR_DS_INVALID_NAME_FOR_SPN                                             syscall.Errno = 8554
+	ERROR_DS_FILTER_USES_CONTRUCTED_ATTRS                                     syscall.Errno = 8555
+	ERROR_DS_UNICODEPWD_NOT_IN_QUOTES                                         syscall.Errno = 8556
+	ERROR_DS_MACHINE_ACCOUNT_QUOTA_EXCEEDED                                   syscall.Errno = 8557
+	ERROR_DS_MUST_BE_RUN_ON_DST_DC                                            syscall.Errno = 8558
+	ERROR_DS_SRC_DC_MUST_BE_SP4_OR_GREATER                                    syscall.Errno = 8559
+	ERROR_DS_CANT_TREE_DELETE_CRITICAL_OBJ                                    syscall.Errno = 8560
+	ERROR_DS_INIT_FAILURE_CONSOLE                                             syscall.Errno = 8561
+	ERROR_DS_SAM_INIT_FAILURE_CONSOLE                                         syscall.Errno = 8562
+	ERROR_DS_FOREST_VERSION_TOO_HIGH                                          syscall.Errno = 8563
+	ERROR_DS_DOMAIN_VERSION_TOO_HIGH                                          syscall.Errno = 8564
+	ERROR_DS_FOREST_VERSION_TOO_LOW                                           syscall.Errno = 8565
+	ERROR_DS_DOMAIN_VERSION_TOO_LOW                                           syscall.Errno = 8566
+	ERROR_DS_INCOMPATIBLE_VERSION                                             syscall.Errno = 8567
+	ERROR_DS_LOW_DSA_VERSION                                                  syscall.Errno = 8568
+	ERROR_DS_NO_BEHAVIOR_VERSION_IN_MIXEDDOMAIN                               syscall.Errno = 8569
+	ERROR_DS_NOT_SUPPORTED_SORT_ORDER                                         syscall.Errno = 8570
+	ERROR_DS_NAME_NOT_UNIQUE                                                  syscall.Errno = 8571
+	ERROR_DS_MACHINE_ACCOUNT_CREATED_PRENT4                                   syscall.Errno = 8572
+	ERROR_DS_OUT_OF_VERSION_STORE                                             syscall.Errno = 8573
+	ERROR_DS_INCOMPATIBLE_CONTROLS_USED                                       syscall.Errno = 8574
+	ERROR_DS_NO_REF_DOMAIN                                                    syscall.Errno = 8575
+	ERROR_DS_RESERVED_LINK_ID                                                 syscall.Errno = 8576
+	ERROR_DS_LINK_ID_NOT_AVAILABLE                                            syscall.Errno = 8577
+	ERROR_DS_AG_CANT_HAVE_UNIVERSAL_MEMBER                                    syscall.Errno = 8578
+	ERROR_DS_MODIFYDN_DISALLOWED_BY_INSTANCE_TYPE                             syscall.Errno = 8579
+	ERROR_DS_NO_OBJECT_MOVE_IN_SCHEMA_NC                                      syscall.Errno = 8580
+	ERROR_DS_MODIFYDN_DISALLOWED_BY_FLAG                                      syscall.Errno = 8581
+	ERROR_DS_MODIFYDN_WRONG_GRANDPARENT                                       syscall.Errno = 8582
+	ERROR_DS_NAME_ERROR_TRUST_REFERRAL                                        syscall.Errno = 8583
+	ERROR_NOT_SUPPORTED_ON_STANDARD_SERVER                                    syscall.Errno = 8584
+	ERROR_DS_CANT_ACCESS_REMOTE_PART_OF_AD                                    syscall.Errno = 8585
+	ERROR_DS_CR_IMPOSSIBLE_TO_VALIDATE_V2                                     syscall.Errno = 8586
+	ERROR_DS_THREAD_LIMIT_EXCEEDED                                            syscall.Errno = 8587
+	ERROR_DS_NOT_CLOSEST                                                      syscall.Errno = 8588
+	ERROR_DS_CANT_DERIVE_SPN_WITHOUT_SERVER_REF                               syscall.Errno = 8589
+	ERROR_DS_SINGLE_USER_MODE_FAILED                                          syscall.Errno = 8590
+	ERROR_DS_NTDSCRIPT_SYNTAX_ERROR                                           syscall.Errno = 8591
+	ERROR_DS_NTDSCRIPT_PROCESS_ERROR                                          syscall.Errno = 8592
+	ERROR_DS_DIFFERENT_REPL_EPOCHS                                            syscall.Errno = 8593
+	ERROR_DS_DRS_EXTENSIONS_CHANGED                                           syscall.Errno = 8594
+	ERROR_DS_REPLICA_SET_CHANGE_NOT_ALLOWED_ON_DISABLED_CR                    syscall.Errno = 8595
+	ERROR_DS_NO_MSDS_INTID                                                    syscall.Errno = 8596
+	ERROR_DS_DUP_MSDS_INTID                                                   syscall.Errno = 8597
+	ERROR_DS_EXISTS_IN_RDNATTID                                               syscall.Errno = 8598
+	ERROR_DS_AUTHORIZATION_FAILED                                             syscall.Errno = 8599
+	ERROR_DS_INVALID_SCRIPT                                                   syscall.Errno = 8600
+	ERROR_DS_REMOTE_CROSSREF_OP_FAILED                                        syscall.Errno = 8601
+	ERROR_DS_CROSS_REF_BUSY                                                   syscall.Errno = 8602
+	ERROR_DS_CANT_DERIVE_SPN_FOR_DELETED_DOMAIN                               syscall.Errno = 8603
+	ERROR_DS_CANT_DEMOTE_WITH_WRITEABLE_NC                                    syscall.Errno = 8604
+	ERROR_DS_DUPLICATE_ID_FOUND                                               syscall.Errno = 8605
+	ERROR_DS_INSUFFICIENT_ATTR_TO_CREATE_OBJECT                               syscall.Errno = 8606
+	ERROR_DS_GROUP_CONVERSION_ERROR                                           syscall.Errno = 8607
+	ERROR_DS_CANT_MOVE_APP_BASIC_GROUP                                        syscall.Errno = 8608
+	ERROR_DS_CANT_MOVE_APP_QUERY_GROUP                                        syscall.Errno = 8609
+	ERROR_DS_ROLE_NOT_VERIFIED                                                syscall.Errno = 8610
+	ERROR_DS_WKO_CONTAINER_CANNOT_BE_SPECIAL                                  syscall.Errno = 8611
+	ERROR_DS_DOMAIN_RENAME_IN_PROGRESS                                        syscall.Errno = 8612
+	ERROR_DS_EXISTING_AD_CHILD_NC                                             syscall.Errno = 8613
+	ERROR_DS_REPL_LIFETIME_EXCEEDED                                           syscall.Errno = 8614
+	ERROR_DS_DISALLOWED_IN_SYSTEM_CONTAINER                                   syscall.Errno = 8615
+	ERROR_DS_LDAP_SEND_QUEUE_FULL                                             syscall.Errno = 8616
+	ERROR_DS_DRA_OUT_SCHEDULE_WINDOW                                          syscall.Errno = 8617
+	ERROR_DS_POLICY_NOT_KNOWN                                                 syscall.Errno = 8618
+	ERROR_NO_SITE_SETTINGS_OBJECT                                             syscall.Errno = 8619
+	ERROR_NO_SECRETS                                                          syscall.Errno = 8620
+	ERROR_NO_WRITABLE_DC_FOUND                                                syscall.Errno = 8621
+	ERROR_DS_NO_SERVER_OBJECT                                                 syscall.Errno = 8622
+	ERROR_DS_NO_NTDSA_OBJECT                                                  syscall.Errno = 8623
+	ERROR_DS_NON_ASQ_SEARCH                                                   syscall.Errno = 8624
+	ERROR_DS_AUDIT_FAILURE                                                    syscall.Errno = 8625
+	ERROR_DS_INVALID_SEARCH_FLAG_SUBTREE                                      syscall.Errno = 8626
+	ERROR_DS_INVALID_SEARCH_FLAG_TUPLE                                        syscall.Errno = 8627
+	ERROR_DS_HIERARCHY_TABLE_TOO_DEEP                                         syscall.Errno = 8628
+	ERROR_DS_DRA_CORRUPT_UTD_VECTOR                                           syscall.Errno = 8629
+	ERROR_DS_DRA_SECRETS_DENIED                                               syscall.Errno = 8630
+	ERROR_DS_RESERVED_MAPI_ID                                                 syscall.Errno = 8631
+	ERROR_DS_MAPI_ID_NOT_AVAILABLE                                            syscall.Errno = 8632
+	ERROR_DS_DRA_MISSING_KRBTGT_SECRET                                        syscall.Errno = 8633
+	ERROR_DS_DOMAIN_NAME_EXISTS_IN_FOREST                                     syscall.Errno = 8634
+	ERROR_DS_FLAT_NAME_EXISTS_IN_FOREST                                       syscall.Errno = 8635
+	ERROR_INVALID_USER_PRINCIPAL_NAME                                         syscall.Errno = 8636
+	ERROR_DS_OID_MAPPED_GROUP_CANT_HAVE_MEMBERS                               syscall.Errno = 8637
+	ERROR_DS_OID_NOT_FOUND                                                    syscall.Errno = 8638
+	ERROR_DS_DRA_RECYCLED_TARGET                                              syscall.Errno = 8639
+	ERROR_DS_DISALLOWED_NC_REDIRECT                                           syscall.Errno = 8640
+	ERROR_DS_HIGH_ADLDS_FFL                                                   syscall.Errno = 8641
+	ERROR_DS_HIGH_DSA_VERSION                                                 syscall.Errno = 8642
+	ERROR_DS_LOW_ADLDS_FFL                                                    syscall.Errno = 8643
+	ERROR_DOMAIN_SID_SAME_AS_LOCAL_WORKSTATION                                syscall.Errno = 8644
+	ERROR_DS_UNDELETE_SAM_VALIDATION_FAILED                                   syscall.Errno = 8645
+	ERROR_INCORRECT_ACCOUNT_TYPE                                              syscall.Errno = 8646
+	ERROR_DS_SPN_VALUE_NOT_UNIQUE_IN_FOREST                                   syscall.Errno = 8647
+	ERROR_DS_UPN_VALUE_NOT_UNIQUE_IN_FOREST                                   syscall.Errno = 8648
+	ERROR_DS_MISSING_FOREST_TRUST                                             syscall.Errno = 8649
+	ERROR_DS_VALUE_KEY_NOT_UNIQUE                                             syscall.Errno = 8650
+	DNS_ERROR_RESPONSE_CODES_BASE                                             syscall.Errno = 9000
+	DNS_ERROR_RCODE_NO_ERROR                                                                = ERROR_SUCCESS
+	DNS_ERROR_MASK                                                            syscall.Errno = 0x00002328
+	DNS_ERROR_RCODE_FORMAT_ERROR                                              syscall.Errno = 9001
+	DNS_ERROR_RCODE_SERVER_FAILURE                                            syscall.Errno = 9002
+	DNS_ERROR_RCODE_NAME_ERROR                                                syscall.Errno = 9003
+	DNS_ERROR_RCODE_NOT_IMPLEMENTED                                           syscall.Errno = 9004
+	DNS_ERROR_RCODE_REFUSED                                                   syscall.Errno = 9005
+	DNS_ERROR_RCODE_YXDOMAIN                                                  syscall.Errno = 9006
+	DNS_ERROR_RCODE_YXRRSET                                                   syscall.Errno = 9007
+	DNS_ERROR_RCODE_NXRRSET                                                   syscall.Errno = 9008
+	DNS_ERROR_RCODE_NOTAUTH                                                   syscall.Errno = 9009
+	DNS_ERROR_RCODE_NOTZONE                                                   syscall.Errno = 9010
+	DNS_ERROR_RCODE_BADSIG                                                    syscall.Errno = 9016
+	DNS_ERROR_RCODE_BADKEY                                                    syscall.Errno = 9017
+	DNS_ERROR_RCODE_BADTIME                                                   syscall.Errno = 9018
+	DNS_ERROR_RCODE_LAST                                                                    = DNS_ERROR_RCODE_BADTIME
+	DNS_ERROR_DNSSEC_BASE                                                     syscall.Errno = 9100
+	DNS_ERROR_KEYMASTER_REQUIRED                                              syscall.Errno = 9101
+	DNS_ERROR_NOT_ALLOWED_ON_SIGNED_ZONE                                      syscall.Errno = 9102
+	DNS_ERROR_NSEC3_INCOMPATIBLE_WITH_RSA_SHA1                                syscall.Errno = 9103
+	DNS_ERROR_NOT_ENOUGH_SIGNING_KEY_DESCRIPTORS                              syscall.Errno = 9104
+	DNS_ERROR_UNSUPPORTED_ALGORITHM                                           syscall.Errno = 9105
+	DNS_ERROR_INVALID_KEY_SIZE                                                syscall.Errno = 9106
+	DNS_ERROR_SIGNING_KEY_NOT_ACCESSIBLE                                      syscall.Errno = 9107
+	DNS_ERROR_KSP_DOES_NOT_SUPPORT_PROTECTION                                 syscall.Errno = 9108
+	DNS_ERROR_UNEXPECTED_DATA_PROTECTION_ERROR                                syscall.Errno = 9109
+	DNS_ERROR_UNEXPECTED_CNG_ERROR                                            syscall.Errno = 9110
+	DNS_ERROR_UNKNOWN_SIGNING_PARAMETER_VERSION                               syscall.Errno = 9111
+	DNS_ERROR_KSP_NOT_ACCESSIBLE                                              syscall.Errno = 9112
+	DNS_ERROR_TOO_MANY_SKDS                                                   syscall.Errno = 9113
+	DNS_ERROR_INVALID_ROLLOVER_PERIOD                                         syscall.Errno = 9114
+	DNS_ERROR_INVALID_INITIAL_ROLLOVER_OFFSET                                 syscall.Errno = 9115
+	DNS_ERROR_ROLLOVER_IN_PROGRESS                                            syscall.Errno = 9116
+	DNS_ERROR_STANDBY_KEY_NOT_PRESENT                                         syscall.Errno = 9117
+	DNS_ERROR_NOT_ALLOWED_ON_ZSK                                              syscall.Errno = 9118
+	DNS_ERROR_NOT_ALLOWED_ON_ACTIVE_SKD                                       syscall.Errno = 9119
+	DNS_ERROR_ROLLOVER_ALREADY_QUEUED                                         syscall.Errno = 9120
+	DNS_ERROR_NOT_ALLOWED_ON_UNSIGNED_ZONE                                    syscall.Errno = 9121
+	DNS_ERROR_BAD_KEYMASTER                                                   syscall.Errno = 9122
+	DNS_ERROR_INVALID_SIGNATURE_VALIDITY_PERIOD                               syscall.Errno = 9123
+	DNS_ERROR_INVALID_NSEC3_ITERATION_COUNT                                   syscall.Errno = 9124
+	DNS_ERROR_DNSSEC_IS_DISABLED                                              syscall.Errno = 9125
+	DNS_ERROR_INVALID_XML                                                     syscall.Errno = 9126
+	DNS_ERROR_NO_VALID_TRUST_ANCHORS                                          syscall.Errno = 9127
+	DNS_ERROR_ROLLOVER_NOT_POKEABLE                                           syscall.Errno = 9128
+	DNS_ERROR_NSEC3_NAME_COLLISION                                            syscall.Errno = 9129
+	DNS_ERROR_NSEC_INCOMPATIBLE_WITH_NSEC3_RSA_SHA1                           syscall.Errno = 9130
+	DNS_ERROR_PACKET_FMT_BASE                                                 syscall.Errno = 9500
+	DNS_INFO_NO_RECORDS                                                       syscall.Errno = 9501
+	DNS_ERROR_BAD_PACKET                                                      syscall.Errno = 9502
+	DNS_ERROR_NO_PACKET                                                       syscall.Errno = 9503
+	DNS_ERROR_RCODE                                                           syscall.Errno = 9504
+	DNS_ERROR_UNSECURE_PACKET                                                 syscall.Errno = 9505
+	DNS_STATUS_PACKET_UNSECURE                                                              = DNS_ERROR_UNSECURE_PACKET
+	DNS_REQUEST_PENDING                                                       syscall.Errno = 9506
+	DNS_ERROR_NO_MEMORY                                                                     = ERROR_OUTOFMEMORY
+	DNS_ERROR_INVALID_NAME                                                                  = ERROR_INVALID_NAME
+	DNS_ERROR_INVALID_DATA                                                                  = ERROR_INVALID_DATA
+	DNS_ERROR_GENERAL_API_BASE                                                syscall.Errno = 9550
+	DNS_ERROR_INVALID_TYPE                                                    syscall.Errno = 9551
+	DNS_ERROR_INVALID_IP_ADDRESS                                              syscall.Errno = 9552
+	DNS_ERROR_INVALID_PROPERTY                                                syscall.Errno = 9553
+	DNS_ERROR_TRY_AGAIN_LATER                                                 syscall.Errno = 9554
+	DNS_ERROR_NOT_UNIQUE                                                      syscall.Errno = 9555
+	DNS_ERROR_NON_RFC_NAME                                                    syscall.Errno = 9556
+	DNS_STATUS_FQDN                                                           syscall.Errno = 9557
+	DNS_STATUS_DOTTED_NAME                                                    syscall.Errno = 9558
+	DNS_STATUS_SINGLE_PART_NAME                                               syscall.Errno = 9559
+	DNS_ERROR_INVALID_NAME_CHAR                                               syscall.Errno = 9560
+	DNS_ERROR_NUMERIC_NAME                                                    syscall.Errno = 9561
+	DNS_ERROR_NOT_ALLOWED_ON_ROOT_SERVER                                      syscall.Errno = 9562
+	DNS_ERROR_NOT_ALLOWED_UNDER_DELEGATION                                    syscall.Errno = 9563
+	DNS_ERROR_CANNOT_FIND_ROOT_HINTS                                          syscall.Errno = 9564
+	DNS_ERROR_INCONSISTENT_ROOT_HINTS                                         syscall.Errno = 9565
+	DNS_ERROR_DWORD_VALUE_TOO_SMALL                                           syscall.Errno = 9566
+	DNS_ERROR_DWORD_VALUE_TOO_LARGE                                           syscall.Errno = 9567
+	DNS_ERROR_BACKGROUND_LOADING                                              syscall.Errno = 9568
+	DNS_ERROR_NOT_ALLOWED_ON_RODC                                             syscall.Errno = 9569
+	DNS_ERROR_NOT_ALLOWED_UNDER_DNAME                                         syscall.Errno = 9570
+	DNS_ERROR_DELEGATION_REQUIRED                                             syscall.Errno = 9571
+	DNS_ERROR_INVALID_POLICY_TABLE                                            syscall.Errno = 9572
+	DNS_ERROR_ADDRESS_REQUIRED                                                syscall.Errno = 9573
+	DNS_ERROR_ZONE_BASE                                                       syscall.Errno = 9600
+	DNS_ERROR_ZONE_DOES_NOT_EXIST                                             syscall.Errno = 9601
+	DNS_ERROR_NO_ZONE_INFO                                                    syscall.Errno = 9602
+	DNS_ERROR_INVALID_ZONE_OPERATION                                          syscall.Errno = 9603
+	DNS_ERROR_ZONE_CONFIGURATION_ERROR                                        syscall.Errno = 9604
+	DNS_ERROR_ZONE_HAS_NO_SOA_RECORD                                          syscall.Errno = 9605
+	DNS_ERROR_ZONE_HAS_NO_NS_RECORDS                                          syscall.Errno = 9606
+	DNS_ERROR_ZONE_LOCKED                                                     syscall.Errno = 9607
+	DNS_ERROR_ZONE_CREATION_FAILED                                            syscall.Errno = 9608
+	DNS_ERROR_ZONE_ALREADY_EXISTS                                             syscall.Errno = 9609
+	DNS_ERROR_AUTOZONE_ALREADY_EXISTS                                         syscall.Errno = 9610
+	DNS_ERROR_INVALID_ZONE_TYPE                                               syscall.Errno = 9611
+	DNS_ERROR_SECONDARY_REQUIRES_MASTER_IP                                    syscall.Errno = 9612
+	DNS_ERROR_ZONE_NOT_SECONDARY                                              syscall.Errno = 9613
+	DNS_ERROR_NEED_SECONDARY_ADDRESSES                                        syscall.Errno = 9614
+	DNS_ERROR_WINS_INIT_FAILED                                                syscall.Errno = 9615
+	DNS_ERROR_NEED_WINS_SERVERS                                               syscall.Errno = 9616
+	DNS_ERROR_NBSTAT_INIT_FAILED                                              syscall.Errno = 9617
+	DNS_ERROR_SOA_DELETE_INVALID                                              syscall.Errno = 9618
+	DNS_ERROR_FORWARDER_ALREADY_EXISTS                                        syscall.Errno = 9619
+	DNS_ERROR_ZONE_REQUIRES_MASTER_IP                                         syscall.Errno = 9620
+	DNS_ERROR_ZONE_IS_SHUTDOWN                                                syscall.Errno = 9621
+	DNS_ERROR_ZONE_LOCKED_FOR_SIGNING                                         syscall.Errno = 9622
+	DNS_ERROR_DATAFILE_BASE                                                   syscall.Errno = 9650
+	DNS_ERROR_PRIMARY_REQUIRES_DATAFILE                                       syscall.Errno = 9651
+	DNS_ERROR_INVALID_DATAFILE_NAME                                           syscall.Errno = 9652
+	DNS_ERROR_DATAFILE_OPEN_FAILURE                                           syscall.Errno = 9653
+	DNS_ERROR_FILE_WRITEBACK_FAILED                                           syscall.Errno = 9654
+	DNS_ERROR_DATAFILE_PARSING                                                syscall.Errno = 9655
+	DNS_ERROR_DATABASE_BASE                                                   syscall.Errno = 9700
+	DNS_ERROR_RECORD_DOES_NOT_EXIST                                           syscall.Errno = 9701
+	DNS_ERROR_RECORD_FORMAT                                                   syscall.Errno = 9702
+	DNS_ERROR_NODE_CREATION_FAILED                                            syscall.Errno = 9703
+	DNS_ERROR_UNKNOWN_RECORD_TYPE                                             syscall.Errno = 9704
+	DNS_ERROR_RECORD_TIMED_OUT                                                syscall.Errno = 9705
+	DNS_ERROR_NAME_NOT_IN_ZONE                                                syscall.Errno = 9706
+	DNS_ERROR_CNAME_LOOP                                                      syscall.Errno = 9707
+	DNS_ERROR_NODE_IS_CNAME                                                   syscall.Errno = 9708
+	DNS_ERROR_CNAME_COLLISION                                                 syscall.Errno = 9709
+	DNS_ERROR_RECORD_ONLY_AT_ZONE_ROOT                                        syscall.Errno = 9710
+	DNS_ERROR_RECORD_ALREADY_EXISTS                                           syscall.Errno = 9711
+	DNS_ERROR_SECONDARY_DATA                                                  syscall.Errno = 9712
+	DNS_ERROR_NO_CREATE_CACHE_DATA                                            syscall.Errno = 9713
+	DNS_ERROR_NAME_DOES_NOT_EXIST                                             syscall.Errno = 9714
+	DNS_WARNING_PTR_CREATE_FAILED                                             syscall.Errno = 9715
+	DNS_WARNING_DOMAIN_UNDELETED                                              syscall.Errno = 9716
+	DNS_ERROR_DS_UNAVAILABLE                                                  syscall.Errno = 9717
+	DNS_ERROR_DS_ZONE_ALREADY_EXISTS                                          syscall.Errno = 9718
+	DNS_ERROR_NO_BOOTFILE_IF_DS_ZONE                                          syscall.Errno = 9719
+	DNS_ERROR_NODE_IS_DNAME                                                   syscall.Errno = 9720
+	DNS_ERROR_DNAME_COLLISION                                                 syscall.Errno = 9721
+	DNS_ERROR_ALIAS_LOOP                                                      syscall.Errno = 9722
+	DNS_ERROR_OPERATION_BASE                                                  syscall.Errno = 9750
+	DNS_INFO_AXFR_COMPLETE                                                    syscall.Errno = 9751
+	DNS_ERROR_AXFR                                                            syscall.Errno = 9752
+	DNS_INFO_ADDED_LOCAL_WINS                                                 syscall.Errno = 9753
+	DNS_ERROR_SECURE_BASE                                                     syscall.Errno = 9800
+	DNS_STATUS_CONTINUE_NEEDED                                                syscall.Errno = 9801
+	DNS_ERROR_SETUP_BASE                                                      syscall.Errno = 9850
+	DNS_ERROR_NO_TCPIP                                                        syscall.Errno = 9851
+	DNS_ERROR_NO_DNS_SERVERS                                                  syscall.Errno = 9852
+	DNS_ERROR_DP_BASE                                                         syscall.Errno = 9900
+	DNS_ERROR_DP_DOES_NOT_EXIST                                               syscall.Errno = 9901
+	DNS_ERROR_DP_ALREADY_EXISTS                                               syscall.Errno = 9902
+	DNS_ERROR_DP_NOT_ENLISTED                                                 syscall.Errno = 9903
+	DNS_ERROR_DP_ALREADY_ENLISTED                                             syscall.Errno = 9904
+	DNS_ERROR_DP_NOT_AVAILABLE                                                syscall.Errno = 9905
+	DNS_ERROR_DP_FSMO_ERROR                                                   syscall.Errno = 9906
+	DNS_ERROR_RRL_NOT_ENABLED                                                 syscall.Errno = 9911
+	DNS_ERROR_RRL_INVALID_WINDOW_SIZE                                         syscall.Errno = 9912
+	DNS_ERROR_RRL_INVALID_IPV4_PREFIX                                         syscall.Errno = 9913
+	DNS_ERROR_RRL_INVALID_IPV6_PREFIX                                         syscall.Errno = 9914
+	DNS_ERROR_RRL_INVALID_TC_RATE                                             syscall.Errno = 9915
+	DNS_ERROR_RRL_INVALID_LEAK_RATE                                           syscall.Errno = 9916
+	DNS_ERROR_RRL_LEAK_RATE_LESSTHAN_TC_RATE                                  syscall.Errno = 9917
+	DNS_ERROR_VIRTUALIZATION_INSTANCE_ALREADY_EXISTS                          syscall.Errno = 9921
+	DNS_ERROR_VIRTUALIZATION_INSTANCE_DOES_NOT_EXIST                          syscall.Errno = 9922
+	DNS_ERROR_VIRTUALIZATION_TREE_LOCKED                                      syscall.Errno = 9923
+	DNS_ERROR_INVAILD_VIRTUALIZATION_INSTANCE_NAME                            syscall.Errno = 9924
+	DNS_ERROR_DEFAULT_VIRTUALIZATION_INSTANCE                                 syscall.Errno = 9925
+	DNS_ERROR_ZONESCOPE_ALREADY_EXISTS                                        syscall.Errno = 9951
+	DNS_ERROR_ZONESCOPE_DOES_NOT_EXIST                                        syscall.Errno = 9952
+	DNS_ERROR_DEFAULT_ZONESCOPE                                               syscall.Errno = 9953
+	DNS_ERROR_INVALID_ZONESCOPE_NAME                                          syscall.Errno = 9954
+	DNS_ERROR_NOT_ALLOWED_WITH_ZONESCOPES                                     syscall.Errno = 9955
+	DNS_ERROR_LOAD_ZONESCOPE_FAILED                                           syscall.Errno = 9956
+	DNS_ERROR_ZONESCOPE_FILE_WRITEBACK_FAILED                                 syscall.Errno = 9957
+	DNS_ERROR_INVALID_SCOPE_NAME                                              syscall.Errno = 9958
+	DNS_ERROR_SCOPE_DOES_NOT_EXIST                                            syscall.Errno = 9959
+	DNS_ERROR_DEFAULT_SCOPE                                                   syscall.Errno = 9960
+	DNS_ERROR_INVALID_SCOPE_OPERATION                                         syscall.Errno = 9961
+	DNS_ERROR_SCOPE_LOCKED                                                    syscall.Errno = 9962
+	DNS_ERROR_SCOPE_ALREADY_EXISTS                                            syscall.Errno = 9963
+	DNS_ERROR_POLICY_ALREADY_EXISTS                                           syscall.Errno = 9971
+	DNS_ERROR_POLICY_DOES_NOT_EXIST                                           syscall.Errno = 9972
+	DNS_ERROR_POLICY_INVALID_CRITERIA                                         syscall.Errno = 9973
+	DNS_ERROR_POLICY_INVALID_SETTINGS                                         syscall.Errno = 9974
+	DNS_ERROR_CLIENT_SUBNET_IS_ACCESSED                                       syscall.Errno = 9975
+	DNS_ERROR_CLIENT_SUBNET_DOES_NOT_EXIST                                    syscall.Errno = 9976
+	DNS_ERROR_CLIENT_SUBNET_ALREADY_EXISTS                                    syscall.Errno = 9977
+	DNS_ERROR_SUBNET_DOES_NOT_EXIST                                           syscall.Errno = 9978
+	DNS_ERROR_SUBNET_ALREADY_EXISTS                                           syscall.Errno = 9979
+	DNS_ERROR_POLICY_LOCKED                                                   syscall.Errno = 9980
+	DNS_ERROR_POLICY_INVALID_WEIGHT                                           syscall.Errno = 9981
+	DNS_ERROR_POLICY_INVALID_NAME                                             syscall.Errno = 9982
+	DNS_ERROR_POLICY_MISSING_CRITERIA                                         syscall.Errno = 9983
+	DNS_ERROR_INVALID_CLIENT_SUBNET_NAME                                      syscall.Errno = 9984
+	DNS_ERROR_POLICY_PROCESSING_ORDER_INVALID                                 syscall.Errno = 9985
+	DNS_ERROR_POLICY_SCOPE_MISSING                                            syscall.Errno = 9986
+	DNS_ERROR_POLICY_SCOPE_NOT_ALLOWED                                        syscall.Errno = 9987
+	DNS_ERROR_SERVERSCOPE_IS_REFERENCED                                       syscall.Errno = 9988
+	DNS_ERROR_ZONESCOPE_IS_REFERENCED                                         syscall.Errno = 9989
+	DNS_ERROR_POLICY_INVALID_CRITERIA_CLIENT_SUBNET                           syscall.Errno = 9990
+	DNS_ERROR_POLICY_INVALID_CRITERIA_TRANSPORT_PROTOCOL                      syscall.Errno = 9991
+	DNS_ERROR_POLICY_INVALID_CRITERIA_NETWORK_PROTOCOL                        syscall.Errno = 9992
+	DNS_ERROR_POLICY_INVALID_CRITERIA_INTERFACE                               syscall.Errno = 9993
+	DNS_ERROR_POLICY_INVALID_CRITERIA_FQDN                                    syscall.Errno = 9994
+	DNS_ERROR_POLICY_INVALID_CRITERIA_QUERY_TYPE                              syscall.Errno = 9995
+	DNS_ERROR_POLICY_INVALID_CRITERIA_TIME_OF_DAY                             syscall.Errno = 9996
+	WSABASEERR                                                                syscall.Errno = 10000
+	WSAEINTR                                                                  syscall.Errno = 10004
+	WSAEBADF                                                                  syscall.Errno = 10009
+	WSAEACCES                                                                 syscall.Errno = 10013
+	WSAEFAULT                                                                 syscall.Errno = 10014
+	WSAEINVAL                                                                 syscall.Errno = 10022
+	WSAEMFILE                                                                 syscall.Errno = 10024
+	WSAEWOULDBLOCK                                                            syscall.Errno = 10035
+	WSAEINPROGRESS                                                            syscall.Errno = 10036
+	WSAEALREADY                                                               syscall.Errno = 10037
+	WSAENOTSOCK                                                               syscall.Errno = 10038
+	WSAEDESTADDRREQ                                                           syscall.Errno = 10039
+	WSAEMSGSIZE                                                               syscall.Errno = 10040
+	WSAEPROTOTYPE                                                             syscall.Errno = 10041
+	WSAENOPROTOOPT                                                            syscall.Errno = 10042
+	WSAEPROTONOSUPPORT                                                        syscall.Errno = 10043
+	WSAESOCKTNOSUPPORT                                                        syscall.Errno = 10044
+	WSAEOPNOTSUPP                                                             syscall.Errno = 10045
+	WSAEPFNOSUPPORT                                                           syscall.Errno = 10046
+	WSAEAFNOSUPPORT                                                           syscall.Errno = 10047
+	WSAEADDRINUSE                                                             syscall.Errno = 10048
+	WSAEADDRNOTAVAIL                                                          syscall.Errno = 10049
+	WSAENETDOWN                                                               syscall.Errno = 10050
+	WSAENETUNREACH                                                            syscall.Errno = 10051
+	WSAENETRESET                                                              syscall.Errno = 10052
+	WSAECONNABORTED                                                           syscall.Errno = 10053
+	WSAECONNRESET                                                             syscall.Errno = 10054
+	WSAENOBUFS                                                                syscall.Errno = 10055
+	WSAEISCONN                                                                syscall.Errno = 10056
+	WSAENOTCONN                                                               syscall.Errno = 10057
+	WSAESHUTDOWN                                                              syscall.Errno = 10058
+	WSAETOOMANYREFS                                                           syscall.Errno = 10059
+	WSAETIMEDOUT                                                              syscall.Errno = 10060
+	WSAECONNREFUSED                                                           syscall.Errno = 10061
+	WSAELOOP                                                                  syscall.Errno = 10062
+	WSAENAMETOOLONG                                                           syscall.Errno = 10063
+	WSAEHOSTDOWN                                                              syscall.Errno = 10064
+	WSAEHOSTUNREACH                                                           syscall.Errno = 10065
+	WSAENOTEMPTY                                                              syscall.Errno = 10066
+	WSAEPROCLIM                                                               syscall.Errno = 10067
+	WSAEUSERS                                                                 syscall.Errno = 10068
+	WSAEDQUOT                                                                 syscall.Errno = 10069
+	WSAESTALE                                                                 syscall.Errno = 10070
+	WSAEREMOTE                                                                syscall.Errno = 10071
+	WSASYSNOTREADY                                                            syscall.Errno = 10091
+	WSAVERNOTSUPPORTED                                                        syscall.Errno = 10092
+	WSANOTINITIALISED                                                         syscall.Errno = 10093
+	WSAEDISCON                                                                syscall.Errno = 10101
+	WSAENOMORE                                                                syscall.Errno = 10102
+	WSAECANCELLED                                                             syscall.Errno = 10103
+	WSAEINVALIDPROCTABLE                                                      syscall.Errno = 10104
+	WSAEINVALIDPROVIDER                                                       syscall.Errno = 10105
+	WSAEPROVIDERFAILEDINIT                                                    syscall.Errno = 10106
+	WSASYSCALLFAILURE                                                         syscall.Errno = 10107
+	WSASERVICE_NOT_FOUND                                                      syscall.Errno = 10108
+	WSATYPE_NOT_FOUND                                                         syscall.Errno = 10109
+	WSA_E_NO_MORE                                                             syscall.Errno = 10110
+	WSA_E_CANCELLED                                                           syscall.Errno = 10111
+	WSAEREFUSED                                                               syscall.Errno = 10112
+	WSAHOST_NOT_FOUND                                                         syscall.Errno = 11001
+	WSATRY_AGAIN                                                              syscall.Errno = 11002
+	WSANO_RECOVERY                                                            syscall.Errno = 11003
+	WSANO_DATA                                                                syscall.Errno = 11004
+	WSA_QOS_RECEIVERS                                                         syscall.Errno = 11005
+	WSA_QOS_SENDERS                                                           syscall.Errno = 11006
+	WSA_QOS_NO_SENDERS                                                        syscall.Errno = 11007
+	WSA_QOS_NO_RECEIVERS                                                      syscall.Errno = 11008
+	WSA_QOS_REQUEST_CONFIRMED                                                 syscall.Errno = 11009
+	WSA_QOS_ADMISSION_FAILURE                                                 syscall.Errno = 11010
+	WSA_QOS_POLICY_FAILURE                                                    syscall.Errno = 11011
+	WSA_QOS_BAD_STYLE                                                         syscall.Errno = 11012
+	WSA_QOS_BAD_OBJECT                                                        syscall.Errno = 11013
+	WSA_QOS_TRAFFIC_CTRL_ERROR                                                syscall.Errno = 11014
+	WSA_QOS_GENERIC_ERROR                                                     syscall.Errno = 11015
+	WSA_QOS_ESERVICETYPE                                                      syscall.Errno = 11016
+	WSA_QOS_EFLOWSPEC                                                         syscall.Errno = 11017
+	WSA_QOS_EPROVSPECBUF                                                      syscall.Errno = 11018
+	WSA_QOS_EFILTERSTYLE                                                      syscall.Errno = 11019
+	WSA_QOS_EFILTERTYPE                                                       syscall.Errno = 11020
+	WSA_QOS_EFILTERCOUNT                                                      syscall.Errno = 11021
+	WSA_QOS_EOBJLENGTH                                                        syscall.Errno = 11022
+	WSA_QOS_EFLOWCOUNT                                                        syscall.Errno = 11023
+	WSA_QOS_EUNKOWNPSOBJ                                                      syscall.Errno = 11024
+	WSA_QOS_EPOLICYOBJ                                                        syscall.Errno = 11025
+	WSA_QOS_EFLOWDESC                                                         syscall.Errno = 11026
+	WSA_QOS_EPSFLOWSPEC                                                       syscall.Errno = 11027
+	WSA_QOS_EPSFILTERSPEC                                                     syscall.Errno = 11028
+	WSA_QOS_ESDMODEOBJ                                                        syscall.Errno = 11029
+	WSA_QOS_ESHAPERATEOBJ                                                     syscall.Errno = 11030
+	WSA_QOS_RESERVED_PETYPE                                                   syscall.Errno = 11031
+	WSA_SECURE_HOST_NOT_FOUND                                                 syscall.Errno = 11032
+	WSA_IPSEC_NAME_POLICY_ERROR                                               syscall.Errno = 11033
+	ERROR_IPSEC_QM_POLICY_EXISTS                                              syscall.Errno = 13000
+	ERROR_IPSEC_QM_POLICY_NOT_FOUND                                           syscall.Errno = 13001
+	ERROR_IPSEC_QM_POLICY_IN_USE                                              syscall.Errno = 13002
+	ERROR_IPSEC_MM_POLICY_EXISTS                                              syscall.Errno = 13003
+	ERROR_IPSEC_MM_POLICY_NOT_FOUND                                           syscall.Errno = 13004
+	ERROR_IPSEC_MM_POLICY_IN_USE                                              syscall.Errno = 13005
+	ERROR_IPSEC_MM_FILTER_EXISTS                                              syscall.Errno = 13006
+	ERROR_IPSEC_MM_FILTER_NOT_FOUND                                           syscall.Errno = 13007
+	ERROR_IPSEC_TRANSPORT_FILTER_EXISTS                                       syscall.Errno = 13008
+	ERROR_IPSEC_TRANSPORT_FILTER_NOT_FOUND                                    syscall.Errno = 13009
+	ERROR_IPSEC_MM_AUTH_EXISTS                                                syscall.Errno = 13010
+	ERROR_IPSEC_MM_AUTH_NOT_FOUND                                             syscall.Errno = 13011
+	ERROR_IPSEC_MM_AUTH_IN_USE                                                syscall.Errno = 13012
+	ERROR_IPSEC_DEFAULT_MM_POLICY_NOT_FOUND                                   syscall.Errno = 13013
+	ERROR_IPSEC_DEFAULT_MM_AUTH_NOT_FOUND                                     syscall.Errno = 13014
+	ERROR_IPSEC_DEFAULT_QM_POLICY_NOT_FOUND                                   syscall.Errno = 13015
+	ERROR_IPSEC_TUNNEL_FILTER_EXISTS                                          syscall.Errno = 13016
+	ERROR_IPSEC_TUNNEL_FILTER_NOT_FOUND                                       syscall.Errno = 13017
+	ERROR_IPSEC_MM_FILTER_PENDING_DELETION                                    syscall.Errno = 13018
+	ERROR_IPSEC_TRANSPORT_FILTER_PENDING_DELETION                             syscall.Errno = 13019
+	ERROR_IPSEC_TUNNEL_FILTER_PENDING_DELETION                                syscall.Errno = 13020
+	ERROR_IPSEC_MM_POLICY_PENDING_DELETION                                    syscall.Errno = 13021
+	ERROR_IPSEC_MM_AUTH_PENDING_DELETION                                      syscall.Errno = 13022
+	ERROR_IPSEC_QM_POLICY_PENDING_DELETION                                    syscall.Errno = 13023
+	WARNING_IPSEC_MM_POLICY_PRUNED                                            syscall.Errno = 13024
+	WARNING_IPSEC_QM_POLICY_PRUNED                                            syscall.Errno = 13025
+	ERROR_IPSEC_IKE_NEG_STATUS_BEGIN                                          syscall.Errno = 13800
+	ERROR_IPSEC_IKE_AUTH_FAIL                                                 syscall.Errno = 13801
+	ERROR_IPSEC_IKE_ATTRIB_FAIL                                               syscall.Errno = 13802
+	ERROR_IPSEC_IKE_NEGOTIATION_PENDING                                       syscall.Errno = 13803
+	ERROR_IPSEC_IKE_GENERAL_PROCESSING_ERROR                                  syscall.Errno = 13804
+	ERROR_IPSEC_IKE_TIMED_OUT                                                 syscall.Errno = 13805
+	ERROR_IPSEC_IKE_NO_CERT                                                   syscall.Errno = 13806
+	ERROR_IPSEC_IKE_SA_DELETED                                                syscall.Errno = 13807
+	ERROR_IPSEC_IKE_SA_REAPED                                                 syscall.Errno = 13808
+	ERROR_IPSEC_IKE_MM_ACQUIRE_DROP                                           syscall.Errno = 13809
+	ERROR_IPSEC_IKE_QM_ACQUIRE_DROP                                           syscall.Errno = 13810
+	ERROR_IPSEC_IKE_QUEUE_DROP_MM                                             syscall.Errno = 13811
+	ERROR_IPSEC_IKE_QUEUE_DROP_NO_MM                                          syscall.Errno = 13812
+	ERROR_IPSEC_IKE_DROP_NO_RESPONSE                                          syscall.Errno = 13813
+	ERROR_IPSEC_IKE_MM_DELAY_DROP                                             syscall.Errno = 13814
+	ERROR_IPSEC_IKE_QM_DELAY_DROP                                             syscall.Errno = 13815
+	ERROR_IPSEC_IKE_ERROR                                                     syscall.Errno = 13816
+	ERROR_IPSEC_IKE_CRL_FAILED                                                syscall.Errno = 13817
+	ERROR_IPSEC_IKE_INVALID_KEY_USAGE                                         syscall.Errno = 13818
+	ERROR_IPSEC_IKE_INVALID_CERT_TYPE                                         syscall.Errno = 13819
+	ERROR_IPSEC_IKE_NO_PRIVATE_KEY                                            syscall.Errno = 13820
+	ERROR_IPSEC_IKE_SIMULTANEOUS_REKEY                                        syscall.Errno = 13821
+	ERROR_IPSEC_IKE_DH_FAIL                                                   syscall.Errno = 13822
+	ERROR_IPSEC_IKE_CRITICAL_PAYLOAD_NOT_RECOGNIZED                           syscall.Errno = 13823
+	ERROR_IPSEC_IKE_INVALID_HEADER                                            syscall.Errno = 13824
+	ERROR_IPSEC_IKE_NO_POLICY                                                 syscall.Errno = 13825
+	ERROR_IPSEC_IKE_INVALID_SIGNATURE                                         syscall.Errno = 13826
+	ERROR_IPSEC_IKE_KERBEROS_ERROR                                            syscall.Errno = 13827
+	ERROR_IPSEC_IKE_NO_PUBLIC_KEY                                             syscall.Errno = 13828
+	ERROR_IPSEC_IKE_PROCESS_ERR                                               syscall.Errno = 13829
+	ERROR_IPSEC_IKE_PROCESS_ERR_SA                                            syscall.Errno = 13830
+	ERROR_IPSEC_IKE_PROCESS_ERR_PROP                                          syscall.Errno = 13831
+	ERROR_IPSEC_IKE_PROCESS_ERR_TRANS                                         syscall.Errno = 13832
+	ERROR_IPSEC_IKE_PROCESS_ERR_KE                                            syscall.Errno = 13833
+	ERROR_IPSEC_IKE_PROCESS_ERR_ID                                            syscall.Errno = 13834
+	ERROR_IPSEC_IKE_PROCESS_ERR_CERT                                          syscall.Errno = 13835
+	ERROR_IPSEC_IKE_PROCESS_ERR_CERT_REQ                                      syscall.Errno = 13836
+	ERROR_IPSEC_IKE_PROCESS_ERR_HASH                                          syscall.Errno = 13837
+	ERROR_IPSEC_IKE_PROCESS_ERR_SIG                                           syscall.Errno = 13838
+	ERROR_IPSEC_IKE_PROCESS_ERR_NONCE                                         syscall.Errno = 13839
+	ERROR_IPSEC_IKE_PROCESS_ERR_NOTIFY                                        syscall.Errno = 13840
+	ERROR_IPSEC_IKE_PROCESS_ERR_DELETE                                        syscall.Errno = 13841
+	ERROR_IPSEC_IKE_PROCESS_ERR_VENDOR                                        syscall.Errno = 13842
+	ERROR_IPSEC_IKE_INVALID_PAYLOAD                                           syscall.Errno = 13843
+	ERROR_IPSEC_IKE_LOAD_SOFT_SA                                              syscall.Errno = 13844
+	ERROR_IPSEC_IKE_SOFT_SA_TORN_DOWN                                         syscall.Errno = 13845
+	ERROR_IPSEC_IKE_INVALID_COOKIE                                            syscall.Errno = 13846
+	ERROR_IPSEC_IKE_NO_PEER_CERT                                              syscall.Errno = 13847
+	ERROR_IPSEC_IKE_PEER_CRL_FAILED                                           syscall.Errno = 13848
+	ERROR_IPSEC_IKE_POLICY_CHANGE                                             syscall.Errno = 13849
+	ERROR_IPSEC_IKE_NO_MM_POLICY                                              syscall.Errno = 13850
+	ERROR_IPSEC_IKE_NOTCBPRIV                                                 syscall.Errno = 13851
+	ERROR_IPSEC_IKE_SECLOADFAIL                                               syscall.Errno = 13852
+	ERROR_IPSEC_IKE_FAILSSPINIT                                               syscall.Errno = 13853
+	ERROR_IPSEC_IKE_FAILQUERYSSP                                              syscall.Errno = 13854
+	ERROR_IPSEC_IKE_SRVACQFAIL                                                syscall.Errno = 13855
+	ERROR_IPSEC_IKE_SRVQUERYCRED                                              syscall.Errno = 13856
+	ERROR_IPSEC_IKE_GETSPIFAIL                                                syscall.Errno = 13857
+	ERROR_IPSEC_IKE_INVALID_FILTER                                            syscall.Errno = 13858
+	ERROR_IPSEC_IKE_OUT_OF_MEMORY                                             syscall.Errno = 13859
+	ERROR_IPSEC_IKE_ADD_UPDATE_KEY_FAILED                                     syscall.Errno = 13860
+	ERROR_IPSEC_IKE_INVALID_POLICY                                            syscall.Errno = 13861
+	ERROR_IPSEC_IKE_UNKNOWN_DOI                                               syscall.Errno = 13862
+	ERROR_IPSEC_IKE_INVALID_SITUATION                                         syscall.Errno = 13863
+	ERROR_IPSEC_IKE_DH_FAILURE                                                syscall.Errno = 13864
+	ERROR_IPSEC_IKE_INVALID_GROUP                                             syscall.Errno = 13865
+	ERROR_IPSEC_IKE_ENCRYPT                                                   syscall.Errno = 13866
+	ERROR_IPSEC_IKE_DECRYPT                                                   syscall.Errno = 13867
+	ERROR_IPSEC_IKE_POLICY_MATCH                                              syscall.Errno = 13868
+	ERROR_IPSEC_IKE_UNSUPPORTED_ID                                            syscall.Errno = 13869
+	ERROR_IPSEC_IKE_INVALID_HASH                                              syscall.Errno = 13870
+	ERROR_IPSEC_IKE_INVALID_HASH_ALG                                          syscall.Errno = 13871
+	ERROR_IPSEC_IKE_INVALID_HASH_SIZE                                         syscall.Errno = 13872
+	ERROR_IPSEC_IKE_INVALID_ENCRYPT_ALG                                       syscall.Errno = 13873
+	ERROR_IPSEC_IKE_INVALID_AUTH_ALG                                          syscall.Errno = 13874
+	ERROR_IPSEC_IKE_INVALID_SIG                                               syscall.Errno = 13875
+	ERROR_IPSEC_IKE_LOAD_FAILED                                               syscall.Errno = 13876
+	ERROR_IPSEC_IKE_RPC_DELETE                                                syscall.Errno = 13877
+	ERROR_IPSEC_IKE_BENIGN_REINIT                                             syscall.Errno = 13878
+	ERROR_IPSEC_IKE_INVALID_RESPONDER_LIFETIME_NOTIFY                         syscall.Errno = 13879
+	ERROR_IPSEC_IKE_INVALID_MAJOR_VERSION                                     syscall.Errno = 13880
+	ERROR_IPSEC_IKE_INVALID_CERT_KEYLEN                                       syscall.Errno = 13881
+	ERROR_IPSEC_IKE_MM_LIMIT                                                  syscall.Errno = 13882
+	ERROR_IPSEC_IKE_NEGOTIATION_DISABLED                                      syscall.Errno = 13883
+	ERROR_IPSEC_IKE_QM_LIMIT                                                  syscall.Errno = 13884
+	ERROR_IPSEC_IKE_MM_EXPIRED                                                syscall.Errno = 13885
+	ERROR_IPSEC_IKE_PEER_MM_ASSUMED_INVALID                                   syscall.Errno = 13886
+	ERROR_IPSEC_IKE_CERT_CHAIN_POLICY_MISMATCH                                syscall.Errno = 13887
+	ERROR_IPSEC_IKE_UNEXPECTED_MESSAGE_ID                                     syscall.Errno = 13888
+	ERROR_IPSEC_IKE_INVALID_AUTH_PAYLOAD                                      syscall.Errno = 13889
+	ERROR_IPSEC_IKE_DOS_COOKIE_SENT                                           syscall.Errno = 13890
+	ERROR_IPSEC_IKE_SHUTTING_DOWN                                             syscall.Errno = 13891
+	ERROR_IPSEC_IKE_CGA_AUTH_FAILED                                           syscall.Errno = 13892
+	ERROR_IPSEC_IKE_PROCESS_ERR_NATOA                                         syscall.Errno = 13893
+	ERROR_IPSEC_IKE_INVALID_MM_FOR_QM                                         syscall.Errno = 13894
+	ERROR_IPSEC_IKE_QM_EXPIRED                                                syscall.Errno = 13895
+	ERROR_IPSEC_IKE_TOO_MANY_FILTERS                                          syscall.Errno = 13896
+	ERROR_IPSEC_IKE_NEG_STATUS_END                                            syscall.Errno = 13897
+	ERROR_IPSEC_IKE_KILL_DUMMY_NAP_TUNNEL                                     syscall.Errno = 13898
+	ERROR_IPSEC_IKE_INNER_IP_ASSIGNMENT_FAILURE                               syscall.Errno = 13899
+	ERROR_IPSEC_IKE_REQUIRE_CP_PAYLOAD_MISSING                                syscall.Errno = 13900
+	ERROR_IPSEC_KEY_MODULE_IMPERSONATION_NEGOTIATION_PENDING                  syscall.Errno = 13901
+	ERROR_IPSEC_IKE_COEXISTENCE_SUPPRESS                                      syscall.Errno = 13902
+	ERROR_IPSEC_IKE_RATELIMIT_DROP                                            syscall.Errno = 13903
+	ERROR_IPSEC_IKE_PEER_DOESNT_SUPPORT_MOBIKE                                syscall.Errno = 13904
+	ERROR_IPSEC_IKE_AUTHORIZATION_FAILURE                                     syscall.Errno = 13905
+	ERROR_IPSEC_IKE_STRONG_CRED_AUTHORIZATION_FAILURE                         syscall.Errno = 13906
+	ERROR_IPSEC_IKE_AUTHORIZATION_FAILURE_WITH_OPTIONAL_RETRY                 syscall.Errno = 13907
+	ERROR_IPSEC_IKE_STRONG_CRED_AUTHORIZATION_AND_CERTMAP_FAILURE             syscall.Errno = 13908
+	ERROR_IPSEC_IKE_NEG_STATUS_EXTENDED_END                                   syscall.Errno = 13909
+	ERROR_IPSEC_BAD_SPI                                                       syscall.Errno = 13910
+	ERROR_IPSEC_SA_LIFETIME_EXPIRED                                           syscall.Errno = 13911
+	ERROR_IPSEC_WRONG_SA                                                      syscall.Errno = 13912
+	ERROR_IPSEC_REPLAY_CHECK_FAILED                                           syscall.Errno = 13913
+	ERROR_IPSEC_INVALID_PACKET                                                syscall.Errno = 13914
+	ERROR_IPSEC_INTEGRITY_CHECK_FAILED                                        syscall.Errno = 13915
+	ERROR_IPSEC_CLEAR_TEXT_DROP                                               syscall.Errno = 13916
+	ERROR_IPSEC_AUTH_FIREWALL_DROP                                            syscall.Errno = 13917
+	ERROR_IPSEC_THROTTLE_DROP                                                 syscall.Errno = 13918
+	ERROR_IPSEC_DOSP_BLOCK                                                    syscall.Errno = 13925
+	ERROR_IPSEC_DOSP_RECEIVED_MULTICAST                                       syscall.Errno = 13926
+	ERROR_IPSEC_DOSP_INVALID_PACKET                                           syscall.Errno = 13927
+	ERROR_IPSEC_DOSP_STATE_LOOKUP_FAILED                                      syscall.Errno = 13928
+	ERROR_IPSEC_DOSP_MAX_ENTRIES                                              syscall.Errno = 13929
+	ERROR_IPSEC_DOSP_KEYMOD_NOT_ALLOWED                                       syscall.Errno = 13930
+	ERROR_IPSEC_DOSP_NOT_INSTALLED                                            syscall.Errno = 13931
+	ERROR_IPSEC_DOSP_MAX_PER_IP_RATELIMIT_QUEUES                              syscall.Errno = 13932
+	ERROR_SXS_SECTION_NOT_FOUND                                               syscall.Errno = 14000
+	ERROR_SXS_CANT_GEN_ACTCTX                                                 syscall.Errno = 14001
+	ERROR_SXS_INVALID_ACTCTXDATA_FORMAT                                       syscall.Errno = 14002
+	ERROR_SXS_ASSEMBLY_NOT_FOUND                                              syscall.Errno = 14003
+	ERROR_SXS_MANIFEST_FORMAT_ERROR                                           syscall.Errno = 14004
+	ERROR_SXS_MANIFEST_PARSE_ERROR                                            syscall.Errno = 14005
+	ERROR_SXS_ACTIVATION_CONTEXT_DISABLED                                     syscall.Errno = 14006
+	ERROR_SXS_KEY_NOT_FOUND                                                   syscall.Errno = 14007
+	ERROR_SXS_VERSION_CONFLICT                                                syscall.Errno = 14008
+	ERROR_SXS_WRONG_SECTION_TYPE                                              syscall.Errno = 14009
+	ERROR_SXS_THREAD_QUERIES_DISABLED                                         syscall.Errno = 14010
+	ERROR_SXS_PROCESS_DEFAULT_ALREADY_SET                                     syscall.Errno = 14011
+	ERROR_SXS_UNKNOWN_ENCODING_GROUP                                          syscall.Errno = 14012
+	ERROR_SXS_UNKNOWN_ENCODING                                                syscall.Errno = 14013
+	ERROR_SXS_INVALID_XML_NAMESPACE_URI                                       syscall.Errno = 14014
+	ERROR_SXS_ROOT_MANIFEST_DEPENDENCY_NOT_INSTALLED                          syscall.Errno = 14015
+	ERROR_SXS_LEAF_MANIFEST_DEPENDENCY_NOT_INSTALLED                          syscall.Errno = 14016
+	ERROR_SXS_INVALID_ASSEMBLY_IDENTITY_ATTRIBUTE                             syscall.Errno = 14017
+	ERROR_SXS_MANIFEST_MISSING_REQUIRED_DEFAULT_NAMESPACE                     syscall.Errno = 14018
+	ERROR_SXS_MANIFEST_INVALID_REQUIRED_DEFAULT_NAMESPACE                     syscall.Errno = 14019
+	ERROR_SXS_PRIVATE_MANIFEST_CROSS_PATH_WITH_REPARSE_POINT                  syscall.Errno = 14020
+	ERROR_SXS_DUPLICATE_DLL_NAME                                              syscall.Errno = 14021
+	ERROR_SXS_DUPLICATE_WINDOWCLASS_NAME                                      syscall.Errno = 14022
+	ERROR_SXS_DUPLICATE_CLSID                                                 syscall.Errno = 14023
+	ERROR_SXS_DUPLICATE_IID                                                   syscall.Errno = 14024
+	ERROR_SXS_DUPLICATE_TLBID                                                 syscall.Errno = 14025
+	ERROR_SXS_DUPLICATE_PROGID                                                syscall.Errno = 14026
+	ERROR_SXS_DUPLICATE_ASSEMBLY_NAME                                         syscall.Errno = 14027
+	ERROR_SXS_FILE_HASH_MISMATCH                                              syscall.Errno = 14028
+	ERROR_SXS_POLICY_PARSE_ERROR                                              syscall.Errno = 14029
+	ERROR_SXS_XML_E_MISSINGQUOTE                                              syscall.Errno = 14030
+	ERROR_SXS_XML_E_COMMENTSYNTAX                                             syscall.Errno = 14031
+	ERROR_SXS_XML_E_BADSTARTNAMECHAR                                          syscall.Errno = 14032
+	ERROR_SXS_XML_E_BADNAMECHAR                                               syscall.Errno = 14033
+	ERROR_SXS_XML_E_BADCHARINSTRING                                           syscall.Errno = 14034
+	ERROR_SXS_XML_E_XMLDECLSYNTAX                                             syscall.Errno = 14035
+	ERROR_SXS_XML_E_BADCHARDATA                                               syscall.Errno = 14036
+	ERROR_SXS_XML_E_MISSINGWHITESPACE                                         syscall.Errno = 14037
+	ERROR_SXS_XML_E_EXPECTINGTAGEND                                           syscall.Errno = 14038
+	ERROR_SXS_XML_E_MISSINGSEMICOLON                                          syscall.Errno = 14039
+	ERROR_SXS_XML_E_UNBALANCEDPAREN                                           syscall.Errno = 14040
+	ERROR_SXS_XML_E_INTERNALERROR                                             syscall.Errno = 14041
+	ERROR_SXS_XML_E_UNEXPECTED_WHITESPACE                                     syscall.Errno = 14042
+	ERROR_SXS_XML_E_INCOMPLETE_ENCODING                                       syscall.Errno = 14043
+	ERROR_SXS_XML_E_MISSING_PAREN                                             syscall.Errno = 14044
+	ERROR_SXS_XML_E_EXPECTINGCLOSEQUOTE                                       syscall.Errno = 14045
+	ERROR_SXS_XML_E_MULTIPLE_COLONS                                           syscall.Errno = 14046
+	ERROR_SXS_XML_E_INVALID_DECIMAL                                           syscall.Errno = 14047
+	ERROR_SXS_XML_E_INVALID_HEXIDECIMAL                                       syscall.Errno = 14048
+	ERROR_SXS_XML_E_INVALID_UNICODE                                           syscall.Errno = 14049
+	ERROR_SXS_XML_E_WHITESPACEORQUESTIONMARK                                  syscall.Errno = 14050
+	ERROR_SXS_XML_E_UNEXPECTEDENDTAG                                          syscall.Errno = 14051
+	ERROR_SXS_XML_E_UNCLOSEDTAG                                               syscall.Errno = 14052
+	ERROR_SXS_XML_E_DUPLICATEATTRIBUTE                                        syscall.Errno = 14053
+	ERROR_SXS_XML_E_MULTIPLEROOTS                                             syscall.Errno = 14054
+	ERROR_SXS_XML_E_INVALIDATROOTLEVEL                                        syscall.Errno = 14055
+	ERROR_SXS_XML_E_BADXMLDECL                                                syscall.Errno = 14056
+	ERROR_SXS_XML_E_MISSINGROOT                                               syscall.Errno = 14057
+	ERROR_SXS_XML_E_UNEXPECTEDEOF                                             syscall.Errno = 14058
+	ERROR_SXS_XML_E_BADPEREFINSUBSET                                          syscall.Errno = 14059
+	ERROR_SXS_XML_E_UNCLOSEDSTARTTAG                                          syscall.Errno = 14060
+	ERROR_SXS_XML_E_UNCLOSEDENDTAG                                            syscall.Errno = 14061
+	ERROR_SXS_XML_E_UNCLOSEDSTRING                                            syscall.Errno = 14062
+	ERROR_SXS_XML_E_UNCLOSEDCOMMENT                                           syscall.Errno = 14063
+	ERROR_SXS_XML_E_UNCLOSEDDECL                                              syscall.Errno = 14064
+	ERROR_SXS_XML_E_UNCLOSEDCDATA                                             syscall.Errno = 14065
+	ERROR_SXS_XML_E_RESERVEDNAMESPACE                                         syscall.Errno = 14066
+	ERROR_SXS_XML_E_INVALIDENCODING                                           syscall.Errno = 14067
+	ERROR_SXS_XML_E_INVALIDSWITCH                                             syscall.Errno = 14068
+	ERROR_SXS_XML_E_BADXMLCASE                                                syscall.Errno = 14069
+	ERROR_SXS_XML_E_INVALID_STANDALONE                                        syscall.Errno = 14070
+	ERROR_SXS_XML_E_UNEXPECTED_STANDALONE                                     syscall.Errno = 14071
+	ERROR_SXS_XML_E_INVALID_VERSION                                           syscall.Errno = 14072
+	ERROR_SXS_XML_E_MISSINGEQUALS                                             syscall.Errno = 14073
+	ERROR_SXS_PROTECTION_RECOVERY_FAILED                                      syscall.Errno = 14074
+	ERROR_SXS_PROTECTION_PUBLIC_KEY_TOO_SHORT                                 syscall.Errno = 14075
+	ERROR_SXS_PROTECTION_CATALOG_NOT_VALID                                    syscall.Errno = 14076
+	ERROR_SXS_UNTRANSLATABLE_HRESULT                                          syscall.Errno = 14077
+	ERROR_SXS_PROTECTION_CATALOG_FILE_MISSING                                 syscall.Errno = 14078
+	ERROR_SXS_MISSING_ASSEMBLY_IDENTITY_ATTRIBUTE                             syscall.Errno = 14079
+	ERROR_SXS_INVALID_ASSEMBLY_IDENTITY_ATTRIBUTE_NAME                        syscall.Errno = 14080
+	ERROR_SXS_ASSEMBLY_MISSING                                                syscall.Errno = 14081
+	ERROR_SXS_CORRUPT_ACTIVATION_STACK                                        syscall.Errno = 14082
+	ERROR_SXS_CORRUPTION                                                      syscall.Errno = 14083
+	ERROR_SXS_EARLY_DEACTIVATION                                              syscall.Errno = 14084
+	ERROR_SXS_INVALID_DEACTIVATION                                            syscall.Errno = 14085
+	ERROR_SXS_MULTIPLE_DEACTIVATION                                           syscall.Errno = 14086
+	ERROR_SXS_PROCESS_TERMINATION_REQUESTED                                   syscall.Errno = 14087
+	ERROR_SXS_RELEASE_ACTIVATION_CONTEXT                                      syscall.Errno = 14088
+	ERROR_SXS_SYSTEM_DEFAULT_ACTIVATION_CONTEXT_EMPTY                         syscall.Errno = 14089
+	ERROR_SXS_INVALID_IDENTITY_ATTRIBUTE_VALUE                                syscall.Errno = 14090
+	ERROR_SXS_INVALID_IDENTITY_ATTRIBUTE_NAME                                 syscall.Errno = 14091
+	ERROR_SXS_IDENTITY_DUPLICATE_ATTRIBUTE                                    syscall.Errno = 14092
+	ERROR_SXS_IDENTITY_PARSE_ERROR                                            syscall.Errno = 14093
+	ERROR_MALFORMED_SUBSTITUTION_STRING                                       syscall.Errno = 14094
+	ERROR_SXS_INCORRECT_PUBLIC_KEY_TOKEN                                      syscall.Errno = 14095
+	ERROR_UNMAPPED_SUBSTITUTION_STRING                                        syscall.Errno = 14096
+	ERROR_SXS_ASSEMBLY_NOT_LOCKED                                             syscall.Errno = 14097
+	ERROR_SXS_COMPONENT_STORE_CORRUPT                                         syscall.Errno = 14098
+	ERROR_ADVANCED_INSTALLER_FAILED                                           syscall.Errno = 14099
+	ERROR_XML_ENCODING_MISMATCH                                               syscall.Errno = 14100
+	ERROR_SXS_MANIFEST_IDENTITY_SAME_BUT_CONTENTS_DIFFERENT                   syscall.Errno = 14101
+	ERROR_SXS_IDENTITIES_DIFFERENT                                            syscall.Errno = 14102
+	ERROR_SXS_ASSEMBLY_IS_NOT_A_DEPLOYMENT                                    syscall.Errno = 14103
+	ERROR_SXS_FILE_NOT_PART_OF_ASSEMBLY                                       syscall.Errno = 14104
+	ERROR_SXS_MANIFEST_TOO_BIG                                                syscall.Errno = 14105
+	ERROR_SXS_SETTING_NOT_REGISTERED                                          syscall.Errno = 14106
+	ERROR_SXS_TRANSACTION_CLOSURE_INCOMPLETE                                  syscall.Errno = 14107
+	ERROR_SMI_PRIMITIVE_INSTALLER_FAILED                                      syscall.Errno = 14108
+	ERROR_GENERIC_COMMAND_FAILED                                              syscall.Errno = 14109
+	ERROR_SXS_FILE_HASH_MISSING                                               syscall.Errno = 14110
+	ERROR_EVT_INVALID_CHANNEL_PATH                                            syscall.Errno = 15000
+	ERROR_EVT_INVALID_QUERY                                                   syscall.Errno = 15001
+	ERROR_EVT_PUBLISHER_METADATA_NOT_FOUND                                    syscall.Errno = 15002
+	ERROR_EVT_EVENT_TEMPLATE_NOT_FOUND                                        syscall.Errno = 15003
+	ERROR_EVT_INVALID_PUBLISHER_NAME                                          syscall.Errno = 15004
+	ERROR_EVT_INVALID_EVENT_DATA                                              syscall.Errno = 15005
+	ERROR_EVT_CHANNEL_NOT_FOUND                                               syscall.Errno = 15007
+	ERROR_EVT_MALFORMED_XML_TEXT                                              syscall.Errno = 15008
+	ERROR_EVT_SUBSCRIPTION_TO_DIRECT_CHANNEL                                  syscall.Errno = 15009
+	ERROR_EVT_CONFIGURATION_ERROR                                             syscall.Errno = 15010
+	ERROR_EVT_QUERY_RESULT_STALE                                              syscall.Errno = 15011
+	ERROR_EVT_QUERY_RESULT_INVALID_POSITION                                   syscall.Errno = 15012
+	ERROR_EVT_NON_VALIDATING_MSXML                                            syscall.Errno = 15013
+	ERROR_EVT_FILTER_ALREADYSCOPED                                            syscall.Errno = 15014
+	ERROR_EVT_FILTER_NOTELTSET                                                syscall.Errno = 15015
+	ERROR_EVT_FILTER_INVARG                                                   syscall.Errno = 15016
+	ERROR_EVT_FILTER_INVTEST                                                  syscall.Errno = 15017
+	ERROR_EVT_FILTER_INVTYPE                                                  syscall.Errno = 15018
+	ERROR_EVT_FILTER_PARSEERR                                                 syscall.Errno = 15019
+	ERROR_EVT_FILTER_UNSUPPORTEDOP                                            syscall.Errno = 15020
+	ERROR_EVT_FILTER_UNEXPECTEDTOKEN                                          syscall.Errno = 15021
+	ERROR_EVT_INVALID_OPERATION_OVER_ENABLED_DIRECT_CHANNEL                   syscall.Errno = 15022
+	ERROR_EVT_INVALID_CHANNEL_PROPERTY_VALUE                                  syscall.Errno = 15023
+	ERROR_EVT_INVALID_PUBLISHER_PROPERTY_VALUE                                syscall.Errno = 15024
+	ERROR_EVT_CHANNEL_CANNOT_ACTIVATE                                         syscall.Errno = 15025
+	ERROR_EVT_FILTER_TOO_COMPLEX                                              syscall.Errno = 15026
+	ERROR_EVT_MESSAGE_NOT_FOUND                                               syscall.Errno = 15027
+	ERROR_EVT_MESSAGE_ID_NOT_FOUND                                            syscall.Errno = 15028
+	ERROR_EVT_UNRESOLVED_VALUE_INSERT                                         syscall.Errno = 15029
+	ERROR_EVT_UNRESOLVED_PARAMETER_INSERT                                     syscall.Errno = 15030
+	ERROR_EVT_MAX_INSERTS_REACHED                                             syscall.Errno = 15031
+	ERROR_EVT_EVENT_DEFINITION_NOT_FOUND                                      syscall.Errno = 15032
+	ERROR_EVT_MESSAGE_LOCALE_NOT_FOUND                                        syscall.Errno = 15033
+	ERROR_EVT_VERSION_TOO_OLD                                                 syscall.Errno = 15034
+	ERROR_EVT_VERSION_TOO_NEW                                                 syscall.Errno = 15035
+	ERROR_EVT_CANNOT_OPEN_CHANNEL_OF_QUERY                                    syscall.Errno = 15036
+	ERROR_EVT_PUBLISHER_DISABLED                                              syscall.Errno = 15037
+	ERROR_EVT_FILTER_OUT_OF_RANGE                                             syscall.Errno = 15038
+	ERROR_EC_SUBSCRIPTION_CANNOT_ACTIVATE                                     syscall.Errno = 15080
+	ERROR_EC_LOG_DISABLED                                                     syscall.Errno = 15081
+	ERROR_EC_CIRCULAR_FORWARDING                                              syscall.Errno = 15082
+	ERROR_EC_CREDSTORE_FULL                                                   syscall.Errno = 15083
+	ERROR_EC_CRED_NOT_FOUND                                                   syscall.Errno = 15084
+	ERROR_EC_NO_ACTIVE_CHANNEL                                                syscall.Errno = 15085
+	ERROR_MUI_FILE_NOT_FOUND                                                  syscall.Errno = 15100
+	ERROR_MUI_INVALID_FILE                                                    syscall.Errno = 15101
+	ERROR_MUI_INVALID_RC_CONFIG                                               syscall.Errno = 15102
+	ERROR_MUI_INVALID_LOCALE_NAME                                             syscall.Errno = 15103
+	ERROR_MUI_INVALID_ULTIMATEFALLBACK_NAME                                   syscall.Errno = 15104
+	ERROR_MUI_FILE_NOT_LOADED                                                 syscall.Errno = 15105
+	ERROR_RESOURCE_ENUM_USER_STOP                                             syscall.Errno = 15106
+	ERROR_MUI_INTLSETTINGS_UILANG_NOT_INSTALLED                               syscall.Errno = 15107
+	ERROR_MUI_INTLSETTINGS_INVALID_LOCALE_NAME                                syscall.Errno = 15108
+	ERROR_MRM_RUNTIME_NO_DEFAULT_OR_NEUTRAL_RESOURCE                          syscall.Errno = 15110
+	ERROR_MRM_INVALID_PRICONFIG                                               syscall.Errno = 15111
+	ERROR_MRM_INVALID_FILE_TYPE                                               syscall.Errno = 15112
+	ERROR_MRM_UNKNOWN_QUALIFIER                                               syscall.Errno = 15113
+	ERROR_MRM_INVALID_QUALIFIER_VALUE                                         syscall.Errno = 15114
+	ERROR_MRM_NO_CANDIDATE                                                    syscall.Errno = 15115
+	ERROR_MRM_NO_MATCH_OR_DEFAULT_CANDIDATE                                   syscall.Errno = 15116
+	ERROR_MRM_RESOURCE_TYPE_MISMATCH                                          syscall.Errno = 15117
+	ERROR_MRM_DUPLICATE_MAP_NAME                                              syscall.Errno = 15118
+	ERROR_MRM_DUPLICATE_ENTRY                                                 syscall.Errno = 15119
+	ERROR_MRM_INVALID_RESOURCE_IDENTIFIER                                     syscall.Errno = 15120
+	ERROR_MRM_FILEPATH_TOO_LONG                                               syscall.Errno = 15121
+	ERROR_MRM_UNSUPPORTED_DIRECTORY_TYPE                                      syscall.Errno = 15122
+	ERROR_MRM_INVALID_PRI_FILE                                                syscall.Errno = 15126
+	ERROR_MRM_NAMED_RESOURCE_NOT_FOUND                                        syscall.Errno = 15127
+	ERROR_MRM_MAP_NOT_FOUND                                                   syscall.Errno = 15135
+	ERROR_MRM_UNSUPPORTED_PROFILE_TYPE                                        syscall.Errno = 15136
+	ERROR_MRM_INVALID_QUALIFIER_OPERATOR                                      syscall.Errno = 15137
+	ERROR_MRM_INDETERMINATE_QUALIFIER_VALUE                                   syscall.Errno = 15138
+	ERROR_MRM_AUTOMERGE_ENABLED                                               syscall.Errno = 15139
+	ERROR_MRM_TOO_MANY_RESOURCES                                              syscall.Errno = 15140
+	ERROR_MRM_UNSUPPORTED_FILE_TYPE_FOR_MERGE                                 syscall.Errno = 15141
+	ERROR_MRM_UNSUPPORTED_FILE_TYPE_FOR_LOAD_UNLOAD_PRI_FILE                  syscall.Errno = 15142
+	ERROR_MRM_NO_CURRENT_VIEW_ON_THREAD                                       syscall.Errno = 15143
+	ERROR_DIFFERENT_PROFILE_RESOURCE_MANAGER_EXIST                            syscall.Errno = 15144
+	ERROR_OPERATION_NOT_ALLOWED_FROM_SYSTEM_COMPONENT                         syscall.Errno = 15145
+	ERROR_MRM_DIRECT_REF_TO_NON_DEFAULT_RESOURCE                              syscall.Errno = 15146
+	ERROR_MRM_GENERATION_COUNT_MISMATCH                                       syscall.Errno = 15147
+	ERROR_PRI_MERGE_VERSION_MISMATCH                                          syscall.Errno = 15148
+	ERROR_PRI_MERGE_MISSING_SCHEMA                                            syscall.Errno = 15149
+	ERROR_PRI_MERGE_LOAD_FILE_FAILED                                          syscall.Errno = 15150
+	ERROR_PRI_MERGE_ADD_FILE_FAILED                                           syscall.Errno = 15151
+	ERROR_PRI_MERGE_WRITE_FILE_FAILED                                         syscall.Errno = 15152
+	ERROR_PRI_MERGE_MULTIPLE_PACKAGE_FAMILIES_NOT_ALLOWED                     syscall.Errno = 15153
+	ERROR_PRI_MERGE_MULTIPLE_MAIN_PACKAGES_NOT_ALLOWED                        syscall.Errno = 15154
+	ERROR_PRI_MERGE_BUNDLE_PACKAGES_NOT_ALLOWED                               syscall.Errno = 15155
+	ERROR_PRI_MERGE_MAIN_PACKAGE_REQUIRED                                     syscall.Errno = 15156
+	ERROR_PRI_MERGE_RESOURCE_PACKAGE_REQUIRED                                 syscall.Errno = 15157
+	ERROR_PRI_MERGE_INVALID_FILE_NAME                                         syscall.Errno = 15158
+	ERROR_MRM_PACKAGE_NOT_FOUND                                               syscall.Errno = 15159
+	ERROR_MCA_INVALID_CAPABILITIES_STRING                                     syscall.Errno = 15200
+	ERROR_MCA_INVALID_VCP_VERSION                                             syscall.Errno = 15201
+	ERROR_MCA_MONITOR_VIOLATES_MCCS_SPECIFICATION                             syscall.Errno = 15202
+	ERROR_MCA_MCCS_VERSION_MISMATCH                                           syscall.Errno = 15203
+	ERROR_MCA_UNSUPPORTED_MCCS_VERSION                                        syscall.Errno = 15204
+	ERROR_MCA_INTERNAL_ERROR                                                  syscall.Errno = 15205
+	ERROR_MCA_INVALID_TECHNOLOGY_TYPE_RETURNED                                syscall.Errno = 15206
+	ERROR_MCA_UNSUPPORTED_COLOR_TEMPERATURE                                   syscall.Errno = 15207
+	ERROR_AMBIGUOUS_SYSTEM_DEVICE                                             syscall.Errno = 15250
+	ERROR_SYSTEM_DEVICE_NOT_FOUND                                             syscall.Errno = 15299
+	ERROR_HASH_NOT_SUPPORTED                                                  syscall.Errno = 15300
+	ERROR_HASH_NOT_PRESENT                                                    syscall.Errno = 15301
+	ERROR_SECONDARY_IC_PROVIDER_NOT_REGISTERED                                syscall.Errno = 15321
+	ERROR_GPIO_CLIENT_INFORMATION_INVALID                                     syscall.Errno = 15322
+	ERROR_GPIO_VERSION_NOT_SUPPORTED                                          syscall.Errno = 15323
+	ERROR_GPIO_INVALID_REGISTRATION_PACKET                                    syscall.Errno = 15324
+	ERROR_GPIO_OPERATION_DENIED                                               syscall.Errno = 15325
+	ERROR_GPIO_INCOMPATIBLE_CONNECT_MODE                                      syscall.Errno = 15326
+	ERROR_GPIO_INTERRUPT_ALREADY_UNMASKED                                     syscall.Errno = 15327
+	ERROR_CANNOT_SWITCH_RUNLEVEL                                              syscall.Errno = 15400
+	ERROR_INVALID_RUNLEVEL_SETTING                                            syscall.Errno = 15401
+	ERROR_RUNLEVEL_SWITCH_TIMEOUT                                             syscall.Errno = 15402
+	ERROR_RUNLEVEL_SWITCH_AGENT_TIMEOUT                                       syscall.Errno = 15403
+	ERROR_RUNLEVEL_SWITCH_IN_PROGRESS                                         syscall.Errno = 15404
+	ERROR_SERVICES_FAILED_AUTOSTART                                           syscall.Errno = 15405
+	ERROR_COM_TASK_STOP_PENDING                                               syscall.Errno = 15501
+	ERROR_INSTALL_OPEN_PACKAGE_FAILED                                         syscall.Errno = 15600
+	ERROR_INSTALL_PACKAGE_NOT_FOUND                                           syscall.Errno = 15601
+	ERROR_INSTALL_INVALID_PACKAGE                                             syscall.Errno = 15602
+	ERROR_INSTALL_RESOLVE_DEPENDENCY_FAILED                                   syscall.Errno = 15603
+	ERROR_INSTALL_OUT_OF_DISK_SPACE                                           syscall.Errno = 15604
+	ERROR_INSTALL_NETWORK_FAILURE                                             syscall.Errno = 15605
+	ERROR_INSTALL_REGISTRATION_FAILURE                                        syscall.Errno = 15606
+	ERROR_INSTALL_DEREGISTRATION_FAILURE                                      syscall.Errno = 15607
+	ERROR_INSTALL_CANCEL                                                      syscall.Errno = 15608
+	ERROR_INSTALL_FAILED                                                      syscall.Errno = 15609
+	ERROR_REMOVE_FAILED                                                       syscall.Errno = 15610
+	ERROR_PACKAGE_ALREADY_EXISTS                                              syscall.Errno = 15611
+	ERROR_NEEDS_REMEDIATION                                                   syscall.Errno = 15612
+	ERROR_INSTALL_PREREQUISITE_FAILED                                         syscall.Errno = 15613
+	ERROR_PACKAGE_REPOSITORY_CORRUPTED                                        syscall.Errno = 15614
+	ERROR_INSTALL_POLICY_FAILURE                                              syscall.Errno = 15615
+	ERROR_PACKAGE_UPDATING                                                    syscall.Errno = 15616
+	ERROR_DEPLOYMENT_BLOCKED_BY_POLICY                                        syscall.Errno = 15617
+	ERROR_PACKAGES_IN_USE                                                     syscall.Errno = 15618
+	ERROR_RECOVERY_FILE_CORRUPT                                               syscall.Errno = 15619
+	ERROR_INVALID_STAGED_SIGNATURE                                            syscall.Errno = 15620
+	ERROR_DELETING_EXISTING_APPLICATIONDATA_STORE_FAILED                      syscall.Errno = 15621
+	ERROR_INSTALL_PACKAGE_DOWNGRADE                                           syscall.Errno = 15622
+	ERROR_SYSTEM_NEEDS_REMEDIATION                                            syscall.Errno = 15623
+	ERROR_APPX_INTEGRITY_FAILURE_CLR_NGEN                                     syscall.Errno = 15624
+	ERROR_RESILIENCY_FILE_CORRUPT                                             syscall.Errno = 15625
+	ERROR_INSTALL_FIREWALL_SERVICE_NOT_RUNNING                                syscall.Errno = 15626
+	ERROR_PACKAGE_MOVE_FAILED                                                 syscall.Errno = 15627
+	ERROR_INSTALL_VOLUME_NOT_EMPTY                                            syscall.Errno = 15628
+	ERROR_INSTALL_VOLUME_OFFLINE                                              syscall.Errno = 15629
+	ERROR_INSTALL_VOLUME_CORRUPT                                              syscall.Errno = 15630
+	ERROR_NEEDS_REGISTRATION                                                  syscall.Errno = 15631
+	ERROR_INSTALL_WRONG_PROCESSOR_ARCHITECTURE                                syscall.Errno = 15632
+	ERROR_DEV_SIDELOAD_LIMIT_EXCEEDED                                         syscall.Errno = 15633
+	ERROR_INSTALL_OPTIONAL_PACKAGE_REQUIRES_MAIN_PACKAGE                      syscall.Errno = 15634
+	ERROR_PACKAGE_NOT_SUPPORTED_ON_FILESYSTEM                                 syscall.Errno = 15635
+	ERROR_PACKAGE_MOVE_BLOCKED_BY_STREAMING                                   syscall.Errno = 15636
+	ERROR_INSTALL_OPTIONAL_PACKAGE_APPLICATIONID_NOT_UNIQUE                   syscall.Errno = 15637
+	ERROR_PACKAGE_STAGING_ONHOLD                                              syscall.Errno = 15638
+	ERROR_INSTALL_INVALID_RELATED_SET_UPDATE                                  syscall.Errno = 15639
+	ERROR_INSTALL_OPTIONAL_PACKAGE_REQUIRES_MAIN_PACKAGE_FULLTRUST_CAPABILITY syscall.Errno = 15640
+	ERROR_DEPLOYMENT_BLOCKED_BY_USER_LOG_OFF                                  syscall.Errno = 15641
+	ERROR_PROVISION_OPTIONAL_PACKAGE_REQUIRES_MAIN_PACKAGE_PROVISIONED        syscall.Errno = 15642
+	ERROR_PACKAGES_REPUTATION_CHECK_FAILED                                    syscall.Errno = 15643
+	ERROR_PACKAGES_REPUTATION_CHECK_TIMEDOUT                                  syscall.Errno = 15644
+	ERROR_DEPLOYMENT_OPTION_NOT_SUPPORTED                                     syscall.Errno = 15645
+	ERROR_APPINSTALLER_ACTIVATION_BLOCKED                                     syscall.Errno = 15646
+	ERROR_REGISTRATION_FROM_REMOTE_DRIVE_NOT_SUPPORTED                        syscall.Errno = 15647
+	APPMODEL_ERROR_NO_PACKAGE                                                 syscall.Errno = 15700
+	APPMODEL_ERROR_PACKAGE_RUNTIME_CORRUPT                                    syscall.Errno = 15701
+	APPMODEL_ERROR_PACKAGE_IDENTITY_CORRUPT                                   syscall.Errno = 15702
+	APPMODEL_ERROR_NO_APPLICATION                                             syscall.Errno = 15703
+	APPMODEL_ERROR_DYNAMIC_PROPERTY_READ_FAILED                               syscall.Errno = 15704
+	APPMODEL_ERROR_DYNAMIC_PROPERTY_INVALID                                   syscall.Errno = 15705
+	APPMODEL_ERROR_PACKAGE_NOT_AVAILABLE                                      syscall.Errno = 15706
+	ERROR_STATE_LOAD_STORE_FAILED                                             syscall.Errno = 15800
+	ERROR_STATE_GET_VERSION_FAILED                                            syscall.Errno = 15801
+	ERROR_STATE_SET_VERSION_FAILED                                            syscall.Errno = 15802
+	ERROR_STATE_STRUCTURED_RESET_FAILED                                       syscall.Errno = 15803
+	ERROR_STATE_OPEN_CONTAINER_FAILED                                         syscall.Errno = 15804
+	ERROR_STATE_CREATE_CONTAINER_FAILED                                       syscall.Errno = 15805
+	ERROR_STATE_DELETE_CONTAINER_FAILED                                       syscall.Errno = 15806
+	ERROR_STATE_READ_SETTING_FAILED                                           syscall.Errno = 15807
+	ERROR_STATE_WRITE_SETTING_FAILED                                          syscall.Errno = 15808
+	ERROR_STATE_DELETE_SETTING_FAILED                                         syscall.Errno = 15809
+	ERROR_STATE_QUERY_SETTING_FAILED                                          syscall.Errno = 15810
+	ERROR_STATE_READ_COMPOSITE_SETTING_FAILED                                 syscall.Errno = 15811
+	ERROR_STATE_WRITE_COMPOSITE_SETTING_FAILED                                syscall.Errno = 15812
+	ERROR_STATE_ENUMERATE_CONTAINER_FAILED                                    syscall.Errno = 15813
+	ERROR_STATE_ENUMERATE_SETTINGS_FAILED                                     syscall.Errno = 15814
+	ERROR_STATE_COMPOSITE_SETTING_VALUE_SIZE_LIMIT_EXCEEDED                   syscall.Errno = 15815
+	ERROR_STATE_SETTING_VALUE_SIZE_LIMIT_EXCEEDED                             syscall.Errno = 15816
+	ERROR_STATE_SETTING_NAME_SIZE_LIMIT_EXCEEDED                              syscall.Errno = 15817
+	ERROR_STATE_CONTAINER_NAME_SIZE_LIMIT_EXCEEDED                            syscall.Errno = 15818
+	ERROR_API_UNAVAILABLE                                                     syscall.Errno = 15841
+	STORE_ERROR_UNLICENSED                                                    syscall.Errno = 15861
+	STORE_ERROR_UNLICENSED_USER                                               syscall.Errno = 15862
+	STORE_ERROR_PENDING_COM_TRANSACTION                                       syscall.Errno = 15863
+	STORE_ERROR_LICENSE_REVOKED                                               syscall.Errno = 15864
+	SEVERITY_SUCCESS                                                          syscall.Errno = 0
+	SEVERITY_ERROR                                                            syscall.Errno = 1
+	FACILITY_NT_BIT                                                                         = 0x10000000
+	E_NOT_SET                                                                               = ERROR_NOT_FOUND
+	E_NOT_VALID_STATE                                                                       = ERROR_INVALID_STATE
+	E_NOT_SUFFICIENT_BUFFER                                                                 = ERROR_INSUFFICIENT_BUFFER
+	E_TIME_CRITICAL_THREAD                                                                  = ERROR_TIME_CRITICAL_THREAD
+	NOERROR                                                                   syscall.Errno = 0
+	E_UNEXPECTED                                                              Handle        = 0x8000FFFF
+	E_NOTIMPL                                                                 Handle        = 0x80004001
+	E_OUTOFMEMORY                                                             Handle        = 0x8007000E
+	E_INVALIDARG                                                              Handle        = 0x80070057
+	E_NOINTERFACE                                                             Handle        = 0x80004002
+	E_POINTER                                                                 Handle        = 0x80004003
+	E_HANDLE                                                                  Handle        = 0x80070006
+	E_ABORT                                                                   Handle        = 0x80004004
+	E_FAIL                                                                    Handle        = 0x80004005
+	E_ACCESSDENIED                                                            Handle        = 0x80070005
+	E_PENDING                                                                 Handle        = 0x8000000A
+	E_BOUNDS                                                                  Handle        = 0x8000000B
+	E_CHANGED_STATE                                                           Handle        = 0x8000000C
+	E_ILLEGAL_STATE_CHANGE                                                    Handle        = 0x8000000D
+	E_ILLEGAL_METHOD_CALL                                                     Handle        = 0x8000000E
+	RO_E_METADATA_NAME_NOT_FOUND                                              Handle        = 0x8000000F
+	RO_E_METADATA_NAME_IS_NAMESPACE                                           Handle        = 0x80000010
+	RO_E_METADATA_INVALID_TYPE_FORMAT                                         Handle        = 0x80000011
+	RO_E_INVALID_METADATA_FILE                                                Handle        = 0x80000012
+	RO_E_CLOSED                                                               Handle        = 0x80000013
+	RO_E_EXCLUSIVE_WRITE                                                      Handle        = 0x80000014
+	RO_E_CHANGE_NOTIFICATION_IN_PROGRESS                                      Handle        = 0x80000015
+	RO_E_ERROR_STRING_NOT_FOUND                                               Handle        = 0x80000016
+	E_STRING_NOT_NULL_TERMINATED                                              Handle        = 0x80000017
+	E_ILLEGAL_DELEGATE_ASSIGNMENT                                             Handle        = 0x80000018
+	E_ASYNC_OPERATION_NOT_STARTED                                             Handle        = 0x80000019
+	E_APPLICATION_EXITING                                                     Handle        = 0x8000001A
+	E_APPLICATION_VIEW_EXITING                                                Handle        = 0x8000001B
+	RO_E_MUST_BE_AGILE                                                        Handle        = 0x8000001C
+	RO_E_UNSUPPORTED_FROM_MTA                                                 Handle        = 0x8000001D
+	RO_E_COMMITTED                                                            Handle        = 0x8000001E
+	RO_E_BLOCKED_CROSS_ASTA_CALL                                              Handle        = 0x8000001F
+	RO_E_CANNOT_ACTIVATE_FULL_TRUST_SERVER                                    Handle        = 0x80000020
+	RO_E_CANNOT_ACTIVATE_UNIVERSAL_APPLICATION_SERVER                         Handle        = 0x80000021
+	CO_E_INIT_TLS                                                             Handle        = 0x80004006
+	CO_E_INIT_SHARED_ALLOCATOR                                                Handle        = 0x80004007
+	CO_E_INIT_MEMORY_ALLOCATOR                                                Handle        = 0x80004008
+	CO_E_INIT_CLASS_CACHE                                                     Handle        = 0x80004009
+	CO_E_INIT_RPC_CHANNEL                                                     Handle        = 0x8000400A
+	CO_E_INIT_TLS_SET_CHANNEL_CONTROL                                         Handle        = 0x8000400B
+	CO_E_INIT_TLS_CHANNEL_CONTROL                                             Handle        = 0x8000400C
+	CO_E_INIT_UNACCEPTED_USER_ALLOCATOR                                       Handle        = 0x8000400D
+	CO_E_INIT_SCM_MUTEX_EXISTS                                                Handle        = 0x8000400E
+	CO_E_INIT_SCM_FILE_MAPPING_EXISTS                                         Handle        = 0x8000400F
+	CO_E_INIT_SCM_MAP_VIEW_OF_FILE                                            Handle        = 0x80004010
+	CO_E_INIT_SCM_EXEC_FAILURE                                                Handle        = 0x80004011
+	CO_E_INIT_ONLY_SINGLE_THREADED                                            Handle        = 0x80004012
+	CO_E_CANT_REMOTE                                                          Handle        = 0x80004013
+	CO_E_BAD_SERVER_NAME                                                      Handle        = 0x80004014
+	CO_E_WRONG_SERVER_IDENTITY                                                Handle        = 0x80004015
+	CO_E_OLE1DDE_DISABLED                                                     Handle        = 0x80004016
+	CO_E_RUNAS_SYNTAX                                                         Handle        = 0x80004017
+	CO_E_CREATEPROCESS_FAILURE                                                Handle        = 0x80004018
+	CO_E_RUNAS_CREATEPROCESS_FAILURE                                          Handle        = 0x80004019
+	CO_E_RUNAS_LOGON_FAILURE                                                  Handle        = 0x8000401A
+	CO_E_LAUNCH_PERMSSION_DENIED                                              Handle        = 0x8000401B
+	CO_E_START_SERVICE_FAILURE                                                Handle        = 0x8000401C
+	CO_E_REMOTE_COMMUNICATION_FAILURE                                         Handle        = 0x8000401D
+	CO_E_SERVER_START_TIMEOUT                                                 Handle        = 0x8000401E
+	CO_E_CLSREG_INCONSISTENT                                                  Handle        = 0x8000401F
+	CO_E_IIDREG_INCONSISTENT                                                  Handle        = 0x80004020
+	CO_E_NOT_SUPPORTED                                                        Handle        = 0x80004021
+	CO_E_RELOAD_DLL                                                           Handle        = 0x80004022
+	CO_E_MSI_ERROR                                                            Handle        = 0x80004023
+	CO_E_ATTEMPT_TO_CREATE_OUTSIDE_CLIENT_CONTEXT                             Handle        = 0x80004024
+	CO_E_SERVER_PAUSED                                                        Handle        = 0x80004025
+	CO_E_SERVER_NOT_PAUSED                                                    Handle        = 0x80004026
+	CO_E_CLASS_DISABLED                                                       Handle        = 0x80004027
+	CO_E_CLRNOTAVAILABLE                                                      Handle        = 0x80004028
+	CO_E_ASYNC_WORK_REJECTED                                                  Handle        = 0x80004029
+	CO_E_SERVER_INIT_TIMEOUT                                                  Handle        = 0x8000402A
+	CO_E_NO_SECCTX_IN_ACTIVATE                                                Handle        = 0x8000402B
+	CO_E_TRACKER_CONFIG                                                       Handle        = 0x80004030
+	CO_E_THREADPOOL_CONFIG                                                    Handle        = 0x80004031
+	CO_E_SXS_CONFIG                                                           Handle        = 0x80004032
+	CO_E_MALFORMED_SPN                                                        Handle        = 0x80004033
+	CO_E_UNREVOKED_REGISTRATION_ON_APARTMENT_SHUTDOWN                         Handle        = 0x80004034
+	CO_E_PREMATURE_STUB_RUNDOWN                                               Handle        = 0x80004035
+	S_OK                                                                      Handle        = 0
+	S_FALSE                                                                   Handle        = 1
+	OLE_E_FIRST                                                               Handle        = 0x80040000
+	OLE_E_LAST                                                                Handle        = 0x800400FF
+	OLE_S_FIRST                                                               Handle        = 0x00040000
+	OLE_S_LAST                                                                Handle        = 0x000400FF
+	OLE_E_OLEVERB                                                             Handle        = 0x80040000
+	OLE_E_ADVF                                                                Handle        = 0x80040001
+	OLE_E_ENUM_NOMORE                                                         Handle        = 0x80040002
+	OLE_E_ADVISENOTSUPPORTED                                                  Handle        = 0x80040003
+	OLE_E_NOCONNECTION                                                        Handle        = 0x80040004
+	OLE_E_NOTRUNNING                                                          Handle        = 0x80040005
+	OLE_E_NOCACHE                                                             Handle        = 0x80040006
+	OLE_E_BLANK                                                               Handle        = 0x80040007
+	OLE_E_CLASSDIFF                                                           Handle        = 0x80040008
+	OLE_E_CANT_GETMONIKER                                                     Handle        = 0x80040009
+	OLE_E_CANT_BINDTOSOURCE                                                   Handle        = 0x8004000A
+	OLE_E_STATIC                                                              Handle        = 0x8004000B
+	OLE_E_PROMPTSAVECANCELLED                                                 Handle        = 0x8004000C
+	OLE_E_INVALIDRECT                                                         Handle        = 0x8004000D
+	OLE_E_WRONGCOMPOBJ                                                        Handle        = 0x8004000E
+	OLE_E_INVALIDHWND                                                         Handle        = 0x8004000F
+	OLE_E_NOT_INPLACEACTIVE                                                   Handle        = 0x80040010
+	OLE_E_CANTCONVERT                                                         Handle        = 0x80040011
+	OLE_E_NOSTORAGE                                                           Handle        = 0x80040012
+	DV_E_FORMATETC                                                            Handle        = 0x80040064
+	DV_E_DVTARGETDEVICE                                                       Handle        = 0x80040065
+	DV_E_STGMEDIUM                                                            Handle        = 0x80040066
+	DV_E_STATDATA                                                             Handle        = 0x80040067
+	DV_E_LINDEX                                                               Handle        = 0x80040068
+	DV_E_TYMED                                                                Handle        = 0x80040069
+	DV_E_CLIPFORMAT                                                           Handle        = 0x8004006A
+	DV_E_DVASPECT                                                             Handle        = 0x8004006B
+	DV_E_DVTARGETDEVICE_SIZE                                                  Handle        = 0x8004006C
+	DV_E_NOIVIEWOBJECT                                                        Handle        = 0x8004006D
+	DRAGDROP_E_FIRST                                                          syscall.Errno = 0x80040100
+	DRAGDROP_E_LAST                                                           syscall.Errno = 0x8004010F
+	DRAGDROP_S_FIRST                                                          syscall.Errno = 0x00040100
+	DRAGDROP_S_LAST                                                           syscall.Errno = 0x0004010F
+	DRAGDROP_E_NOTREGISTERED                                                  Handle        = 0x80040100
+	DRAGDROP_E_ALREADYREGISTERED                                              Handle        = 0x80040101
+	DRAGDROP_E_INVALIDHWND                                                    Handle        = 0x80040102
+	DRAGDROP_E_CONCURRENT_DRAG_ATTEMPTED                                      Handle        = 0x80040103
+	CLASSFACTORY_E_FIRST                                                      syscall.Errno = 0x80040110
+	CLASSFACTORY_E_LAST                                                       syscall.Errno = 0x8004011F
+	CLASSFACTORY_S_FIRST                                                      syscall.Errno = 0x00040110
+	CLASSFACTORY_S_LAST                                                       syscall.Errno = 0x0004011F
+	CLASS_E_NOAGGREGATION                                                     Handle        = 0x80040110
+	CLASS_E_CLASSNOTAVAILABLE                                                 Handle        = 0x80040111
+	CLASS_E_NOTLICENSED                                                       Handle        = 0x80040112
+	MARSHAL_E_FIRST                                                           syscall.Errno = 0x80040120
+	MARSHAL_E_LAST                                                            syscall.Errno = 0x8004012F
+	MARSHAL_S_FIRST                                                           syscall.Errno = 0x00040120
+	MARSHAL_S_LAST                                                            syscall.Errno = 0x0004012F
+	DATA_E_FIRST                                                              syscall.Errno = 0x80040130
+	DATA_E_LAST                                                               syscall.Errno = 0x8004013F
+	DATA_S_FIRST                                                              syscall.Errno = 0x00040130
+	DATA_S_LAST                                                               syscall.Errno = 0x0004013F
+	VIEW_E_FIRST                                                              syscall.Errno = 0x80040140
+	VIEW_E_LAST                                                               syscall.Errno = 0x8004014F
+	VIEW_S_FIRST                                                              syscall.Errno = 0x00040140
+	VIEW_S_LAST                                                               syscall.Errno = 0x0004014F
+	VIEW_E_DRAW                                                               Handle        = 0x80040140
+	REGDB_E_FIRST                                                             syscall.Errno = 0x80040150
+	REGDB_E_LAST                                                              syscall.Errno = 0x8004015F
+	REGDB_S_FIRST                                                             syscall.Errno = 0x00040150
+	REGDB_S_LAST                                                              syscall.Errno = 0x0004015F
+	REGDB_E_READREGDB                                                         Handle        = 0x80040150
+	REGDB_E_WRITEREGDB                                                        Handle        = 0x80040151
+	REGDB_E_KEYMISSING                                                        Handle        = 0x80040152
+	REGDB_E_INVALIDVALUE                                                      Handle        = 0x80040153
+	REGDB_E_CLASSNOTREG                                                       Handle        = 0x80040154
+	REGDB_E_IIDNOTREG                                                         Handle        = 0x80040155
+	REGDB_E_BADTHREADINGMODEL                                                 Handle        = 0x80040156
+	REGDB_E_PACKAGEPOLICYVIOLATION                                            Handle        = 0x80040157
+	CAT_E_FIRST                                                               syscall.Errno = 0x80040160
+	CAT_E_LAST                                                                syscall.Errno = 0x80040161
+	CAT_E_CATIDNOEXIST                                                        Handle        = 0x80040160
+	CAT_E_NODESCRIPTION                                                       Handle        = 0x80040161
+	CS_E_FIRST                                                                syscall.Errno = 0x80040164
+	CS_E_LAST                                                                 syscall.Errno = 0x8004016F
+	CS_E_PACKAGE_NOTFOUND                                                     Handle        = 0x80040164
+	CS_E_NOT_DELETABLE                                                        Handle        = 0x80040165
+	CS_E_CLASS_NOTFOUND                                                       Handle        = 0x80040166
+	CS_E_INVALID_VERSION                                                      Handle        = 0x80040167
+	CS_E_NO_CLASSSTORE                                                        Handle        = 0x80040168
+	CS_E_OBJECT_NOTFOUND                                                      Handle        = 0x80040169
+	CS_E_OBJECT_ALREADY_EXISTS                                                Handle        = 0x8004016A
+	CS_E_INVALID_PATH                                                         Handle        = 0x8004016B
+	CS_E_NETWORK_ERROR                                                        Handle        = 0x8004016C
+	CS_E_ADMIN_LIMIT_EXCEEDED                                                 Handle        = 0x8004016D
+	CS_E_SCHEMA_MISMATCH                                                      Handle        = 0x8004016E
+	CS_E_INTERNAL_ERROR                                                       Handle        = 0x8004016F
+	CACHE_E_FIRST                                                             syscall.Errno = 0x80040170
+	CACHE_E_LAST                                                              syscall.Errno = 0x8004017F
+	CACHE_S_FIRST                                                             syscall.Errno = 0x00040170
+	CACHE_S_LAST                                                              syscall.Errno = 0x0004017F
+	CACHE_E_NOCACHE_UPDATED                                                   Handle        = 0x80040170
+	OLEOBJ_E_FIRST                                                            syscall.Errno = 0x80040180
+	OLEOBJ_E_LAST                                                             syscall.Errno = 0x8004018F
+	OLEOBJ_S_FIRST                                                            syscall.Errno = 0x00040180
+	OLEOBJ_S_LAST                                                             syscall.Errno = 0x0004018F
+	OLEOBJ_E_NOVERBS                                                          Handle        = 0x80040180
+	OLEOBJ_E_INVALIDVERB                                                      Handle        = 0x80040181
+	CLIENTSITE_E_FIRST                                                        syscall.Errno = 0x80040190
+	CLIENTSITE_E_LAST                                                         syscall.Errno = 0x8004019F
+	CLIENTSITE_S_FIRST                                                        syscall.Errno = 0x00040190
+	CLIENTSITE_S_LAST                                                         syscall.Errno = 0x0004019F
+	INPLACE_E_NOTUNDOABLE                                                     Handle        = 0x800401A0
+	INPLACE_E_NOTOOLSPACE                                                     Handle        = 0x800401A1
+	INPLACE_E_FIRST                                                           syscall.Errno = 0x800401A0
+	INPLACE_E_LAST                                                            syscall.Errno = 0x800401AF
+	INPLACE_S_FIRST                                                           syscall.Errno = 0x000401A0
+	INPLACE_S_LAST                                                            syscall.Errno = 0x000401AF
+	ENUM_E_FIRST                                                              syscall.Errno = 0x800401B0
+	ENUM_E_LAST                                                               syscall.Errno = 0x800401BF
+	ENUM_S_FIRST                                                              syscall.Errno = 0x000401B0
+	ENUM_S_LAST                                                               syscall.Errno = 0x000401BF
+	CONVERT10_E_FIRST                                                         syscall.Errno = 0x800401C0
+	CONVERT10_E_LAST                                                          syscall.Errno = 0x800401CF
+	CONVERT10_S_FIRST                                                         syscall.Errno = 0x000401C0
+	CONVERT10_S_LAST                                                          syscall.Errno = 0x000401CF
+	CONVERT10_E_OLESTREAM_GET                                                 Handle        = 0x800401C0
+	CONVERT10_E_OLESTREAM_PUT                                                 Handle        = 0x800401C1
+	CONVERT10_E_OLESTREAM_FMT                                                 Handle        = 0x800401C2
+	CONVERT10_E_OLESTREAM_BITMAP_TO_DIB                                       Handle        = 0x800401C3
+	CONVERT10_E_STG_FMT                                                       Handle        = 0x800401C4
+	CONVERT10_E_STG_NO_STD_STREAM                                             Handle        = 0x800401C5
+	CONVERT10_E_STG_DIB_TO_BITMAP                                             Handle        = 0x800401C6
+	CLIPBRD_E_FIRST                                                           syscall.Errno = 0x800401D0
+	CLIPBRD_E_LAST                                                            syscall.Errno = 0x800401DF
+	CLIPBRD_S_FIRST                                                           syscall.Errno = 0x000401D0
+	CLIPBRD_S_LAST                                                            syscall.Errno = 0x000401DF
+	CLIPBRD_E_CANT_OPEN                                                       Handle        = 0x800401D0
+	CLIPBRD_E_CANT_EMPTY                                                      Handle        = 0x800401D1
+	CLIPBRD_E_CANT_SET                                                        Handle        = 0x800401D2
+	CLIPBRD_E_BAD_DATA                                                        Handle        = 0x800401D3
+	CLIPBRD_E_CANT_CLOSE                                                      Handle        = 0x800401D4
+	MK_E_FIRST                                                                syscall.Errno = 0x800401E0
+	MK_E_LAST                                                                 syscall.Errno = 0x800401EF
+	MK_S_FIRST                                                                syscall.Errno = 0x000401E0
+	MK_S_LAST                                                                 syscall.Errno = 0x000401EF
+	MK_E_CONNECTMANUALLY                                                      Handle        = 0x800401E0
+	MK_E_EXCEEDEDDEADLINE                                                     Handle        = 0x800401E1
+	MK_E_NEEDGENERIC                                                          Handle        = 0x800401E2
+	MK_E_UNAVAILABLE                                                          Handle        = 0x800401E3
+	MK_E_SYNTAX                                                               Handle        = 0x800401E4
+	MK_E_NOOBJECT                                                             Handle        = 0x800401E5
+	MK_E_INVALIDEXTENSION                                                     Handle        = 0x800401E6
+	MK_E_INTERMEDIATEINTERFACENOTSUPPORTED                                    Handle        = 0x800401E7
+	MK_E_NOTBINDABLE                                                          Handle        = 0x800401E8
+	MK_E_NOTBOUND                                                             Handle        = 0x800401E9
+	MK_E_CANTOPENFILE                                                         Handle        = 0x800401EA
+	MK_E_MUSTBOTHERUSER                                                       Handle        = 0x800401EB
+	MK_E_NOINVERSE                                                            Handle        = 0x800401EC
+	MK_E_NOSTORAGE                                                            Handle        = 0x800401ED
+	MK_E_NOPREFIX                                                             Handle        = 0x800401EE
+	MK_E_ENUMERATION_FAILED                                                   Handle        = 0x800401EF
+	CO_E_FIRST                                                                syscall.Errno = 0x800401F0
+	CO_E_LAST                                                                 syscall.Errno = 0x800401FF
+	CO_S_FIRST                                                                syscall.Errno = 0x000401F0
+	CO_S_LAST                                                                 syscall.Errno = 0x000401FF
+	CO_E_NOTINITIALIZED                                                       Handle        = 0x800401F0
+	CO_E_ALREADYINITIALIZED                                                   Handle        = 0x800401F1
+	CO_E_CANTDETERMINECLASS                                                   Handle        = 0x800401F2
+	CO_E_CLASSSTRING                                                          Handle        = 0x800401F3
+	CO_E_IIDSTRING                                                            Handle        = 0x800401F4
+	CO_E_APPNOTFOUND                                                          Handle        = 0x800401F5
+	CO_E_APPSINGLEUSE                                                         Handle        = 0x800401F6
+	CO_E_ERRORINAPP                                                           Handle        = 0x800401F7
+	CO_E_DLLNOTFOUND                                                          Handle        = 0x800401F8
+	CO_E_ERRORINDLL                                                           Handle        = 0x800401F9
+	CO_E_WRONGOSFORAPP                                                        Handle        = 0x800401FA
+	CO_E_OBJNOTREG                                                            Handle        = 0x800401FB
+	CO_E_OBJISREG                                                             Handle        = 0x800401FC
+	CO_E_OBJNOTCONNECTED                                                      Handle        = 0x800401FD
+	CO_E_APPDIDNTREG                                                          Handle        = 0x800401FE
+	CO_E_RELEASED                                                             Handle        = 0x800401FF
+	EVENT_E_FIRST                                                             syscall.Errno = 0x80040200
+	EVENT_E_LAST                                                              syscall.Errno = 0x8004021F
+	EVENT_S_FIRST                                                             syscall.Errno = 0x00040200
+	EVENT_S_LAST                                                              syscall.Errno = 0x0004021F
+	EVENT_S_SOME_SUBSCRIBERS_FAILED                                           Handle        = 0x00040200
+	EVENT_E_ALL_SUBSCRIBERS_FAILED                                            Handle        = 0x80040201
+	EVENT_S_NOSUBSCRIBERS                                                     Handle        = 0x00040202
+	EVENT_E_QUERYSYNTAX                                                       Handle        = 0x80040203
+	EVENT_E_QUERYFIELD                                                        Handle        = 0x80040204
+	EVENT_E_INTERNALEXCEPTION                                                 Handle        = 0x80040205
+	EVENT_E_INTERNALERROR                                                     Handle        = 0x80040206
+	EVENT_E_INVALID_PER_USER_SID                                              Handle        = 0x80040207
+	EVENT_E_USER_EXCEPTION                                                    Handle        = 0x80040208
+	EVENT_E_TOO_MANY_METHODS                                                  Handle        = 0x80040209
+	EVENT_E_MISSING_EVENTCLASS                                                Handle        = 0x8004020A
+	EVENT_E_NOT_ALL_REMOVED                                                   Handle        = 0x8004020B
+	EVENT_E_COMPLUS_NOT_INSTALLED                                             Handle        = 0x8004020C
+	EVENT_E_CANT_MODIFY_OR_DELETE_UNCONFIGURED_OBJECT                         Handle        = 0x8004020D
+	EVENT_E_CANT_MODIFY_OR_DELETE_CONFIGURED_OBJECT                           Handle        = 0x8004020E
+	EVENT_E_INVALID_EVENT_CLASS_PARTITION                                     Handle        = 0x8004020F
+	EVENT_E_PER_USER_SID_NOT_LOGGED_ON                                        Handle        = 0x80040210
+	TPC_E_INVALID_PROPERTY                                                    Handle        = 0x80040241
+	TPC_E_NO_DEFAULT_TABLET                                                   Handle        = 0x80040212
+	TPC_E_UNKNOWN_PROPERTY                                                    Handle        = 0x8004021B
+	TPC_E_INVALID_INPUT_RECT                                                  Handle        = 0x80040219
+	TPC_E_INVALID_STROKE                                                      Handle        = 0x80040222
+	TPC_E_INITIALIZE_FAIL                                                     Handle        = 0x80040223
+	TPC_E_NOT_RELEVANT                                                        Handle        = 0x80040232
+	TPC_E_INVALID_PACKET_DESCRIPTION                                          Handle        = 0x80040233
+	TPC_E_RECOGNIZER_NOT_REGISTERED                                           Handle        = 0x80040235
+	TPC_E_INVALID_RIGHTS                                                      Handle        = 0x80040236
+	TPC_E_OUT_OF_ORDER_CALL                                                   Handle        = 0x80040237
+	TPC_E_QUEUE_FULL                                                          Handle        = 0x80040238
+	TPC_E_INVALID_CONFIGURATION                                               Handle        = 0x80040239
+	TPC_E_INVALID_DATA_FROM_RECOGNIZER                                        Handle        = 0x8004023A
+	TPC_S_TRUNCATED                                                           Handle        = 0x00040252
+	TPC_S_INTERRUPTED                                                         Handle        = 0x00040253
+	TPC_S_NO_DATA_TO_PROCESS                                                  Handle        = 0x00040254
+	XACT_E_FIRST                                                              syscall.Errno = 0x8004D000
+	XACT_E_LAST                                                               syscall.Errno = 0x8004D02B
+	XACT_S_FIRST                                                              syscall.Errno = 0x0004D000
+	XACT_S_LAST                                                               syscall.Errno = 0x0004D010
+	XACT_E_ALREADYOTHERSINGLEPHASE                                            Handle        = 0x8004D000
+	XACT_E_CANTRETAIN                                                         Handle        = 0x8004D001
+	XACT_E_COMMITFAILED                                                       Handle        = 0x8004D002
+	XACT_E_COMMITPREVENTED                                                    Handle        = 0x8004D003
+	XACT_E_HEURISTICABORT                                                     Handle        = 0x8004D004
+	XACT_E_HEURISTICCOMMIT                                                    Handle        = 0x8004D005
+	XACT_E_HEURISTICDAMAGE                                                    Handle        = 0x8004D006
+	XACT_E_HEURISTICDANGER                                                    Handle        = 0x8004D007
+	XACT_E_ISOLATIONLEVEL                                                     Handle        = 0x8004D008
+	XACT_E_NOASYNC                                                            Handle        = 0x8004D009
+	XACT_E_NOENLIST                                                           Handle        = 0x8004D00A
+	XACT_E_NOISORETAIN                                                        Handle        = 0x8004D00B
+	XACT_E_NORESOURCE                                                         Handle        = 0x8004D00C
+	XACT_E_NOTCURRENT                                                         Handle        = 0x8004D00D
+	XACT_E_NOTRANSACTION                                                      Handle        = 0x8004D00E
+	XACT_E_NOTSUPPORTED                                                       Handle        = 0x8004D00F
+	XACT_E_UNKNOWNRMGRID                                                      Handle        = 0x8004D010
+	XACT_E_WRONGSTATE                                                         Handle        = 0x8004D011
+	XACT_E_WRONGUOW                                                           Handle        = 0x8004D012
+	XACT_E_XTIONEXISTS                                                        Handle        = 0x8004D013
+	XACT_E_NOIMPORTOBJECT                                                     Handle        = 0x8004D014
+	XACT_E_INVALIDCOOKIE                                                      Handle        = 0x8004D015
+	XACT_E_INDOUBT                                                            Handle        = 0x8004D016
+	XACT_E_NOTIMEOUT                                                          Handle        = 0x8004D017
+	XACT_E_ALREADYINPROGRESS                                                  Handle        = 0x8004D018
+	XACT_E_ABORTED                                                            Handle        = 0x8004D019
+	XACT_E_LOGFULL                                                            Handle        = 0x8004D01A
+	XACT_E_TMNOTAVAILABLE                                                     Handle        = 0x8004D01B
+	XACT_E_CONNECTION_DOWN                                                    Handle        = 0x8004D01C
+	XACT_E_CONNECTION_DENIED                                                  Handle        = 0x8004D01D
+	XACT_E_REENLISTTIMEOUT                                                    Handle        = 0x8004D01E
+	XACT_E_TIP_CONNECT_FAILED                                                 Handle        = 0x8004D01F
+	XACT_E_TIP_PROTOCOL_ERROR                                                 Handle        = 0x8004D020
+	XACT_E_TIP_PULL_FAILED                                                    Handle        = 0x8004D021
+	XACT_E_DEST_TMNOTAVAILABLE                                                Handle        = 0x8004D022
+	XACT_E_TIP_DISABLED                                                       Handle        = 0x8004D023
+	XACT_E_NETWORK_TX_DISABLED                                                Handle        = 0x8004D024
+	XACT_E_PARTNER_NETWORK_TX_DISABLED                                        Handle        = 0x8004D025
+	XACT_E_XA_TX_DISABLED                                                     Handle        = 0x8004D026
+	XACT_E_UNABLE_TO_READ_DTC_CONFIG                                          Handle        = 0x8004D027
+	XACT_E_UNABLE_TO_LOAD_DTC_PROXY                                           Handle        = 0x8004D028
+	XACT_E_ABORTING                                                           Handle        = 0x8004D029
+	XACT_E_PUSH_COMM_FAILURE                                                  Handle        = 0x8004D02A
+	XACT_E_PULL_COMM_FAILURE                                                  Handle        = 0x8004D02B
+	XACT_E_LU_TX_DISABLED                                                     Handle        = 0x8004D02C
+	XACT_E_CLERKNOTFOUND                                                      Handle        = 0x8004D080
+	XACT_E_CLERKEXISTS                                                        Handle        = 0x8004D081
+	XACT_E_RECOVERYINPROGRESS                                                 Handle        = 0x8004D082
+	XACT_E_TRANSACTIONCLOSED                                                  Handle        = 0x8004D083
+	XACT_E_INVALIDLSN                                                         Handle        = 0x8004D084
+	XACT_E_REPLAYREQUEST                                                      Handle        = 0x8004D085
+	XACT_S_ASYNC                                                              Handle        = 0x0004D000
+	XACT_S_DEFECT                                                             Handle        = 0x0004D001
+	XACT_S_READONLY                                                           Handle        = 0x0004D002
+	XACT_S_SOMENORETAIN                                                       Handle        = 0x0004D003
+	XACT_S_OKINFORM                                                           Handle        = 0x0004D004
+	XACT_S_MADECHANGESCONTENT                                                 Handle        = 0x0004D005
+	XACT_S_MADECHANGESINFORM                                                  Handle        = 0x0004D006
+	XACT_S_ALLNORETAIN                                                        Handle        = 0x0004D007
+	XACT_S_ABORTING                                                           Handle        = 0x0004D008
+	XACT_S_SINGLEPHASE                                                        Handle        = 0x0004D009
+	XACT_S_LOCALLY_OK                                                         Handle        = 0x0004D00A
+	XACT_S_LASTRESOURCEMANAGER                                                Handle        = 0x0004D010
+	CONTEXT_E_FIRST                                                           syscall.Errno = 0x8004E000
+	CONTEXT_E_LAST                                                            syscall.Errno = 0x8004E02F
+	CONTEXT_S_FIRST                                                           syscall.Errno = 0x0004E000
+	CONTEXT_S_LAST                                                            syscall.Errno = 0x0004E02F
+	CONTEXT_E_ABORTED                                                         Handle        = 0x8004E002
+	CONTEXT_E_ABORTING                                                        Handle        = 0x8004E003
+	CONTEXT_E_NOCONTEXT                                                       Handle        = 0x8004E004
+	CONTEXT_E_WOULD_DEADLOCK                                                  Handle        = 0x8004E005
+	CONTEXT_E_SYNCH_TIMEOUT                                                   Handle        = 0x8004E006
+	CONTEXT_E_OLDREF                                                          Handle        = 0x8004E007
+	CONTEXT_E_ROLENOTFOUND                                                    Handle        = 0x8004E00C
+	CONTEXT_E_TMNOTAVAILABLE                                                  Handle        = 0x8004E00F
+	CO_E_ACTIVATIONFAILED                                                     Handle        = 0x8004E021
+	CO_E_ACTIVATIONFAILED_EVENTLOGGED                                         Handle        = 0x8004E022
+	CO_E_ACTIVATIONFAILED_CATALOGERROR                                        Handle        = 0x8004E023
+	CO_E_ACTIVATIONFAILED_TIMEOUT                                             Handle        = 0x8004E024
+	CO_E_INITIALIZATIONFAILED                                                 Handle        = 0x8004E025
+	CONTEXT_E_NOJIT                                                           Handle        = 0x8004E026
+	CONTEXT_E_NOTRANSACTION                                                   Handle        = 0x8004E027
+	CO_E_THREADINGMODEL_CHANGED                                               Handle        = 0x8004E028
+	CO_E_NOIISINTRINSICS                                                      Handle        = 0x8004E029
+	CO_E_NOCOOKIES                                                            Handle        = 0x8004E02A
+	CO_E_DBERROR                                                              Handle        = 0x8004E02B
+	CO_E_NOTPOOLED                                                            Handle        = 0x8004E02C
+	CO_E_NOTCONSTRUCTED                                                       Handle        = 0x8004E02D
+	CO_E_NOSYNCHRONIZATION                                                    Handle        = 0x8004E02E
+	CO_E_ISOLEVELMISMATCH                                                     Handle        = 0x8004E02F
+	CO_E_CALL_OUT_OF_TX_SCOPE_NOT_ALLOWED                                     Handle        = 0x8004E030
+	CO_E_EXIT_TRANSACTION_SCOPE_NOT_CALLED                                    Handle        = 0x8004E031
+	OLE_S_USEREG                                                              Handle        = 0x00040000
+	OLE_S_STATIC                                                              Handle        = 0x00040001
+	OLE_S_MAC_CLIPFORMAT                                                      Handle        = 0x00040002
+	DRAGDROP_S_DROP                                                           Handle        = 0x00040100
+	DRAGDROP_S_CANCEL                                                         Handle        = 0x00040101
+	DRAGDROP_S_USEDEFAULTCURSORS                                              Handle        = 0x00040102
+	DATA_S_SAMEFORMATETC                                                      Handle        = 0x00040130
+	VIEW_S_ALREADY_FROZEN                                                     Handle        = 0x00040140
+	CACHE_S_FORMATETC_NOTSUPPORTED                                            Handle        = 0x00040170
+	CACHE_S_SAMECACHE                                                         Handle        = 0x00040171
+	CACHE_S_SOMECACHES_NOTUPDATED                                             Handle        = 0x00040172
+	OLEOBJ_S_INVALIDVERB                                                      Handle        = 0x00040180
+	OLEOBJ_S_CANNOT_DOVERB_NOW                                                Handle        = 0x00040181
+	OLEOBJ_S_INVALIDHWND                                                      Handle        = 0x00040182
+	INPLACE_S_TRUNCATED                                                       Handle        = 0x000401A0
+	CONVERT10_S_NO_PRESENTATION                                               Handle        = 0x000401C0
+	MK_S_REDUCED_TO_SELF                                                      Handle        = 0x000401E2
+	MK_S_ME                                                                   Handle        = 0x000401E4
+	MK_S_HIM                                                                  Handle        = 0x000401E5
+	MK_S_US                                                                   Handle        = 0x000401E6
+	MK_S_MONIKERALREADYREGISTERED                                             Handle        = 0x000401E7
+	SCHED_S_TASK_READY                                                        Handle        = 0x00041300
+	SCHED_S_TASK_RUNNING                                                      Handle        = 0x00041301
+	SCHED_S_TASK_DISABLED                                                     Handle        = 0x00041302
+	SCHED_S_TASK_HAS_NOT_RUN                                                  Handle        = 0x00041303
+	SCHED_S_TASK_NO_MORE_RUNS                                                 Handle        = 0x00041304
+	SCHED_S_TASK_NOT_SCHEDULED                                                Handle        = 0x00041305
+	SCHED_S_TASK_TERMINATED                                                   Handle        = 0x00041306
+	SCHED_S_TASK_NO_VALID_TRIGGERS                                            Handle        = 0x00041307
+	SCHED_S_EVENT_TRIGGER                                                     Handle        = 0x00041308
+	SCHED_E_TRIGGER_NOT_FOUND                                                 Handle        = 0x80041309
+	SCHED_E_TASK_NOT_READY                                                    Handle        = 0x8004130A
+	SCHED_E_TASK_NOT_RUNNING                                                  Handle        = 0x8004130B
+	SCHED_E_SERVICE_NOT_INSTALLED                                             Handle        = 0x8004130C
+	SCHED_E_CANNOT_OPEN_TASK                                                  Handle        = 0x8004130D
+	SCHED_E_INVALID_TASK                                                      Handle        = 0x8004130E
+	SCHED_E_ACCOUNT_INFORMATION_NOT_SET                                       Handle        = 0x8004130F
+	SCHED_E_ACCOUNT_NAME_NOT_FOUND                                            Handle        = 0x80041310
+	SCHED_E_ACCOUNT_DBASE_CORRUPT                                             Handle        = 0x80041311
+	SCHED_E_NO_SECURITY_SERVICES                                              Handle        = 0x80041312
+	SCHED_E_UNKNOWN_OBJECT_VERSION                                            Handle        = 0x80041313
+	SCHED_E_UNSUPPORTED_ACCOUNT_OPTION                                        Handle        = 0x80041314
+	SCHED_E_SERVICE_NOT_RUNNING                                               Handle        = 0x80041315
+	SCHED_E_UNEXPECTEDNODE                                                    Handle        = 0x80041316
+	SCHED_E_NAMESPACE                                                         Handle        = 0x80041317
+	SCHED_E_INVALIDVALUE                                                      Handle        = 0x80041318
+	SCHED_E_MISSINGNODE                                                       Handle        = 0x80041319
+	SCHED_E_MALFORMEDXML                                                      Handle        = 0x8004131A
+	SCHED_S_SOME_TRIGGERS_FAILED                                              Handle        = 0x0004131B
+	SCHED_S_BATCH_LOGON_PROBLEM                                               Handle        = 0x0004131C
+	SCHED_E_TOO_MANY_NODES                                                    Handle        = 0x8004131D
+	SCHED_E_PAST_END_BOUNDARY                                                 Handle        = 0x8004131E
+	SCHED_E_ALREADY_RUNNING                                                   Handle        = 0x8004131F
+	SCHED_E_USER_NOT_LOGGED_ON                                                Handle        = 0x80041320
+	SCHED_E_INVALID_TASK_HASH                                                 Handle        = 0x80041321
+	SCHED_E_SERVICE_NOT_AVAILABLE                                             Handle        = 0x80041322
+	SCHED_E_SERVICE_TOO_BUSY                                                  Handle        = 0x80041323
+	SCHED_E_TASK_ATTEMPTED                                                    Handle        = 0x80041324
+	SCHED_S_TASK_QUEUED                                                       Handle        = 0x00041325
+	SCHED_E_TASK_DISABLED                                                     Handle        = 0x80041326
+	SCHED_E_TASK_NOT_V1_COMPAT                                                Handle        = 0x80041327
+	SCHED_E_START_ON_DEMAND                                                   Handle        = 0x80041328
+	SCHED_E_TASK_NOT_UBPM_COMPAT                                              Handle        = 0x80041329
+	SCHED_E_DEPRECATED_FEATURE_USED                                           Handle        = 0x80041330
+	CO_E_CLASS_CREATE_FAILED                                                  Handle        = 0x80080001
+	CO_E_SCM_ERROR                                                            Handle        = 0x80080002
+	CO_E_SCM_RPC_FAILURE                                                      Handle        = 0x80080003
+	CO_E_BAD_PATH                                                             Handle        = 0x80080004
+	CO_E_SERVER_EXEC_FAILURE                                                  Handle        = 0x80080005
+	CO_E_OBJSRV_RPC_FAILURE                                                   Handle        = 0x80080006
+	MK_E_NO_NORMALIZED                                                        Handle        = 0x80080007
+	CO_E_SERVER_STOPPING                                                      Handle        = 0x80080008
+	MEM_E_INVALID_ROOT                                                        Handle        = 0x80080009
+	MEM_E_INVALID_LINK                                                        Handle        = 0x80080010
+	MEM_E_INVALID_SIZE                                                        Handle        = 0x80080011
+	CO_S_NOTALLINTERFACES                                                     Handle        = 0x00080012
+	CO_S_MACHINENAMENOTFOUND                                                  Handle        = 0x00080013
+	CO_E_MISSING_DISPLAYNAME                                                  Handle        = 0x80080015
+	CO_E_RUNAS_VALUE_MUST_BE_AAA                                              Handle        = 0x80080016
+	CO_E_ELEVATION_DISABLED                                                   Handle        = 0x80080017
+	APPX_E_PACKAGING_INTERNAL                                                 Handle        = 0x80080200
+	APPX_E_INTERLEAVING_NOT_ALLOWED                                           Handle        = 0x80080201
+	APPX_E_RELATIONSHIPS_NOT_ALLOWED                                          Handle        = 0x80080202
+	APPX_E_MISSING_REQUIRED_FILE                                              Handle        = 0x80080203
+	APPX_E_INVALID_MANIFEST                                                   Handle        = 0x80080204
+	APPX_E_INVALID_BLOCKMAP                                                   Handle        = 0x80080205
+	APPX_E_CORRUPT_CONTENT                                                    Handle        = 0x80080206
+	APPX_E_BLOCK_HASH_INVALID                                                 Handle        = 0x80080207
+	APPX_E_REQUESTED_RANGE_TOO_LARGE                                          Handle        = 0x80080208
+	APPX_E_INVALID_SIP_CLIENT_DATA                                            Handle        = 0x80080209
+	APPX_E_INVALID_KEY_INFO                                                   Handle        = 0x8008020A
+	APPX_E_INVALID_CONTENTGROUPMAP                                            Handle        = 0x8008020B
+	APPX_E_INVALID_APPINSTALLER                                               Handle        = 0x8008020C
+	APPX_E_DELTA_BASELINE_VERSION_MISMATCH                                    Handle        = 0x8008020D
+	APPX_E_DELTA_PACKAGE_MISSING_FILE                                         Handle        = 0x8008020E
+	APPX_E_INVALID_DELTA_PACKAGE                                              Handle        = 0x8008020F
+	APPX_E_DELTA_APPENDED_PACKAGE_NOT_ALLOWED                                 Handle        = 0x80080210
+	APPX_E_INVALID_PACKAGING_LAYOUT                                           Handle        = 0x80080211
+	APPX_E_INVALID_PACKAGESIGNCONFIG                                          Handle        = 0x80080212
+	APPX_E_RESOURCESPRI_NOT_ALLOWED                                           Handle        = 0x80080213
+	APPX_E_FILE_COMPRESSION_MISMATCH                                          Handle        = 0x80080214
+	APPX_E_INVALID_PAYLOAD_PACKAGE_EXTENSION                                  Handle        = 0x80080215
+	APPX_E_INVALID_ENCRYPTION_EXCLUSION_FILE_LIST                             Handle        = 0x80080216
+	BT_E_SPURIOUS_ACTIVATION                                                  Handle        = 0x80080300
+	DISP_E_UNKNOWNINTERFACE                                                   Handle        = 0x80020001
+	DISP_E_MEMBERNOTFOUND                                                     Handle        = 0x80020003
+	DISP_E_PARAMNOTFOUND                                                      Handle        = 0x80020004
+	DISP_E_TYPEMISMATCH                                                       Handle        = 0x80020005
+	DISP_E_UNKNOWNNAME                                                        Handle        = 0x80020006
+	DISP_E_NONAMEDARGS                                                        Handle        = 0x80020007
+	DISP_E_BADVARTYPE                                                         Handle        = 0x80020008
+	DISP_E_EXCEPTION                                                          Handle        = 0x80020009
+	DISP_E_OVERFLOW                                                           Handle        = 0x8002000A
+	DISP_E_BADINDEX                                                           Handle        = 0x8002000B
+	DISP_E_UNKNOWNLCID                                                        Handle        = 0x8002000C
+	DISP_E_ARRAYISLOCKED                                                      Handle        = 0x8002000D
+	DISP_E_BADPARAMCOUNT                                                      Handle        = 0x8002000E
+	DISP_E_PARAMNOTOPTIONAL                                                   Handle        = 0x8002000F
+	DISP_E_BADCALLEE                                                          Handle        = 0x80020010
+	DISP_E_NOTACOLLECTION                                                     Handle        = 0x80020011
+	DISP_E_DIVBYZERO                                                          Handle        = 0x80020012
+	DISP_E_BUFFERTOOSMALL                                                     Handle        = 0x80020013
+	TYPE_E_BUFFERTOOSMALL                                                     Handle        = 0x80028016
+	TYPE_E_FIELDNOTFOUND                                                      Handle        = 0x80028017
+	TYPE_E_INVDATAREAD                                                        Handle        = 0x80028018
+	TYPE_E_UNSUPFORMAT                                                        Handle        = 0x80028019
+	TYPE_E_REGISTRYACCESS                                                     Handle        = 0x8002801C
+	TYPE_E_LIBNOTREGISTERED                                                   Handle        = 0x8002801D
+	TYPE_E_UNDEFINEDTYPE                                                      Handle        = 0x80028027
+	TYPE_E_QUALIFIEDNAMEDISALLOWED                                            Handle        = 0x80028028
+	TYPE_E_INVALIDSTATE                                                       Handle        = 0x80028029
+	TYPE_E_WRONGTYPEKIND                                                      Handle        = 0x8002802A
+	TYPE_E_ELEMENTNOTFOUND                                                    Handle        = 0x8002802B
+	TYPE_E_AMBIGUOUSNAME                                                      Handle        = 0x8002802C
+	TYPE_E_NAMECONFLICT                                                       Handle        = 0x8002802D
+	TYPE_E_UNKNOWNLCID                                                        Handle        = 0x8002802E
+	TYPE_E_DLLFUNCTIONNOTFOUND                                                Handle        = 0x8002802F
+	TYPE_E_BADMODULEKIND                                                      Handle        = 0x800288BD
+	TYPE_E_SIZETOOBIG                                                         Handle        = 0x800288C5
+	TYPE_E_DUPLICATEID                                                        Handle        = 0x800288C6
+	TYPE_E_INVALIDID                                                          Handle        = 0x800288CF
+	TYPE_E_TYPEMISMATCH                                                       Handle        = 0x80028CA0
+	TYPE_E_OUTOFBOUNDS                                                        Handle        = 0x80028CA1
+	TYPE_E_IOERROR                                                            Handle        = 0x80028CA2
+	TYPE_E_CANTCREATETMPFILE                                                  Handle        = 0x80028CA3
+	TYPE_E_CANTLOADLIBRARY                                                    Handle        = 0x80029C4A
+	TYPE_E_INCONSISTENTPROPFUNCS                                              Handle        = 0x80029C83
+	TYPE_E_CIRCULARTYPE                                                       Handle        = 0x80029C84
+	STG_E_INVALIDFUNCTION                                                     Handle        = 0x80030001
+	STG_E_FILENOTFOUND                                                        Handle        = 0x80030002
+	STG_E_PATHNOTFOUND                                                        Handle        = 0x80030003
+	STG_E_TOOMANYOPENFILES                                                    Handle        = 0x80030004
+	STG_E_ACCESSDENIED                                                        Handle        = 0x80030005
+	STG_E_INVALIDHANDLE                                                       Handle        = 0x80030006
+	STG_E_INSUFFICIENTMEMORY                                                  Handle        = 0x80030008
+	STG_E_INVALIDPOINTER                                                      Handle        = 0x80030009
+	STG_E_NOMOREFILES                                                         Handle        = 0x80030012
+	STG_E_DISKISWRITEPROTECTED                                                Handle        = 0x80030013
+	STG_E_SEEKERROR                                                           Handle        = 0x80030019
+	STG_E_WRITEFAULT                                                          Handle        = 0x8003001D
+	STG_E_READFAULT                                                           Handle        = 0x8003001E
+	STG_E_SHAREVIOLATION                                                      Handle        = 0x80030020
+	STG_E_LOCKVIOLATION                                                       Handle        = 0x80030021
+	STG_E_FILEALREADYEXISTS                                                   Handle        = 0x80030050
+	STG_E_INVALIDPARAMETER                                                    Handle        = 0x80030057
+	STG_E_MEDIUMFULL                                                          Handle        = 0x80030070
+	STG_E_PROPSETMISMATCHED                                                   Handle        = 0x800300F0
+	STG_E_ABNORMALAPIEXIT                                                     Handle        = 0x800300FA
+	STG_E_INVALIDHEADER                                                       Handle        = 0x800300FB
+	STG_E_INVALIDNAME                                                         Handle        = 0x800300FC
+	STG_E_UNKNOWN                                                             Handle        = 0x800300FD
+	STG_E_UNIMPLEMENTEDFUNCTION                                               Handle        = 0x800300FE
+	STG_E_INVALIDFLAG                                                         Handle        = 0x800300FF
+	STG_E_INUSE                                                               Handle        = 0x80030100
+	STG_E_NOTCURRENT                                                          Handle        = 0x80030101
+	STG_E_REVERTED                                                            Handle        = 0x80030102
+	STG_E_CANTSAVE                                                            Handle        = 0x80030103
+	STG_E_OLDFORMAT                                                           Handle        = 0x80030104
+	STG_E_OLDDLL                                                              Handle        = 0x80030105
+	STG_E_SHAREREQUIRED                                                       Handle        = 0x80030106
+	STG_E_NOTFILEBASEDSTORAGE                                                 Handle        = 0x80030107
+	STG_E_EXTANTMARSHALLINGS                                                  Handle        = 0x80030108
+	STG_E_DOCFILECORRUPT                                                      Handle        = 0x80030109
+	STG_E_BADBASEADDRESS                                                      Handle        = 0x80030110
+	STG_E_DOCFILETOOLARGE                                                     Handle        = 0x80030111
+	STG_E_NOTSIMPLEFORMAT                                                     Handle        = 0x80030112
+	STG_E_INCOMPLETE                                                          Handle        = 0x80030201
+	STG_E_TERMINATED                                                          Handle        = 0x80030202
+	STG_S_CONVERTED                                                           Handle        = 0x00030200
+	STG_S_BLOCK                                                               Handle        = 0x00030201
+	STG_S_RETRYNOW                                                            Handle        = 0x00030202
+	STG_S_MONITORING                                                          Handle        = 0x00030203
+	STG_S_MULTIPLEOPENS                                                       Handle        = 0x00030204
+	STG_S_CONSOLIDATIONFAILED                                                 Handle        = 0x00030205
+	STG_S_CANNOTCONSOLIDATE                                                   Handle        = 0x00030206
+	STG_S_POWER_CYCLE_REQUIRED                                                Handle        = 0x00030207
+	STG_E_FIRMWARE_SLOT_INVALID                                               Handle        = 0x80030208
+	STG_E_FIRMWARE_IMAGE_INVALID                                              Handle        = 0x80030209
+	STG_E_DEVICE_UNRESPONSIVE                                                 Handle        = 0x8003020A
+	STG_E_STATUS_COPY_PROTECTION_FAILURE                                      Handle        = 0x80030305
+	STG_E_CSS_AUTHENTICATION_FAILURE                                          Handle        = 0x80030306
+	STG_E_CSS_KEY_NOT_PRESENT                                                 Handle        = 0x80030307
+	STG_E_CSS_KEY_NOT_ESTABLISHED                                             Handle        = 0x80030308
+	STG_E_CSS_SCRAMBLED_SECTOR                                                Handle        = 0x80030309
+	STG_E_CSS_REGION_MISMATCH                                                 Handle        = 0x8003030A
+	STG_E_RESETS_EXHAUSTED                                                    Handle        = 0x8003030B
+	RPC_E_CALL_REJECTED                                                       Handle        = 0x80010001
+	RPC_E_CALL_CANCELED                                                       Handle        = 0x80010002
+	RPC_E_CANTPOST_INSENDCALL                                                 Handle        = 0x80010003
+	RPC_E_CANTCALLOUT_INASYNCCALL                                             Handle        = 0x80010004
+	RPC_E_CANTCALLOUT_INEXTERNALCALL                                          Handle        = 0x80010005
+	RPC_E_CONNECTION_TERMINATED                                               Handle        = 0x80010006
+	RPC_E_SERVER_DIED                                                         Handle        = 0x80010007
+	RPC_E_CLIENT_DIED                                                         Handle        = 0x80010008
+	RPC_E_INVALID_DATAPACKET                                                  Handle        = 0x80010009
+	RPC_E_CANTTRANSMIT_CALL                                                   Handle        = 0x8001000A
+	RPC_E_CLIENT_CANTMARSHAL_DATA                                             Handle        = 0x8001000B
+	RPC_E_CLIENT_CANTUNMARSHAL_DATA                                           Handle        = 0x8001000C
+	RPC_E_SERVER_CANTMARSHAL_DATA                                             Handle        = 0x8001000D
+	RPC_E_SERVER_CANTUNMARSHAL_DATA                                           Handle        = 0x8001000E
+	RPC_E_INVALID_DATA                                                        Handle        = 0x8001000F
+	RPC_E_INVALID_PARAMETER                                                   Handle        = 0x80010010
+	RPC_E_CANTCALLOUT_AGAIN                                                   Handle        = 0x80010011
+	RPC_E_SERVER_DIED_DNE                                                     Handle        = 0x80010012
+	RPC_E_SYS_CALL_FAILED                                                     Handle        = 0x80010100
+	RPC_E_OUT_OF_RESOURCES                                                    Handle        = 0x80010101
+	RPC_E_ATTEMPTED_MULTITHREAD                                               Handle        = 0x80010102
+	RPC_E_NOT_REGISTERED                                                      Handle        = 0x80010103
+	RPC_E_FAULT                                                               Handle        = 0x80010104
+	RPC_E_SERVERFAULT                                                         Handle        = 0x80010105
+	RPC_E_CHANGED_MODE                                                        Handle        = 0x80010106
+	RPC_E_INVALIDMETHOD                                                       Handle        = 0x80010107
+	RPC_E_DISCONNECTED                                                        Handle        = 0x80010108
+	RPC_E_RETRY                                                               Handle        = 0x80010109
+	RPC_E_SERVERCALL_RETRYLATER                                               Handle        = 0x8001010A
+	RPC_E_SERVERCALL_REJECTED                                                 Handle        = 0x8001010B
+	RPC_E_INVALID_CALLDATA                                                    Handle        = 0x8001010C
+	RPC_E_CANTCALLOUT_ININPUTSYNCCALL                                         Handle        = 0x8001010D
+	RPC_E_WRONG_THREAD                                                        Handle        = 0x8001010E
+	RPC_E_THREAD_NOT_INIT                                                     Handle        = 0x8001010F
+	RPC_E_VERSION_MISMATCH                                                    Handle        = 0x80010110
+	RPC_E_INVALID_HEADER                                                      Handle        = 0x80010111
+	RPC_E_INVALID_EXTENSION                                                   Handle        = 0x80010112
+	RPC_E_INVALID_IPID                                                        Handle        = 0x80010113
+	RPC_E_INVALID_OBJECT                                                      Handle        = 0x80010114
+	RPC_S_CALLPENDING                                                         Handle        = 0x80010115
+	RPC_S_WAITONTIMER                                                         Handle        = 0x80010116
+	RPC_E_CALL_COMPLETE                                                       Handle        = 0x80010117
+	RPC_E_UNSECURE_CALL                                                       Handle        = 0x80010118
+	RPC_E_TOO_LATE                                                            Handle        = 0x80010119
+	RPC_E_NO_GOOD_SECURITY_PACKAGES                                           Handle        = 0x8001011A
+	RPC_E_ACCESS_DENIED                                                       Handle        = 0x8001011B
+	RPC_E_REMOTE_DISABLED                                                     Handle        = 0x8001011C
+	RPC_E_INVALID_OBJREF                                                      Handle        = 0x8001011D
+	RPC_E_NO_CONTEXT                                                          Handle        = 0x8001011E
+	RPC_E_TIMEOUT                                                             Handle        = 0x8001011F
+	RPC_E_NO_SYNC                                                             Handle        = 0x80010120
+	RPC_E_FULLSIC_REQUIRED                                                    Handle        = 0x80010121
+	RPC_E_INVALID_STD_NAME                                                    Handle        = 0x80010122
+	CO_E_FAILEDTOIMPERSONATE                                                  Handle        = 0x80010123
+	CO_E_FAILEDTOGETSECCTX                                                    Handle        = 0x80010124
+	CO_E_FAILEDTOOPENTHREADTOKEN                                              Handle        = 0x80010125
+	CO_E_FAILEDTOGETTOKENINFO                                                 Handle        = 0x80010126
+	CO_E_TRUSTEEDOESNTMATCHCLIENT                                             Handle        = 0x80010127
+	CO_E_FAILEDTOQUERYCLIENTBLANKET                                           Handle        = 0x80010128
+	CO_E_FAILEDTOSETDACL                                                      Handle        = 0x80010129
+	CO_E_ACCESSCHECKFAILED                                                    Handle        = 0x8001012A
+	CO_E_NETACCESSAPIFAILED                                                   Handle        = 0x8001012B
+	CO_E_WRONGTRUSTEENAMESYNTAX                                               Handle        = 0x8001012C
+	CO_E_INVALIDSID                                                           Handle        = 0x8001012D
+	CO_E_CONVERSIONFAILED                                                     Handle        = 0x8001012E
+	CO_E_NOMATCHINGSIDFOUND                                                   Handle        = 0x8001012F
+	CO_E_LOOKUPACCSIDFAILED                                                   Handle        = 0x80010130
+	CO_E_NOMATCHINGNAMEFOUND                                                  Handle        = 0x80010131
+	CO_E_LOOKUPACCNAMEFAILED                                                  Handle        = 0x80010132
+	CO_E_SETSERLHNDLFAILED                                                    Handle        = 0x80010133
+	CO_E_FAILEDTOGETWINDIR                                                    Handle        = 0x80010134
+	CO_E_PATHTOOLONG                                                          Handle        = 0x80010135
+	CO_E_FAILEDTOGENUUID                                                      Handle        = 0x80010136
+	CO_E_FAILEDTOCREATEFILE                                                   Handle        = 0x80010137
+	CO_E_FAILEDTOCLOSEHANDLE                                                  Handle        = 0x80010138
+	CO_E_EXCEEDSYSACLLIMIT                                                    Handle        = 0x80010139
+	CO_E_ACESINWRONGORDER                                                     Handle        = 0x8001013A
+	CO_E_INCOMPATIBLESTREAMVERSION                                            Handle        = 0x8001013B
+	CO_E_FAILEDTOOPENPROCESSTOKEN                                             Handle        = 0x8001013C
+	CO_E_DECODEFAILED                                                         Handle        = 0x8001013D
+	CO_E_ACNOTINITIALIZED                                                     Handle        = 0x8001013F
+	CO_E_CANCEL_DISABLED                                                      Handle        = 0x80010140
+	RPC_E_UNEXPECTED                                                          Handle        = 0x8001FFFF
+	ERROR_AUDITING_DISABLED                                                   Handle        = 0xC0090001
+	ERROR_ALL_SIDS_FILTERED                                                   Handle        = 0xC0090002
+	ERROR_BIZRULES_NOT_ENABLED                                                Handle        = 0xC0090003
+	NTE_BAD_UID                                                               Handle        = 0x80090001
+	NTE_BAD_HASH                                                              Handle        = 0x80090002
+	NTE_BAD_KEY                                                               Handle        = 0x80090003
+	NTE_BAD_LEN                                                               Handle        = 0x80090004
+	NTE_BAD_DATA                                                              Handle        = 0x80090005
+	NTE_BAD_SIGNATURE                                                         Handle        = 0x80090006
+	NTE_BAD_VER                                                               Handle        = 0x80090007
+	NTE_BAD_ALGID                                                             Handle        = 0x80090008
+	NTE_BAD_FLAGS                                                             Handle        = 0x80090009
+	NTE_BAD_TYPE                                                              Handle        = 0x8009000A
+	NTE_BAD_KEY_STATE                                                         Handle        = 0x8009000B
+	NTE_BAD_HASH_STATE                                                        Handle        = 0x8009000C
+	NTE_NO_KEY                                                                Handle        = 0x8009000D
+	NTE_NO_MEMORY                                                             Handle        = 0x8009000E
+	NTE_EXISTS                                                                Handle        = 0x8009000F
+	NTE_PERM                                                                  Handle        = 0x80090010
+	NTE_NOT_FOUND                                                             Handle        = 0x80090011
+	NTE_DOUBLE_ENCRYPT                                                        Handle        = 0x80090012
+	NTE_BAD_PROVIDER                                                          Handle        = 0x80090013
+	NTE_BAD_PROV_TYPE                                                         Handle        = 0x80090014
+	NTE_BAD_PUBLIC_KEY                                                        Handle        = 0x80090015
+	NTE_BAD_KEYSET                                                            Handle        = 0x80090016
+	NTE_PROV_TYPE_NOT_DEF                                                     Handle        = 0x80090017
+	NTE_PROV_TYPE_ENTRY_BAD                                                   Handle        = 0x80090018
+	NTE_KEYSET_NOT_DEF                                                        Handle        = 0x80090019
+	NTE_KEYSET_ENTRY_BAD                                                      Handle        = 0x8009001A
+	NTE_PROV_TYPE_NO_MATCH                                                    Handle        = 0x8009001B
+	NTE_SIGNATURE_FILE_BAD                                                    Handle        = 0x8009001C
+	NTE_PROVIDER_DLL_FAIL                                                     Handle        = 0x8009001D
+	NTE_PROV_DLL_NOT_FOUND                                                    Handle        = 0x8009001E
+	NTE_BAD_KEYSET_PARAM                                                      Handle        = 0x8009001F
+	NTE_FAIL                                                                  Handle        = 0x80090020
+	NTE_SYS_ERR                                                               Handle        = 0x80090021
+	NTE_SILENT_CONTEXT                                                        Handle        = 0x80090022
+	NTE_TOKEN_KEYSET_STORAGE_FULL                                             Handle        = 0x80090023
+	NTE_TEMPORARY_PROFILE                                                     Handle        = 0x80090024
+	NTE_FIXEDPARAMETER                                                        Handle        = 0x80090025
+	NTE_INVALID_HANDLE                                                        Handle        = 0x80090026
+	NTE_INVALID_PARAMETER                                                     Handle        = 0x80090027
+	NTE_BUFFER_TOO_SMALL                                                      Handle        = 0x80090028
+	NTE_NOT_SUPPORTED                                                         Handle        = 0x80090029
+	NTE_NO_MORE_ITEMS                                                         Handle        = 0x8009002A
+	NTE_BUFFERS_OVERLAP                                                       Handle        = 0x8009002B
+	NTE_DECRYPTION_FAILURE                                                    Handle        = 0x8009002C
+	NTE_INTERNAL_ERROR                                                        Handle        = 0x8009002D
+	NTE_UI_REQUIRED                                                           Handle        = 0x8009002E
+	NTE_HMAC_NOT_SUPPORTED                                                    Handle        = 0x8009002F
+	NTE_DEVICE_NOT_READY                                                      Handle        = 0x80090030
+	NTE_AUTHENTICATION_IGNORED                                                Handle        = 0x80090031
+	NTE_VALIDATION_FAILED                                                     Handle        = 0x80090032
+	NTE_INCORRECT_PASSWORD                                                    Handle        = 0x80090033
+	NTE_ENCRYPTION_FAILURE                                                    Handle        = 0x80090034
+	NTE_DEVICE_NOT_FOUND                                                      Handle        = 0x80090035
+	NTE_USER_CANCELLED                                                        Handle        = 0x80090036
+	NTE_PASSWORD_CHANGE_REQUIRED                                              Handle        = 0x80090037
+	NTE_NOT_ACTIVE_CONSOLE                                                    Handle        = 0x80090038
+	SEC_E_INSUFFICIENT_MEMORY                                                 Handle        = 0x80090300
+	SEC_E_INVALID_HANDLE                                                      Handle        = 0x80090301
+	SEC_E_UNSUPPORTED_FUNCTION                                                Handle        = 0x80090302
+	SEC_E_TARGET_UNKNOWN                                                      Handle        = 0x80090303
+	SEC_E_INTERNAL_ERROR                                                      Handle        = 0x80090304
+	SEC_E_SECPKG_NOT_FOUND                                                    Handle        = 0x80090305
+	SEC_E_NOT_OWNER                                                           Handle        = 0x80090306
+	SEC_E_CANNOT_INSTALL                                                      Handle        = 0x80090307
+	SEC_E_INVALID_TOKEN                                                       Handle        = 0x80090308
+	SEC_E_CANNOT_PACK                                                         Handle        = 0x80090309
+	SEC_E_QOP_NOT_SUPPORTED                                                   Handle        = 0x8009030A
+	SEC_E_NO_IMPERSONATION                                                    Handle        = 0x8009030B
+	SEC_E_LOGON_DENIED                                                        Handle        = 0x8009030C
+	SEC_E_UNKNOWN_CREDENTIALS                                                 Handle        = 0x8009030D
+	SEC_E_NO_CREDENTIALS                                                      Handle        = 0x8009030E
+	SEC_E_MESSAGE_ALTERED                                                     Handle        = 0x8009030F
+	SEC_E_OUT_OF_SEQUENCE                                                     Handle        = 0x80090310
+	SEC_E_NO_AUTHENTICATING_AUTHORITY                                         Handle        = 0x80090311
+	SEC_I_CONTINUE_NEEDED                                                     Handle        = 0x00090312
+	SEC_I_COMPLETE_NEEDED                                                     Handle        = 0x00090313
+	SEC_I_COMPLETE_AND_CONTINUE                                               Handle        = 0x00090314
+	SEC_I_LOCAL_LOGON                                                         Handle        = 0x00090315
+	SEC_E_BAD_PKGID                                                           Handle        = 0x80090316
+	SEC_E_CONTEXT_EXPIRED                                                     Handle        = 0x80090317
+	SEC_I_CONTEXT_EXPIRED                                                     Handle        = 0x00090317
+	SEC_E_INCOMPLETE_MESSAGE                                                  Handle        = 0x80090318
+	SEC_E_INCOMPLETE_CREDENTIALS                                              Handle        = 0x80090320
+	SEC_E_BUFFER_TOO_SMALL                                                    Handle        = 0x80090321
+	SEC_I_INCOMPLETE_CREDENTIALS                                              Handle        = 0x00090320
+	SEC_I_RENEGOTIATE                                                         Handle        = 0x00090321
+	SEC_E_WRONG_PRINCIPAL                                                     Handle        = 0x80090322
+	SEC_I_NO_LSA_CONTEXT                                                      Handle        = 0x00090323
+	SEC_E_TIME_SKEW                                                           Handle        = 0x80090324
+	SEC_E_UNTRUSTED_ROOT                                                      Handle        = 0x80090325
+	SEC_E_ILLEGAL_MESSAGE                                                     Handle        = 0x80090326
+	SEC_E_CERT_UNKNOWN                                                        Handle        = 0x80090327
+	SEC_E_CERT_EXPIRED                                                        Handle        = 0x80090328
+	SEC_E_ENCRYPT_FAILURE                                                     Handle        = 0x80090329
+	SEC_E_DECRYPT_FAILURE                                                     Handle        = 0x80090330
+	SEC_E_ALGORITHM_MISMATCH                                                  Handle        = 0x80090331
+	SEC_E_SECURITY_QOS_FAILED                                                 Handle        = 0x80090332
+	SEC_E_UNFINISHED_CONTEXT_DELETED                                          Handle        = 0x80090333
+	SEC_E_NO_TGT_REPLY                                                        Handle        = 0x80090334
+	SEC_E_NO_IP_ADDRESSES                                                     Handle        = 0x80090335
+	SEC_E_WRONG_CREDENTIAL_HANDLE                                             Handle        = 0x80090336
+	SEC_E_CRYPTO_SYSTEM_INVALID                                               Handle        = 0x80090337
+	SEC_E_MAX_REFERRALS_EXCEEDED                                              Handle        = 0x80090338
+	SEC_E_MUST_BE_KDC                                                         Handle        = 0x80090339
+	SEC_E_STRONG_CRYPTO_NOT_SUPPORTED                                         Handle        = 0x8009033A
+	SEC_E_TOO_MANY_PRINCIPALS                                                 Handle        = 0x8009033B
+	SEC_E_NO_PA_DATA                                                          Handle        = 0x8009033C
+	SEC_E_PKINIT_NAME_MISMATCH                                                Handle        = 0x8009033D
+	SEC_E_SMARTCARD_LOGON_REQUIRED                                            Handle        = 0x8009033E
+	SEC_E_SHUTDOWN_IN_PROGRESS                                                Handle        = 0x8009033F
+	SEC_E_KDC_INVALID_REQUEST                                                 Handle        = 0x80090340
+	SEC_E_KDC_UNABLE_TO_REFER                                                 Handle        = 0x80090341
+	SEC_E_KDC_UNKNOWN_ETYPE                                                   Handle        = 0x80090342
+	SEC_E_UNSUPPORTED_PREAUTH                                                 Handle        = 0x80090343
+	SEC_E_DELEGATION_REQUIRED                                                 Handle        = 0x80090345
+	SEC_E_BAD_BINDINGS                                                        Handle        = 0x80090346
+	SEC_E_MULTIPLE_ACCOUNTS                                                   Handle        = 0x80090347
+	SEC_E_NO_KERB_KEY                                                         Handle        = 0x80090348
+	SEC_E_CERT_WRONG_USAGE                                                    Handle        = 0x80090349
+	SEC_E_DOWNGRADE_DETECTED                                                  Handle        = 0x80090350
+	SEC_E_SMARTCARD_CERT_REVOKED                                              Handle        = 0x80090351
+	SEC_E_ISSUING_CA_UNTRUSTED                                                Handle        = 0x80090352
+	SEC_E_REVOCATION_OFFLINE_C                                                Handle        = 0x80090353
+	SEC_E_PKINIT_CLIENT_FAILURE                                               Handle        = 0x80090354
+	SEC_E_SMARTCARD_CERT_EXPIRED                                              Handle        = 0x80090355
+	SEC_E_NO_S4U_PROT_SUPPORT                                                 Handle        = 0x80090356
+	SEC_E_CROSSREALM_DELEGATION_FAILURE                                       Handle        = 0x80090357
+	SEC_E_REVOCATION_OFFLINE_KDC                                              Handle        = 0x80090358
+	SEC_E_ISSUING_CA_UNTRUSTED_KDC                                            Handle        = 0x80090359
+	SEC_E_KDC_CERT_EXPIRED                                                    Handle        = 0x8009035A
+	SEC_E_KDC_CERT_REVOKED                                                    Handle        = 0x8009035B
+	SEC_I_SIGNATURE_NEEDED                                                    Handle        = 0x0009035C
+	SEC_E_INVALID_PARAMETER                                                   Handle        = 0x8009035D
+	SEC_E_DELEGATION_POLICY                                                   Handle        = 0x8009035E
+	SEC_E_POLICY_NLTM_ONLY                                                    Handle        = 0x8009035F
+	SEC_I_NO_RENEGOTIATION                                                    Handle        = 0x00090360
+	SEC_E_NO_CONTEXT                                                          Handle        = 0x80090361
+	SEC_E_PKU2U_CERT_FAILURE                                                  Handle        = 0x80090362
+	SEC_E_MUTUAL_AUTH_FAILED                                                  Handle        = 0x80090363
+	SEC_I_MESSAGE_FRAGMENT                                                    Handle        = 0x00090364
+	SEC_E_ONLY_HTTPS_ALLOWED                                                  Handle        = 0x80090365
+	SEC_I_CONTINUE_NEEDED_MESSAGE_OK                                          Handle        = 0x00090366
+	SEC_E_APPLICATION_PROTOCOL_MISMATCH                                       Handle        = 0x80090367
+	SEC_I_ASYNC_CALL_PENDING                                                  Handle        = 0x00090368
+	SEC_E_INVALID_UPN_NAME                                                    Handle        = 0x80090369
+	SEC_E_NO_SPM                                                                            = SEC_E_INTERNAL_ERROR
+	SEC_E_NOT_SUPPORTED                                                                     = SEC_E_UNSUPPORTED_FUNCTION
+	CRYPT_E_MSG_ERROR                                                         Handle        = 0x80091001
+	CRYPT_E_UNKNOWN_ALGO                                                      Handle        = 0x80091002
+	CRYPT_E_OID_FORMAT                                                        Handle        = 0x80091003
+	CRYPT_E_INVALID_MSG_TYPE                                                  Handle        = 0x80091004
+	CRYPT_E_UNEXPECTED_ENCODING                                               Handle        = 0x80091005
+	CRYPT_E_AUTH_ATTR_MISSING                                                 Handle        = 0x80091006
+	CRYPT_E_HASH_VALUE                                                        Handle        = 0x80091007
+	CRYPT_E_INVALID_INDEX                                                     Handle        = 0x80091008
+	CRYPT_E_ALREADY_DECRYPTED                                                 Handle        = 0x80091009
+	CRYPT_E_NOT_DECRYPTED                                                     Handle        = 0x8009100A
+	CRYPT_E_RECIPIENT_NOT_FOUND                                               Handle        = 0x8009100B
+	CRYPT_E_CONTROL_TYPE                                                      Handle        = 0x8009100C
+	CRYPT_E_ISSUER_SERIALNUMBER                                               Handle        = 0x8009100D
+	CRYPT_E_SIGNER_NOT_FOUND                                                  Handle        = 0x8009100E
+	CRYPT_E_ATTRIBUTES_MISSING                                                Handle        = 0x8009100F
+	CRYPT_E_STREAM_MSG_NOT_READY                                              Handle        = 0x80091010
+	CRYPT_E_STREAM_INSUFFICIENT_DATA                                          Handle        = 0x80091011
+	CRYPT_I_NEW_PROTECTION_REQUIRED                                           Handle        = 0x00091012
+	CRYPT_E_BAD_LEN                                                           Handle        = 0x80092001
+	CRYPT_E_BAD_ENCODE                                                        Handle        = 0x80092002
+	CRYPT_E_FILE_ERROR                                                        Handle        = 0x80092003
+	CRYPT_E_NOT_FOUND                                                         Handle        = 0x80092004
+	CRYPT_E_EXISTS                                                            Handle        = 0x80092005
+	CRYPT_E_NO_PROVIDER                                                       Handle        = 0x80092006
+	CRYPT_E_SELF_SIGNED                                                       Handle        = 0x80092007
+	CRYPT_E_DELETED_PREV                                                      Handle        = 0x80092008
+	CRYPT_E_NO_MATCH                                                          Handle        = 0x80092009
+	CRYPT_E_UNEXPECTED_MSG_TYPE                                               Handle        = 0x8009200A
+	CRYPT_E_NO_KEY_PROPERTY                                                   Handle        = 0x8009200B
+	CRYPT_E_NO_DECRYPT_CERT                                                   Handle        = 0x8009200C
+	CRYPT_E_BAD_MSG                                                           Handle        = 0x8009200D
+	CRYPT_E_NO_SIGNER                                                         Handle        = 0x8009200E
+	CRYPT_E_PENDING_CLOSE                                                     Handle        = 0x8009200F
+	CRYPT_E_REVOKED                                                           Handle        = 0x80092010
+	CRYPT_E_NO_REVOCATION_DLL                                                 Handle        = 0x80092011
+	CRYPT_E_NO_REVOCATION_CHECK                                               Handle        = 0x80092012
+	CRYPT_E_REVOCATION_OFFLINE                                                Handle        = 0x80092013
+	CRYPT_E_NOT_IN_REVOCATION_DATABASE                                        Handle        = 0x80092014
+	CRYPT_E_INVALID_NUMERIC_STRING                                            Handle        = 0x80092020
+	CRYPT_E_INVALID_PRINTABLE_STRING                                          Handle        = 0x80092021
+	CRYPT_E_INVALID_IA5_STRING                                                Handle        = 0x80092022
+	CRYPT_E_INVALID_X500_STRING                                               Handle        = 0x80092023
+	CRYPT_E_NOT_CHAR_STRING                                                   Handle        = 0x80092024
+	CRYPT_E_FILERESIZED                                                       Handle        = 0x80092025
+	CRYPT_E_SECURITY_SETTINGS                                                 Handle        = 0x80092026
+	CRYPT_E_NO_VERIFY_USAGE_DLL                                               Handle        = 0x80092027
+	CRYPT_E_NO_VERIFY_USAGE_CHECK                                             Handle        = 0x80092028
+	CRYPT_E_VERIFY_USAGE_OFFLINE                                              Handle        = 0x80092029
+	CRYPT_E_NOT_IN_CTL                                                        Handle        = 0x8009202A
+	CRYPT_E_NO_TRUSTED_SIGNER                                                 Handle        = 0x8009202B
+	CRYPT_E_MISSING_PUBKEY_PARA                                               Handle        = 0x8009202C
+	CRYPT_E_OBJECT_LOCATOR_OBJECT_NOT_FOUND                                   Handle        = 0x8009202D
+	CRYPT_E_OSS_ERROR                                                         Handle        = 0x80093000
+	OSS_MORE_BUF                                                              Handle        = 0x80093001
+	OSS_NEGATIVE_UINTEGER                                                     Handle        = 0x80093002
+	OSS_PDU_RANGE                                                             Handle        = 0x80093003
+	OSS_MORE_INPUT                                                            Handle        = 0x80093004
+	OSS_DATA_ERROR                                                            Handle        = 0x80093005
+	OSS_BAD_ARG                                                               Handle        = 0x80093006
+	OSS_BAD_VERSION                                                           Handle        = 0x80093007
+	OSS_OUT_MEMORY                                                            Handle        = 0x80093008
+	OSS_PDU_MISMATCH                                                          Handle        = 0x80093009
+	OSS_LIMITED                                                               Handle        = 0x8009300A
+	OSS_BAD_PTR                                                               Handle        = 0x8009300B
+	OSS_BAD_TIME                                                              Handle        = 0x8009300C
+	OSS_INDEFINITE_NOT_SUPPORTED                                              Handle        = 0x8009300D
+	OSS_MEM_ERROR                                                             Handle        = 0x8009300E
+	OSS_BAD_TABLE                                                             Handle        = 0x8009300F
+	OSS_TOO_LONG                                                              Handle        = 0x80093010
+	OSS_CONSTRAINT_VIOLATED                                                   Handle        = 0x80093011
+	OSS_FATAL_ERROR                                                           Handle        = 0x80093012
+	OSS_ACCESS_SERIALIZATION_ERROR                                            Handle        = 0x80093013
+	OSS_NULL_TBL                                                              Handle        = 0x80093014
+	OSS_NULL_FCN                                                              Handle        = 0x80093015
+	OSS_BAD_ENCRULES                                                          Handle        = 0x80093016
+	OSS_UNAVAIL_ENCRULES                                                      Handle        = 0x80093017
+	OSS_CANT_OPEN_TRACE_WINDOW                                                Handle        = 0x80093018
+	OSS_UNIMPLEMENTED                                                         Handle        = 0x80093019
+	OSS_OID_DLL_NOT_LINKED                                                    Handle        = 0x8009301A
+	OSS_CANT_OPEN_TRACE_FILE                                                  Handle        = 0x8009301B
+	OSS_TRACE_FILE_ALREADY_OPEN                                               Handle        = 0x8009301C
+	OSS_TABLE_MISMATCH                                                        Handle        = 0x8009301D
+	OSS_TYPE_NOT_SUPPORTED                                                    Handle        = 0x8009301E
+	OSS_REAL_DLL_NOT_LINKED                                                   Handle        = 0x8009301F
+	OSS_REAL_CODE_NOT_LINKED                                                  Handle        = 0x80093020
+	OSS_OUT_OF_RANGE                                                          Handle        = 0x80093021
+	OSS_COPIER_DLL_NOT_LINKED                                                 Handle        = 0x80093022
+	OSS_CONSTRAINT_DLL_NOT_LINKED                                             Handle        = 0x80093023
+	OSS_COMPARATOR_DLL_NOT_LINKED                                             Handle        = 0x80093024
+	OSS_COMPARATOR_CODE_NOT_LINKED                                            Handle        = 0x80093025
+	OSS_MEM_MGR_DLL_NOT_LINKED                                                Handle        = 0x80093026
+	OSS_PDV_DLL_NOT_LINKED                                                    Handle        = 0x80093027
+	OSS_PDV_CODE_NOT_LINKED                                                   Handle        = 0x80093028
+	OSS_API_DLL_NOT_LINKED                                                    Handle        = 0x80093029
+	OSS_BERDER_DLL_NOT_LINKED                                                 Handle        = 0x8009302A
+	OSS_PER_DLL_NOT_LINKED                                                    Handle        = 0x8009302B
+	OSS_OPEN_TYPE_ERROR                                                       Handle        = 0x8009302C
+	OSS_MUTEX_NOT_CREATED                                                     Handle        = 0x8009302D
+	OSS_CANT_CLOSE_TRACE_FILE                                                 Handle        = 0x8009302E
+	CRYPT_E_ASN1_ERROR                                                        Handle        = 0x80093100
+	CRYPT_E_ASN1_INTERNAL                                                     Handle        = 0x80093101
+	CRYPT_E_ASN1_EOD                                                          Handle        = 0x80093102
+	CRYPT_E_ASN1_CORRUPT                                                      Handle        = 0x80093103
+	CRYPT_E_ASN1_LARGE                                                        Handle        = 0x80093104
+	CRYPT_E_ASN1_CONSTRAINT                                                   Handle        = 0x80093105
+	CRYPT_E_ASN1_MEMORY                                                       Handle        = 0x80093106
+	CRYPT_E_ASN1_OVERFLOW                                                     Handle        = 0x80093107
+	CRYPT_E_ASN1_BADPDU                                                       Handle        = 0x80093108
+	CRYPT_E_ASN1_BADARGS                                                      Handle        = 0x80093109
+	CRYPT_E_ASN1_BADREAL                                                      Handle        = 0x8009310A
+	CRYPT_E_ASN1_BADTAG                                                       Handle        = 0x8009310B
+	CRYPT_E_ASN1_CHOICE                                                       Handle        = 0x8009310C
+	CRYPT_E_ASN1_RULE                                                         Handle        = 0x8009310D
+	CRYPT_E_ASN1_UTF8                                                         Handle        = 0x8009310E
+	CRYPT_E_ASN1_PDU_TYPE                                                     Handle        = 0x80093133
+	CRYPT_E_ASN1_NYI                                                          Handle        = 0x80093134
+	CRYPT_E_ASN1_EXTENDED                                                     Handle        = 0x80093201
+	CRYPT_E_ASN1_NOEOD                                                        Handle        = 0x80093202
+	CERTSRV_E_BAD_REQUESTSUBJECT                                              Handle        = 0x80094001
+	CERTSRV_E_NO_REQUEST                                                      Handle        = 0x80094002
+	CERTSRV_E_BAD_REQUESTSTATUS                                               Handle        = 0x80094003
+	CERTSRV_E_PROPERTY_EMPTY                                                  Handle        = 0x80094004
+	CERTSRV_E_INVALID_CA_CERTIFICATE                                          Handle        = 0x80094005
+	CERTSRV_E_SERVER_SUSPENDED                                                Handle        = 0x80094006
+	CERTSRV_E_ENCODING_LENGTH                                                 Handle        = 0x80094007
+	CERTSRV_E_ROLECONFLICT                                                    Handle        = 0x80094008
+	CERTSRV_E_RESTRICTEDOFFICER                                               Handle        = 0x80094009
+	CERTSRV_E_KEY_ARCHIVAL_NOT_CONFIGURED                                     Handle        = 0x8009400A
+	CERTSRV_E_NO_VALID_KRA                                                    Handle        = 0x8009400B
+	CERTSRV_E_BAD_REQUEST_KEY_ARCHIVAL                                        Handle        = 0x8009400C
+	CERTSRV_E_NO_CAADMIN_DEFINED                                              Handle        = 0x8009400D
+	CERTSRV_E_BAD_RENEWAL_CERT_ATTRIBUTE                                      Handle        = 0x8009400E
+	CERTSRV_E_NO_DB_SESSIONS                                                  Handle        = 0x8009400F
+	CERTSRV_E_ALIGNMENT_FAULT                                                 Handle        = 0x80094010
+	CERTSRV_E_ENROLL_DENIED                                                   Handle        = 0x80094011
+	CERTSRV_E_TEMPLATE_DENIED                                                 Handle        = 0x80094012
+	CERTSRV_E_DOWNLEVEL_DC_SSL_OR_UPGRADE                                     Handle        = 0x80094013
+	CERTSRV_E_ADMIN_DENIED_REQUEST                                            Handle        = 0x80094014
+	CERTSRV_E_NO_POLICY_SERVER                                                Handle        = 0x80094015
+	CERTSRV_E_WEAK_SIGNATURE_OR_KEY                                           Handle        = 0x80094016
+	CERTSRV_E_KEY_ATTESTATION_NOT_SUPPORTED                                   Handle        = 0x80094017
+	CERTSRV_E_ENCRYPTION_CERT_REQUIRED                                        Handle        = 0x80094018
+	CERTSRV_E_UNSUPPORTED_CERT_TYPE                                           Handle        = 0x80094800
+	CERTSRV_E_NO_CERT_TYPE                                                    Handle        = 0x80094801
+	CERTSRV_E_TEMPLATE_CONFLICT                                               Handle        = 0x80094802
+	CERTSRV_E_SUBJECT_ALT_NAME_REQUIRED                                       Handle        = 0x80094803
+	CERTSRV_E_ARCHIVED_KEY_REQUIRED                                           Handle        = 0x80094804
+	CERTSRV_E_SMIME_REQUIRED                                                  Handle        = 0x80094805
+	CERTSRV_E_BAD_RENEWAL_SUBJECT                                             Handle        = 0x80094806
+	CERTSRV_E_BAD_TEMPLATE_VERSION                                            Handle        = 0x80094807
+	CERTSRV_E_TEMPLATE_POLICY_REQUIRED                                        Handle        = 0x80094808
+	CERTSRV_E_SIGNATURE_POLICY_REQUIRED                                       Handle        = 0x80094809
+	CERTSRV_E_SIGNATURE_COUNT                                                 Handle        = 0x8009480A
+	CERTSRV_E_SIGNATURE_REJECTED                                              Handle        = 0x8009480B
+	CERTSRV_E_ISSUANCE_POLICY_REQUIRED                                        Handle        = 0x8009480C
+	CERTSRV_E_SUBJECT_UPN_REQUIRED                                            Handle        = 0x8009480D
+	CERTSRV_E_SUBJECT_DIRECTORY_GUID_REQUIRED                                 Handle        = 0x8009480E
+	CERTSRV_E_SUBJECT_DNS_REQUIRED                                            Handle        = 0x8009480F
+	CERTSRV_E_ARCHIVED_KEY_UNEXPECTED                                         Handle        = 0x80094810
+	CERTSRV_E_KEY_LENGTH                                                      Handle        = 0x80094811
+	CERTSRV_E_SUBJECT_EMAIL_REQUIRED                                          Handle        = 0x80094812
+	CERTSRV_E_UNKNOWN_CERT_TYPE                                               Handle        = 0x80094813
+	CERTSRV_E_CERT_TYPE_OVERLAP                                               Handle        = 0x80094814
+	CERTSRV_E_TOO_MANY_SIGNATURES                                             Handle        = 0x80094815
+	CERTSRV_E_RENEWAL_BAD_PUBLIC_KEY                                          Handle        = 0x80094816
+	CERTSRV_E_INVALID_EK                                                      Handle        = 0x80094817
+	CERTSRV_E_INVALID_IDBINDING                                               Handle        = 0x80094818
+	CERTSRV_E_INVALID_ATTESTATION                                             Handle        = 0x80094819
+	CERTSRV_E_KEY_ATTESTATION                                                 Handle        = 0x8009481A
+	CERTSRV_E_CORRUPT_KEY_ATTESTATION                                         Handle        = 0x8009481B
+	CERTSRV_E_EXPIRED_CHALLENGE                                               Handle        = 0x8009481C
+	CERTSRV_E_INVALID_RESPONSE                                                Handle        = 0x8009481D
+	CERTSRV_E_INVALID_REQUESTID                                               Handle        = 0x8009481E
+	CERTSRV_E_REQUEST_PRECERTIFICATE_MISMATCH                                 Handle        = 0x8009481F
+	CERTSRV_E_PENDING_CLIENT_RESPONSE                                         Handle        = 0x80094820
+	XENROLL_E_KEY_NOT_EXPORTABLE                                              Handle        = 0x80095000
+	XENROLL_E_CANNOT_ADD_ROOT_CERT                                            Handle        = 0x80095001
+	XENROLL_E_RESPONSE_KA_HASH_NOT_FOUND                                      Handle        = 0x80095002
+	XENROLL_E_RESPONSE_UNEXPECTED_KA_HASH                                     Handle        = 0x80095003
+	XENROLL_E_RESPONSE_KA_HASH_MISMATCH                                       Handle        = 0x80095004
+	XENROLL_E_KEYSPEC_SMIME_MISMATCH                                          Handle        = 0x80095005
+	TRUST_E_SYSTEM_ERROR                                                      Handle        = 0x80096001
+	TRUST_E_NO_SIGNER_CERT                                                    Handle        = 0x80096002
+	TRUST_E_COUNTER_SIGNER                                                    Handle        = 0x80096003
+	TRUST_E_CERT_SIGNATURE                                                    Handle        = 0x80096004
+	TRUST_E_TIME_STAMP                                                        Handle        = 0x80096005
+	TRUST_E_BAD_DIGEST                                                        Handle        = 0x80096010
+	TRUST_E_MALFORMED_SIGNATURE                                               Handle        = 0x80096011
+	TRUST_E_BASIC_CONSTRAINTS                                                 Handle        = 0x80096019
+	TRUST_E_FINANCIAL_CRITERIA                                                Handle        = 0x8009601E
+	MSSIPOTF_E_OUTOFMEMRANGE                                                  Handle        = 0x80097001
+	MSSIPOTF_E_CANTGETOBJECT                                                  Handle        = 0x80097002
+	MSSIPOTF_E_NOHEADTABLE                                                    Handle        = 0x80097003
+	MSSIPOTF_E_BAD_MAGICNUMBER                                                Handle        = 0x80097004
+	MSSIPOTF_E_BAD_OFFSET_TABLE                                               Handle        = 0x80097005
+	MSSIPOTF_E_TABLE_TAGORDER                                                 Handle        = 0x80097006
+	MSSIPOTF_E_TABLE_LONGWORD                                                 Handle        = 0x80097007
+	MSSIPOTF_E_BAD_FIRST_TABLE_PLACEMENT                                      Handle        = 0x80097008
+	MSSIPOTF_E_TABLES_OVERLAP                                                 Handle        = 0x80097009
+	MSSIPOTF_E_TABLE_PADBYTES                                                 Handle        = 0x8009700A
+	MSSIPOTF_E_FILETOOSMALL                                                   Handle        = 0x8009700B
+	MSSIPOTF_E_TABLE_CHECKSUM                                                 Handle        = 0x8009700C
+	MSSIPOTF_E_FILE_CHECKSUM                                                  Handle        = 0x8009700D
+	MSSIPOTF_E_FAILED_POLICY                                                  Handle        = 0x80097010
+	MSSIPOTF_E_FAILED_HINTS_CHECK                                             Handle        = 0x80097011
+	MSSIPOTF_E_NOT_OPENTYPE                                                   Handle        = 0x80097012
+	MSSIPOTF_E_FILE                                                           Handle        = 0x80097013
+	MSSIPOTF_E_CRYPT                                                          Handle        = 0x80097014
+	MSSIPOTF_E_BADVERSION                                                     Handle        = 0x80097015
+	MSSIPOTF_E_DSIG_STRUCTURE                                                 Handle        = 0x80097016
+	MSSIPOTF_E_PCONST_CHECK                                                   Handle        = 0x80097017
+	MSSIPOTF_E_STRUCTURE                                                      Handle        = 0x80097018
+	ERROR_CRED_REQUIRES_CONFIRMATION                                          Handle        = 0x80097019
+	NTE_OP_OK                                                                 syscall.Errno = 0
+	TRUST_E_PROVIDER_UNKNOWN                                                  Handle        = 0x800B0001
+	TRUST_E_ACTION_UNKNOWN                                                    Handle        = 0x800B0002
+	TRUST_E_SUBJECT_FORM_UNKNOWN                                              Handle        = 0x800B0003
+	TRUST_E_SUBJECT_NOT_TRUSTED                                               Handle        = 0x800B0004
+	DIGSIG_E_ENCODE                                                           Handle        = 0x800B0005
+	DIGSIG_E_DECODE                                                           Handle        = 0x800B0006
+	DIGSIG_E_EXTENSIBILITY                                                    Handle        = 0x800B0007
+	DIGSIG_E_CRYPTO                                                           Handle        = 0x800B0008
+	PERSIST_E_SIZEDEFINITE                                                    Handle        = 0x800B0009
+	PERSIST_E_SIZEINDEFINITE                                                  Handle        = 0x800B000A
+	PERSIST_E_NOTSELFSIZING                                                   Handle        = 0x800B000B
+	TRUST_E_NOSIGNATURE                                                       Handle        = 0x800B0100
+	CERT_E_EXPIRED                                                            Handle        = 0x800B0101
+	CERT_E_VALIDITYPERIODNESTING                                              Handle        = 0x800B0102
+	CERT_E_ROLE                                                               Handle        = 0x800B0103
+	CERT_E_PATHLENCONST                                                       Handle        = 0x800B0104
+	CERT_E_CRITICAL                                                           Handle        = 0x800B0105
+	CERT_E_PURPOSE                                                            Handle        = 0x800B0106
+	CERT_E_ISSUERCHAINING                                                     Handle        = 0x800B0107
+	CERT_E_MALFORMED                                                          Handle        = 0x800B0108
+	CERT_E_UNTRUSTEDROOT                                                      Handle        = 0x800B0109
+	CERT_E_CHAINING                                                           Handle        = 0x800B010A
+	TRUST_E_FAIL                                                              Handle        = 0x800B010B
+	CERT_E_REVOKED                                                            Handle        = 0x800B010C
+	CERT_E_UNTRUSTEDTESTROOT                                                  Handle        = 0x800B010D
+	CERT_E_REVOCATION_FAILURE                                                 Handle        = 0x800B010E
+	CERT_E_CN_NO_MATCH                                                        Handle        = 0x800B010F
+	CERT_E_WRONG_USAGE                                                        Handle        = 0x800B0110
+	TRUST_E_EXPLICIT_DISTRUST                                                 Handle        = 0x800B0111
+	CERT_E_UNTRUSTEDCA                                                        Handle        = 0x800B0112
+	CERT_E_INVALID_POLICY                                                     Handle        = 0x800B0113
+	CERT_E_INVALID_NAME                                                       Handle        = 0x800B0114
+	SPAPI_E_EXPECTED_SECTION_NAME                                             Handle        = 0x800F0000
+	SPAPI_E_BAD_SECTION_NAME_LINE                                             Handle        = 0x800F0001
+	SPAPI_E_SECTION_NAME_TOO_LONG                                             Handle        = 0x800F0002
+	SPAPI_E_GENERAL_SYNTAX                                                    Handle        = 0x800F0003
+	SPAPI_E_WRONG_INF_STYLE                                                   Handle        = 0x800F0100
+	SPAPI_E_SECTION_NOT_FOUND                                                 Handle        = 0x800F0101
+	SPAPI_E_LINE_NOT_FOUND                                                    Handle        = 0x800F0102
+	SPAPI_E_NO_BACKUP                                                         Handle        = 0x800F0103
+	SPAPI_E_NO_ASSOCIATED_CLASS                                               Handle        = 0x800F0200
+	SPAPI_E_CLASS_MISMATCH                                                    Handle        = 0x800F0201
+	SPAPI_E_DUPLICATE_FOUND                                                   Handle        = 0x800F0202
+	SPAPI_E_NO_DRIVER_SELECTED                                                Handle        = 0x800F0203
+	SPAPI_E_KEY_DOES_NOT_EXIST                                                Handle        = 0x800F0204
+	SPAPI_E_INVALID_DEVINST_NAME                                              Handle        = 0x800F0205
+	SPAPI_E_INVALID_CLASS                                                     Handle        = 0x800F0206
+	SPAPI_E_DEVINST_ALREADY_EXISTS                                            Handle        = 0x800F0207
+	SPAPI_E_DEVINFO_NOT_REGISTERED                                            Handle        = 0x800F0208
+	SPAPI_E_INVALID_REG_PROPERTY                                              Handle        = 0x800F0209
+	SPAPI_E_NO_INF                                                            Handle        = 0x800F020A
+	SPAPI_E_NO_SUCH_DEVINST                                                   Handle        = 0x800F020B
+	SPAPI_E_CANT_LOAD_CLASS_ICON                                              Handle        = 0x800F020C
+	SPAPI_E_INVALID_CLASS_INSTALLER                                           Handle        = 0x800F020D
+	SPAPI_E_DI_DO_DEFAULT                                                     Handle        = 0x800F020E
+	SPAPI_E_DI_NOFILECOPY                                                     Handle        = 0x800F020F
+	SPAPI_E_INVALID_HWPROFILE                                                 Handle        = 0x800F0210
+	SPAPI_E_NO_DEVICE_SELECTED                                                Handle        = 0x800F0211
+	SPAPI_E_DEVINFO_LIST_LOCKED                                               Handle        = 0x800F0212
+	SPAPI_E_DEVINFO_DATA_LOCKED                                               Handle        = 0x800F0213
+	SPAPI_E_DI_BAD_PATH                                                       Handle        = 0x800F0214
+	SPAPI_E_NO_CLASSINSTALL_PARAMS                                            Handle        = 0x800F0215
+	SPAPI_E_FILEQUEUE_LOCKED                                                  Handle        = 0x800F0216
+	SPAPI_E_BAD_SERVICE_INSTALLSECT                                           Handle        = 0x800F0217
+	SPAPI_E_NO_CLASS_DRIVER_LIST                                              Handle        = 0x800F0218
+	SPAPI_E_NO_ASSOCIATED_SERVICE                                             Handle        = 0x800F0219
+	SPAPI_E_NO_DEFAULT_DEVICE_INTERFACE                                       Handle        = 0x800F021A
+	SPAPI_E_DEVICE_INTERFACE_ACTIVE                                           Handle        = 0x800F021B
+	SPAPI_E_DEVICE_INTERFACE_REMOVED                                          Handle        = 0x800F021C
+	SPAPI_E_BAD_INTERFACE_INSTALLSECT                                         Handle        = 0x800F021D
+	SPAPI_E_NO_SUCH_INTERFACE_CLASS                                           Handle        = 0x800F021E
+	SPAPI_E_INVALID_REFERENCE_STRING                                          Handle        = 0x800F021F
+	SPAPI_E_INVALID_MACHINENAME                                               Handle        = 0x800F0220
+	SPAPI_E_REMOTE_COMM_FAILURE                                               Handle        = 0x800F0221
+	SPAPI_E_MACHINE_UNAVAILABLE                                               Handle        = 0x800F0222
+	SPAPI_E_NO_CONFIGMGR_SERVICES                                             Handle        = 0x800F0223
+	SPAPI_E_INVALID_PROPPAGE_PROVIDER                                         Handle        = 0x800F0224
+	SPAPI_E_NO_SUCH_DEVICE_INTERFACE                                          Handle        = 0x800F0225
+	SPAPI_E_DI_POSTPROCESSING_REQUIRED                                        Handle        = 0x800F0226
+	SPAPI_E_INVALID_COINSTALLER                                               Handle        = 0x800F0227
+	SPAPI_E_NO_COMPAT_DRIVERS                                                 Handle        = 0x800F0228
+	SPAPI_E_NO_DEVICE_ICON                                                    Handle        = 0x800F0229
+	SPAPI_E_INVALID_INF_LOGCONFIG                                             Handle        = 0x800F022A
+	SPAPI_E_DI_DONT_INSTALL                                                   Handle        = 0x800F022B
+	SPAPI_E_INVALID_FILTER_DRIVER                                             Handle        = 0x800F022C
+	SPAPI_E_NON_WINDOWS_NT_DRIVER                                             Handle        = 0x800F022D
+	SPAPI_E_NON_WINDOWS_DRIVER                                                Handle        = 0x800F022E
+	SPAPI_E_NO_CATALOG_FOR_OEM_INF                                            Handle        = 0x800F022F
+	SPAPI_E_DEVINSTALL_QUEUE_NONNATIVE                                        Handle        = 0x800F0230
+	SPAPI_E_NOT_DISABLEABLE                                                   Handle        = 0x800F0231
+	SPAPI_E_CANT_REMOVE_DEVINST                                               Handle        = 0x800F0232
+	SPAPI_E_INVALID_TARGET                                                    Handle        = 0x800F0233
+	SPAPI_E_DRIVER_NONNATIVE                                                  Handle        = 0x800F0234
+	SPAPI_E_IN_WOW64                                                          Handle        = 0x800F0235
+	SPAPI_E_SET_SYSTEM_RESTORE_POINT                                          Handle        = 0x800F0236
+	SPAPI_E_INCORRECTLY_COPIED_INF                                            Handle        = 0x800F0237
+	SPAPI_E_SCE_DISABLED                                                      Handle        = 0x800F0238
+	SPAPI_E_UNKNOWN_EXCEPTION                                                 Handle        = 0x800F0239
+	SPAPI_E_PNP_REGISTRY_ERROR                                                Handle        = 0x800F023A
+	SPAPI_E_REMOTE_REQUEST_UNSUPPORTED                                        Handle        = 0x800F023B
+	SPAPI_E_NOT_AN_INSTALLED_OEM_INF                                          Handle        = 0x800F023C
+	SPAPI_E_INF_IN_USE_BY_DEVICES                                             Handle        = 0x800F023D
+	SPAPI_E_DI_FUNCTION_OBSOLETE                                              Handle        = 0x800F023E
+	SPAPI_E_NO_AUTHENTICODE_CATALOG                                           Handle        = 0x800F023F
+	SPAPI_E_AUTHENTICODE_DISALLOWED                                           Handle        = 0x800F0240
+	SPAPI_E_AUTHENTICODE_TRUSTED_PUBLISHER                                    Handle        = 0x800F0241
+	SPAPI_E_AUTHENTICODE_TRUST_NOT_ESTABLISHED                                Handle        = 0x800F0242
+	SPAPI_E_AUTHENTICODE_PUBLISHER_NOT_TRUSTED                                Handle        = 0x800F0243
+	SPAPI_E_SIGNATURE_OSATTRIBUTE_MISMATCH                                    Handle        = 0x800F0244
+	SPAPI_E_ONLY_VALIDATE_VIA_AUTHENTICODE                                    Handle        = 0x800F0245
+	SPAPI_E_DEVICE_INSTALLER_NOT_READY                                        Handle        = 0x800F0246
+	SPAPI_E_DRIVER_STORE_ADD_FAILED                                           Handle        = 0x800F0247
+	SPAPI_E_DEVICE_INSTALL_BLOCKED                                            Handle        = 0x800F0248
+	SPAPI_E_DRIVER_INSTALL_BLOCKED                                            Handle        = 0x800F0249
+	SPAPI_E_WRONG_INF_TYPE                                                    Handle        = 0x800F024A
+	SPAPI_E_FILE_HASH_NOT_IN_CATALOG                                          Handle        = 0x800F024B
+	SPAPI_E_DRIVER_STORE_DELETE_FAILED                                        Handle        = 0x800F024C
+	SPAPI_E_UNRECOVERABLE_STACK_OVERFLOW                                      Handle        = 0x800F0300
+	SPAPI_E_ERROR_NOT_INSTALLED                                               Handle        = 0x800F1000
+	SCARD_S_SUCCESS                                                                         = S_OK
+	SCARD_F_INTERNAL_ERROR                                                    Handle        = 0x80100001
+	SCARD_E_CANCELLED                                                         Handle        = 0x80100002
+	SCARD_E_INVALID_HANDLE                                                    Handle        = 0x80100003
+	SCARD_E_INVALID_PARAMETER                                                 Handle        = 0x80100004
+	SCARD_E_INVALID_TARGET                                                    Handle        = 0x80100005
+	SCARD_E_NO_MEMORY                                                         Handle        = 0x80100006
+	SCARD_F_WAITED_TOO_LONG                                                   Handle        = 0x80100007
+	SCARD_E_INSUFFICIENT_BUFFER                                               Handle        = 0x80100008
+	SCARD_E_UNKNOWN_READER                                                    Handle        = 0x80100009
+	SCARD_E_TIMEOUT                                                           Handle        = 0x8010000A
+	SCARD_E_SHARING_VIOLATION                                                 Handle        = 0x8010000B
+	SCARD_E_NO_SMARTCARD                                                      Handle        = 0x8010000C
+	SCARD_E_UNKNOWN_CARD                                                      Handle        = 0x8010000D
+	SCARD_E_CANT_DISPOSE                                                      Handle        = 0x8010000E
+	SCARD_E_PROTO_MISMATCH                                                    Handle        = 0x8010000F
+	SCARD_E_NOT_READY                                                         Handle        = 0x80100010
+	SCARD_E_INVALID_VALUE                                                     Handle        = 0x80100011
+	SCARD_E_SYSTEM_CANCELLED                                                  Handle        = 0x80100012
+	SCARD_F_COMM_ERROR                                                        Handle        = 0x80100013
+	SCARD_F_UNKNOWN_ERROR                                                     Handle        = 0x80100014
+	SCARD_E_INVALID_ATR                                                       Handle        = 0x80100015
+	SCARD_E_NOT_TRANSACTED                                                    Handle        = 0x80100016
+	SCARD_E_READER_UNAVAILABLE                                                Handle        = 0x80100017
+	SCARD_P_SHUTDOWN                                                          Handle        = 0x80100018
+	SCARD_E_PCI_TOO_SMALL                                                     Handle        = 0x80100019
+	SCARD_E_READER_UNSUPPORTED                                                Handle        = 0x8010001A
+	SCARD_E_DUPLICATE_READER                                                  Handle        = 0x8010001B
+	SCARD_E_CARD_UNSUPPORTED                                                  Handle        = 0x8010001C
+	SCARD_E_NO_SERVICE                                                        Handle        = 0x8010001D
+	SCARD_E_SERVICE_STOPPED                                                   Handle        = 0x8010001E
+	SCARD_E_UNEXPECTED                                                        Handle        = 0x8010001F
+	SCARD_E_ICC_INSTALLATION                                                  Handle        = 0x80100020
+	SCARD_E_ICC_CREATEORDER                                                   Handle        = 0x80100021
+	SCARD_E_UNSUPPORTED_FEATURE                                               Handle        = 0x80100022
+	SCARD_E_DIR_NOT_FOUND                                                     Handle        = 0x80100023
+	SCARD_E_FILE_NOT_FOUND                                                    Handle        = 0x80100024
+	SCARD_E_NO_DIR                                                            Handle        = 0x80100025
+	SCARD_E_NO_FILE                                                           Handle        = 0x80100026
+	SCARD_E_NO_ACCESS                                                         Handle        = 0x80100027
+	SCARD_E_WRITE_TOO_MANY                                                    Handle        = 0x80100028
+	SCARD_E_BAD_SEEK                                                          Handle        = 0x80100029
+	SCARD_E_INVALID_CHV                                                       Handle        = 0x8010002A
+	SCARD_E_UNKNOWN_RES_MNG                                                   Handle        = 0x8010002B
+	SCARD_E_NO_SUCH_CERTIFICATE                                               Handle        = 0x8010002C
+	SCARD_E_CERTIFICATE_UNAVAILABLE                                           Handle        = 0x8010002D
+	SCARD_E_NO_READERS_AVAILABLE                                              Handle        = 0x8010002E
+	SCARD_E_COMM_DATA_LOST                                                    Handle        = 0x8010002F
+	SCARD_E_NO_KEY_CONTAINER                                                  Handle        = 0x80100030
+	SCARD_E_SERVER_TOO_BUSY                                                   Handle        = 0x80100031
+	SCARD_E_PIN_CACHE_EXPIRED                                                 Handle        = 0x80100032
+	SCARD_E_NO_PIN_CACHE                                                      Handle        = 0x80100033
+	SCARD_E_READ_ONLY_CARD                                                    Handle        = 0x80100034
+	SCARD_W_UNSUPPORTED_CARD                                                  Handle        = 0x80100065
+	SCARD_W_UNRESPONSIVE_CARD                                                 Handle        = 0x80100066
+	SCARD_W_UNPOWERED_CARD                                                    Handle        = 0x80100067
+	SCARD_W_RESET_CARD                                                        Handle        = 0x80100068
+	SCARD_W_REMOVED_CARD                                                      Handle        = 0x80100069
+	SCARD_W_SECURITY_VIOLATION                                                Handle        = 0x8010006A
+	SCARD_W_WRONG_CHV                                                         Handle        = 0x8010006B
+	SCARD_W_CHV_BLOCKED                                                       Handle        = 0x8010006C
+	SCARD_W_EOF                                                               Handle        = 0x8010006D
+	SCARD_W_CANCELLED_BY_USER                                                 Handle        = 0x8010006E
+	SCARD_W_CARD_NOT_AUTHENTICATED                                            Handle        = 0x8010006F
+	SCARD_W_CACHE_ITEM_NOT_FOUND                                              Handle        = 0x80100070
+	SCARD_W_CACHE_ITEM_STALE                                                  Handle        = 0x80100071
+	SCARD_W_CACHE_ITEM_TOO_BIG                                                Handle        = 0x80100072
+	COMADMIN_E_OBJECTERRORS                                                   Handle        = 0x80110401
+	COMADMIN_E_OBJECTINVALID                                                  Handle        = 0x80110402
+	COMADMIN_E_KEYMISSING                                                     Handle        = 0x80110403
+	COMADMIN_E_ALREADYINSTALLED                                               Handle        = 0x80110404
+	COMADMIN_E_APP_FILE_WRITEFAIL                                             Handle        = 0x80110407
+	COMADMIN_E_APP_FILE_READFAIL                                              Handle        = 0x80110408
+	COMADMIN_E_APP_FILE_VERSION                                               Handle        = 0x80110409
+	COMADMIN_E_BADPATH                                                        Handle        = 0x8011040A
+	COMADMIN_E_APPLICATIONEXISTS                                              Handle        = 0x8011040B
+	COMADMIN_E_ROLEEXISTS                                                     Handle        = 0x8011040C
+	COMADMIN_E_CANTCOPYFILE                                                   Handle        = 0x8011040D
+	COMADMIN_E_NOUSER                                                         Handle        = 0x8011040F
+	COMADMIN_E_INVALIDUSERIDS                                                 Handle        = 0x80110410
+	COMADMIN_E_NOREGISTRYCLSID                                                Handle        = 0x80110411
+	COMADMIN_E_BADREGISTRYPROGID                                              Handle        = 0x80110412
+	COMADMIN_E_AUTHENTICATIONLEVEL                                            Handle        = 0x80110413
+	COMADMIN_E_USERPASSWDNOTVALID                                             Handle        = 0x80110414
+	COMADMIN_E_CLSIDORIIDMISMATCH                                             Handle        = 0x80110418
+	COMADMIN_E_REMOTEINTERFACE                                                Handle        = 0x80110419
+	COMADMIN_E_DLLREGISTERSERVER                                              Handle        = 0x8011041A
+	COMADMIN_E_NOSERVERSHARE                                                  Handle        = 0x8011041B
+	COMADMIN_E_DLLLOADFAILED                                                  Handle        = 0x8011041D
+	COMADMIN_E_BADREGISTRYLIBID                                               Handle        = 0x8011041E
+	COMADMIN_E_APPDIRNOTFOUND                                                 Handle        = 0x8011041F
+	COMADMIN_E_REGISTRARFAILED                                                Handle        = 0x80110423
+	COMADMIN_E_COMPFILE_DOESNOTEXIST                                          Handle        = 0x80110424
+	COMADMIN_E_COMPFILE_LOADDLLFAIL                                           Handle        = 0x80110425
+	COMADMIN_E_COMPFILE_GETCLASSOBJ                                           Handle        = 0x80110426
+	COMADMIN_E_COMPFILE_CLASSNOTAVAIL                                         Handle        = 0x80110427
+	COMADMIN_E_COMPFILE_BADTLB                                                Handle        = 0x80110428
+	COMADMIN_E_COMPFILE_NOTINSTALLABLE                                        Handle        = 0x80110429
+	COMADMIN_E_NOTCHANGEABLE                                                  Handle        = 0x8011042A
+	COMADMIN_E_NOTDELETEABLE                                                  Handle        = 0x8011042B
+	COMADMIN_E_SESSION                                                        Handle        = 0x8011042C
+	COMADMIN_E_COMP_MOVE_LOCKED                                               Handle        = 0x8011042D
+	COMADMIN_E_COMP_MOVE_BAD_DEST                                             Handle        = 0x8011042E
+	COMADMIN_E_REGISTERTLB                                                    Handle        = 0x80110430
+	COMADMIN_E_SYSTEMAPP                                                      Handle        = 0x80110433
+	COMADMIN_E_COMPFILE_NOREGISTRAR                                           Handle        = 0x80110434
+	COMADMIN_E_COREQCOMPINSTALLED                                             Handle        = 0x80110435
+	COMADMIN_E_SERVICENOTINSTALLED                                            Handle        = 0x80110436
+	COMADMIN_E_PROPERTYSAVEFAILED                                             Handle        = 0x80110437
+	COMADMIN_E_OBJECTEXISTS                                                   Handle        = 0x80110438
+	COMADMIN_E_COMPONENTEXISTS                                                Handle        = 0x80110439
+	COMADMIN_E_REGFILE_CORRUPT                                                Handle        = 0x8011043B
+	COMADMIN_E_PROPERTY_OVERFLOW                                              Handle        = 0x8011043C
+	COMADMIN_E_NOTINREGISTRY                                                  Handle        = 0x8011043E
+	COMADMIN_E_OBJECTNOTPOOLABLE                                              Handle        = 0x8011043F
+	COMADMIN_E_APPLID_MATCHES_CLSID                                           Handle        = 0x80110446
+	COMADMIN_E_ROLE_DOES_NOT_EXIST                                            Handle        = 0x80110447
+	COMADMIN_E_START_APP_NEEDS_COMPONENTS                                     Handle        = 0x80110448
+	COMADMIN_E_REQUIRES_DIFFERENT_PLATFORM                                    Handle        = 0x80110449
+	COMADMIN_E_CAN_NOT_EXPORT_APP_PROXY                                       Handle        = 0x8011044A
+	COMADMIN_E_CAN_NOT_START_APP                                              Handle        = 0x8011044B
+	COMADMIN_E_CAN_NOT_EXPORT_SYS_APP                                         Handle        = 0x8011044C
+	COMADMIN_E_CANT_SUBSCRIBE_TO_COMPONENT                                    Handle        = 0x8011044D
+	COMADMIN_E_EVENTCLASS_CANT_BE_SUBSCRIBER                                  Handle        = 0x8011044E
+	COMADMIN_E_LIB_APP_PROXY_INCOMPATIBLE                                     Handle        = 0x8011044F
+	COMADMIN_E_BASE_PARTITION_ONLY                                            Handle        = 0x80110450
+	COMADMIN_E_START_APP_DISABLED                                             Handle        = 0x80110451
+	COMADMIN_E_CAT_DUPLICATE_PARTITION_NAME                                   Handle        = 0x80110457
+	COMADMIN_E_CAT_INVALID_PARTITION_NAME                                     Handle        = 0x80110458
+	COMADMIN_E_CAT_PARTITION_IN_USE                                           Handle        = 0x80110459
+	COMADMIN_E_FILE_PARTITION_DUPLICATE_FILES                                 Handle        = 0x8011045A
+	COMADMIN_E_CAT_IMPORTED_COMPONENTS_NOT_ALLOWED                            Handle        = 0x8011045B
+	COMADMIN_E_AMBIGUOUS_APPLICATION_NAME                                     Handle        = 0x8011045C
+	COMADMIN_E_AMBIGUOUS_PARTITION_NAME                                       Handle        = 0x8011045D
+	COMADMIN_E_REGDB_NOTINITIALIZED                                           Handle        = 0x80110472
+	COMADMIN_E_REGDB_NOTOPEN                                                  Handle        = 0x80110473
+	COMADMIN_E_REGDB_SYSTEMERR                                                Handle        = 0x80110474
+	COMADMIN_E_REGDB_ALREADYRUNNING                                           Handle        = 0x80110475
+	COMADMIN_E_MIG_VERSIONNOTSUPPORTED                                        Handle        = 0x80110480
+	COMADMIN_E_MIG_SCHEMANOTFOUND                                             Handle        = 0x80110481
+	COMADMIN_E_CAT_BITNESSMISMATCH                                            Handle        = 0x80110482
+	COMADMIN_E_CAT_UNACCEPTABLEBITNESS                                        Handle        = 0x80110483
+	COMADMIN_E_CAT_WRONGAPPBITNESS                                            Handle        = 0x80110484
+	COMADMIN_E_CAT_PAUSE_RESUME_NOT_SUPPORTED                                 Handle        = 0x80110485
+	COMADMIN_E_CAT_SERVERFAULT                                                Handle        = 0x80110486
+	COMQC_E_APPLICATION_NOT_QUEUED                                            Handle        = 0x80110600
+	COMQC_E_NO_QUEUEABLE_INTERFACES                                           Handle        = 0x80110601
+	COMQC_E_QUEUING_SERVICE_NOT_AVAILABLE                                     Handle        = 0x80110602
+	COMQC_E_NO_IPERSISTSTREAM                                                 Handle        = 0x80110603
+	COMQC_E_BAD_MESSAGE                                                       Handle        = 0x80110604
+	COMQC_E_UNAUTHENTICATED                                                   Handle        = 0x80110605
+	COMQC_E_UNTRUSTED_ENQUEUER                                                Handle        = 0x80110606
+	MSDTC_E_DUPLICATE_RESOURCE                                                Handle        = 0x80110701
+	COMADMIN_E_OBJECT_PARENT_MISSING                                          Handle        = 0x80110808
+	COMADMIN_E_OBJECT_DOES_NOT_EXIST                                          Handle        = 0x80110809
+	COMADMIN_E_APP_NOT_RUNNING                                                Handle        = 0x8011080A
+	COMADMIN_E_INVALID_PARTITION                                              Handle        = 0x8011080B
+	COMADMIN_E_SVCAPP_NOT_POOLABLE_OR_RECYCLABLE                              Handle        = 0x8011080D
+	COMADMIN_E_USER_IN_SET                                                    Handle        = 0x8011080E
+	COMADMIN_E_CANTRECYCLELIBRARYAPPS                                         Handle        = 0x8011080F
+	COMADMIN_E_CANTRECYCLESERVICEAPPS                                         Handle        = 0x80110811
+	COMADMIN_E_PROCESSALREADYRECYCLED                                         Handle        = 0x80110812
+	COMADMIN_E_PAUSEDPROCESSMAYNOTBERECYCLED                                  Handle        = 0x80110813
+	COMADMIN_E_CANTMAKEINPROCSERVICE                                          Handle        = 0x80110814
+	COMADMIN_E_PROGIDINUSEBYCLSID                                             Handle        = 0x80110815
+	COMADMIN_E_DEFAULT_PARTITION_NOT_IN_SET                                   Handle        = 0x80110816
+	COMADMIN_E_RECYCLEDPROCESSMAYNOTBEPAUSED                                  Handle        = 0x80110817
+	COMADMIN_E_PARTITION_ACCESSDENIED                                         Handle        = 0x80110818
+	COMADMIN_E_PARTITION_MSI_ONLY                                             Handle        = 0x80110819
+	COMADMIN_E_LEGACYCOMPS_NOT_ALLOWED_IN_1_0_FORMAT                          Handle        = 0x8011081A
+	COMADMIN_E_LEGACYCOMPS_NOT_ALLOWED_IN_NONBASE_PARTITIONS                  Handle        = 0x8011081B
+	COMADMIN_E_COMP_MOVE_SOURCE                                               Handle        = 0x8011081C
+	COMADMIN_E_COMP_MOVE_DEST                                                 Handle        = 0x8011081D
+	COMADMIN_E_COMP_MOVE_PRIVATE                                              Handle        = 0x8011081E
+	COMADMIN_E_BASEPARTITION_REQUIRED_IN_SET                                  Handle        = 0x8011081F
+	COMADMIN_E_CANNOT_ALIAS_EVENTCLASS                                        Handle        = 0x80110820
+	COMADMIN_E_PRIVATE_ACCESSDENIED                                           Handle        = 0x80110821
+	COMADMIN_E_SAFERINVALID                                                   Handle        = 0x80110822
+	COMADMIN_E_REGISTRY_ACCESSDENIED                                          Handle        = 0x80110823
+	COMADMIN_E_PARTITIONS_DISABLED                                            Handle        = 0x80110824
+	WER_S_REPORT_DEBUG                                                        Handle        = 0x001B0000
+	WER_S_REPORT_UPLOADED                                                     Handle        = 0x001B0001
+	WER_S_REPORT_QUEUED                                                       Handle        = 0x001B0002
+	WER_S_DISABLED                                                            Handle        = 0x001B0003
+	WER_S_SUSPENDED_UPLOAD                                                    Handle        = 0x001B0004
+	WER_S_DISABLED_QUEUE                                                      Handle        = 0x001B0005
+	WER_S_DISABLED_ARCHIVE                                                    Handle        = 0x001B0006
+	WER_S_REPORT_ASYNC                                                        Handle        = 0x001B0007
+	WER_S_IGNORE_ASSERT_INSTANCE                                              Handle        = 0x001B0008
+	WER_S_IGNORE_ALL_ASSERTS                                                  Handle        = 0x001B0009
+	WER_S_ASSERT_CONTINUE                                                     Handle        = 0x001B000A
+	WER_S_THROTTLED                                                           Handle        = 0x001B000B
+	WER_S_REPORT_UPLOADED_CAB                                                 Handle        = 0x001B000C
+	WER_E_CRASH_FAILURE                                                       Handle        = 0x801B8000
+	WER_E_CANCELED                                                            Handle        = 0x801B8001
+	WER_E_NETWORK_FAILURE                                                     Handle        = 0x801B8002
+	WER_E_NOT_INITIALIZED                                                     Handle        = 0x801B8003
+	WER_E_ALREADY_REPORTING                                                   Handle        = 0x801B8004
+	WER_E_DUMP_THROTTLED                                                      Handle        = 0x801B8005
+	WER_E_INSUFFICIENT_CONSENT                                                Handle        = 0x801B8006
+	WER_E_TOO_HEAVY                                                           Handle        = 0x801B8007
+	ERROR_FLT_IO_COMPLETE                                                     Handle        = 0x001F0001
+	ERROR_FLT_NO_HANDLER_DEFINED                                              Handle        = 0x801F0001
+	ERROR_FLT_CONTEXT_ALREADY_DEFINED                                         Handle        = 0x801F0002
+	ERROR_FLT_INVALID_ASYNCHRONOUS_REQUEST                                    Handle        = 0x801F0003
+	ERROR_FLT_DISALLOW_FAST_IO                                                Handle        = 0x801F0004
+	ERROR_FLT_INVALID_NAME_REQUEST                                            Handle        = 0x801F0005
+	ERROR_FLT_NOT_SAFE_TO_POST_OPERATION                                      Handle        = 0x801F0006
+	ERROR_FLT_NOT_INITIALIZED                                                 Handle        = 0x801F0007
+	ERROR_FLT_FILTER_NOT_READY                                                Handle        = 0x801F0008
+	ERROR_FLT_POST_OPERATION_CLEANUP                                          Handle        = 0x801F0009
+	ERROR_FLT_INTERNAL_ERROR                                                  Handle        = 0x801F000A
+	ERROR_FLT_DELETING_OBJECT                                                 Handle        = 0x801F000B
+	ERROR_FLT_MUST_BE_NONPAGED_POOL                                           Handle        = 0x801F000C
+	ERROR_FLT_DUPLICATE_ENTRY                                                 Handle        = 0x801F000D
+	ERROR_FLT_CBDQ_DISABLED                                                   Handle        = 0x801F000E
+	ERROR_FLT_DO_NOT_ATTACH                                                   Handle        = 0x801F000F
+	ERROR_FLT_DO_NOT_DETACH                                                   Handle        = 0x801F0010
+	ERROR_FLT_INSTANCE_ALTITUDE_COLLISION                                     Handle        = 0x801F0011
+	ERROR_FLT_INSTANCE_NAME_COLLISION                                         Handle        = 0x801F0012
+	ERROR_FLT_FILTER_NOT_FOUND                                                Handle        = 0x801F0013
+	ERROR_FLT_VOLUME_NOT_FOUND                                                Handle        = 0x801F0014
+	ERROR_FLT_INSTANCE_NOT_FOUND                                              Handle        = 0x801F0015
+	ERROR_FLT_CONTEXT_ALLOCATION_NOT_FOUND                                    Handle        = 0x801F0016
+	ERROR_FLT_INVALID_CONTEXT_REGISTRATION                                    Handle        = 0x801F0017
+	ERROR_FLT_NAME_CACHE_MISS                                                 Handle        = 0x801F0018
+	ERROR_FLT_NO_DEVICE_OBJECT                                                Handle        = 0x801F0019
+	ERROR_FLT_VOLUME_ALREADY_MOUNTED                                          Handle        = 0x801F001A
+	ERROR_FLT_ALREADY_ENLISTED                                                Handle        = 0x801F001B
+	ERROR_FLT_CONTEXT_ALREADY_LINKED                                          Handle        = 0x801F001C
+	ERROR_FLT_NO_WAITER_FOR_REPLY                                             Handle        = 0x801F0020
+	ERROR_FLT_REGISTRATION_BUSY                                               Handle        = 0x801F0023
+	ERROR_HUNG_DISPLAY_DRIVER_THREAD                                          Handle        = 0x80260001
+	DWM_E_COMPOSITIONDISABLED                                                 Handle        = 0x80263001
+	DWM_E_REMOTING_NOT_SUPPORTED                                              Handle        = 0x80263002
+	DWM_E_NO_REDIRECTION_SURFACE_AVAILABLE                                    Handle        = 0x80263003
+	DWM_E_NOT_QUEUING_PRESENTS                                                Handle        = 0x80263004
+	DWM_E_ADAPTER_NOT_FOUND                                                   Handle        = 0x80263005
+	DWM_S_GDI_REDIRECTION_SURFACE                                             Handle        = 0x00263005
+	DWM_E_TEXTURE_TOO_LARGE                                                   Handle        = 0x80263007
+	DWM_S_GDI_REDIRECTION_SURFACE_BLT_VIA_GDI                                 Handle        = 0x00263008
+	ERROR_MONITOR_NO_DESCRIPTOR                                               Handle        = 0x00261001
+	ERROR_MONITOR_UNKNOWN_DESCRIPTOR_FORMAT                                   Handle        = 0x00261002
+	ERROR_MONITOR_INVALID_DESCRIPTOR_CHECKSUM                                 Handle        = 0xC0261003
+	ERROR_MONITOR_INVALID_STANDARD_TIMING_BLOCK                               Handle        = 0xC0261004
+	ERROR_MONITOR_WMI_DATABLOCK_REGISTRATION_FAILED                           Handle        = 0xC0261005
+	ERROR_MONITOR_INVALID_SERIAL_NUMBER_MONDSC_BLOCK                          Handle        = 0xC0261006
+	ERROR_MONITOR_INVALID_USER_FRIENDLY_MONDSC_BLOCK                          Handle        = 0xC0261007
+	ERROR_MONITOR_NO_MORE_DESCRIPTOR_DATA                                     Handle        = 0xC0261008
+	ERROR_MONITOR_INVALID_DETAILED_TIMING_BLOCK                               Handle        = 0xC0261009
+	ERROR_MONITOR_INVALID_MANUFACTURE_DATE                                    Handle        = 0xC026100A
+	ERROR_GRAPHICS_NOT_EXCLUSIVE_MODE_OWNER                                   Handle        = 0xC0262000
+	ERROR_GRAPHICS_INSUFFICIENT_DMA_BUFFER                                    Handle        = 0xC0262001
+	ERROR_GRAPHICS_INVALID_DISPLAY_ADAPTER                                    Handle        = 0xC0262002
+	ERROR_GRAPHICS_ADAPTER_WAS_RESET                                          Handle        = 0xC0262003
+	ERROR_GRAPHICS_INVALID_DRIVER_MODEL                                       Handle        = 0xC0262004
+	ERROR_GRAPHICS_PRESENT_MODE_CHANGED                                       Handle        = 0xC0262005
+	ERROR_GRAPHICS_PRESENT_OCCLUDED                                           Handle        = 0xC0262006
+	ERROR_GRAPHICS_PRESENT_DENIED                                             Handle        = 0xC0262007
+	ERROR_GRAPHICS_CANNOTCOLORCONVERT                                         Handle        = 0xC0262008
+	ERROR_GRAPHICS_DRIVER_MISMATCH                                            Handle        = 0xC0262009
+	ERROR_GRAPHICS_PARTIAL_DATA_POPULATED                                     Handle        = 0x4026200A
+	ERROR_GRAPHICS_PRESENT_REDIRECTION_DISABLED                               Handle        = 0xC026200B
+	ERROR_GRAPHICS_PRESENT_UNOCCLUDED                                         Handle        = 0xC026200C
+	ERROR_GRAPHICS_WINDOWDC_NOT_AVAILABLE                                     Handle        = 0xC026200D
+	ERROR_GRAPHICS_WINDOWLESS_PRESENT_DISABLED                                Handle        = 0xC026200E
+	ERROR_GRAPHICS_PRESENT_INVALID_WINDOW                                     Handle        = 0xC026200F
+	ERROR_GRAPHICS_PRESENT_BUFFER_NOT_BOUND                                   Handle        = 0xC0262010
+	ERROR_GRAPHICS_VAIL_STATE_CHANGED                                         Handle        = 0xC0262011
+	ERROR_GRAPHICS_NO_VIDEO_MEMORY                                            Handle        = 0xC0262100
+	ERROR_GRAPHICS_CANT_LOCK_MEMORY                                           Handle        = 0xC0262101
+	ERROR_GRAPHICS_ALLOCATION_BUSY                                            Handle        = 0xC0262102
+	ERROR_GRAPHICS_TOO_MANY_REFERENCES                                        Handle        = 0xC0262103
+	ERROR_GRAPHICS_TRY_AGAIN_LATER                                            Handle        = 0xC0262104
+	ERROR_GRAPHICS_TRY_AGAIN_NOW                                              Handle        = 0xC0262105
+	ERROR_GRAPHICS_ALLOCATION_INVALID                                         Handle        = 0xC0262106
+	ERROR_GRAPHICS_UNSWIZZLING_APERTURE_UNAVAILABLE                           Handle        = 0xC0262107
+	ERROR_GRAPHICS_UNSWIZZLING_APERTURE_UNSUPPORTED                           Handle        = 0xC0262108
+	ERROR_GRAPHICS_CANT_EVICT_PINNED_ALLOCATION                               Handle        = 0xC0262109
+	ERROR_GRAPHICS_INVALID_ALLOCATION_USAGE                                   Handle        = 0xC0262110
+	ERROR_GRAPHICS_CANT_RENDER_LOCKED_ALLOCATION                              Handle        = 0xC0262111
+	ERROR_GRAPHICS_ALLOCATION_CLOSED                                          Handle        = 0xC0262112
+	ERROR_GRAPHICS_INVALID_ALLOCATION_INSTANCE                                Handle        = 0xC0262113
+	ERROR_GRAPHICS_INVALID_ALLOCATION_HANDLE                                  Handle        = 0xC0262114
+	ERROR_GRAPHICS_WRONG_ALLOCATION_DEVICE                                    Handle        = 0xC0262115
+	ERROR_GRAPHICS_ALLOCATION_CONTENT_LOST                                    Handle        = 0xC0262116
+	ERROR_GRAPHICS_GPU_EXCEPTION_ON_DEVICE                                    Handle        = 0xC0262200
+	ERROR_GRAPHICS_SKIP_ALLOCATION_PREPARATION                                Handle        = 0x40262201
+	ERROR_GRAPHICS_INVALID_VIDPN_TOPOLOGY                                     Handle        = 0xC0262300
+	ERROR_GRAPHICS_VIDPN_TOPOLOGY_NOT_SUPPORTED                               Handle        = 0xC0262301
+	ERROR_GRAPHICS_VIDPN_TOPOLOGY_CURRENTLY_NOT_SUPPORTED                     Handle        = 0xC0262302
+	ERROR_GRAPHICS_INVALID_VIDPN                                              Handle        = 0xC0262303
+	ERROR_GRAPHICS_INVALID_VIDEO_PRESENT_SOURCE                               Handle        = 0xC0262304
+	ERROR_GRAPHICS_INVALID_VIDEO_PRESENT_TARGET                               Handle        = 0xC0262305
+	ERROR_GRAPHICS_VIDPN_MODALITY_NOT_SUPPORTED                               Handle        = 0xC0262306
+	ERROR_GRAPHICS_MODE_NOT_PINNED                                            Handle        = 0x00262307
+	ERROR_GRAPHICS_INVALID_VIDPN_SOURCEMODESET                                Handle        = 0xC0262308
+	ERROR_GRAPHICS_INVALID_VIDPN_TARGETMODESET                                Handle        = 0xC0262309
+	ERROR_GRAPHICS_INVALID_FREQUENCY                                          Handle        = 0xC026230A
+	ERROR_GRAPHICS_INVALID_ACTIVE_REGION                                      Handle        = 0xC026230B
+	ERROR_GRAPHICS_INVALID_TOTAL_REGION                                       Handle        = 0xC026230C
+	ERROR_GRAPHICS_INVALID_VIDEO_PRESENT_SOURCE_MODE                          Handle        = 0xC0262310
+	ERROR_GRAPHICS_INVALID_VIDEO_PRESENT_TARGET_MODE                          Handle        = 0xC0262311
+	ERROR_GRAPHICS_PINNED_MODE_MUST_REMAIN_IN_SET                             Handle        = 0xC0262312
+	ERROR_GRAPHICS_PATH_ALREADY_IN_TOPOLOGY                                   Handle        = 0xC0262313
+	ERROR_GRAPHICS_MODE_ALREADY_IN_MODESET                                    Handle        = 0xC0262314
+	ERROR_GRAPHICS_INVALID_VIDEOPRESENTSOURCESET                              Handle        = 0xC0262315
+	ERROR_GRAPHICS_INVALID_VIDEOPRESENTTARGETSET                              Handle        = 0xC0262316
+	ERROR_GRAPHICS_SOURCE_ALREADY_IN_SET                                      Handle        = 0xC0262317
+	ERROR_GRAPHICS_TARGET_ALREADY_IN_SET                                      Handle        = 0xC0262318
+	ERROR_GRAPHICS_INVALID_VIDPN_PRESENT_PATH                                 Handle        = 0xC0262319
+	ERROR_GRAPHICS_NO_RECOMMENDED_VIDPN_TOPOLOGY                              Handle        = 0xC026231A
+	ERROR_GRAPHICS_INVALID_MONITOR_FREQUENCYRANGESET                          Handle        = 0xC026231B
+	ERROR_GRAPHICS_INVALID_MONITOR_FREQUENCYRANGE                             Handle        = 0xC026231C
+	ERROR_GRAPHICS_FREQUENCYRANGE_NOT_IN_SET                                  Handle        = 0xC026231D
+	ERROR_GRAPHICS_NO_PREFERRED_MODE                                          Handle        = 0x0026231E
+	ERROR_GRAPHICS_FREQUENCYRANGE_ALREADY_IN_SET                              Handle        = 0xC026231F
+	ERROR_GRAPHICS_STALE_MODESET                                              Handle        = 0xC0262320
+	ERROR_GRAPHICS_INVALID_MONITOR_SOURCEMODESET                              Handle        = 0xC0262321
+	ERROR_GRAPHICS_INVALID_MONITOR_SOURCE_MODE                                Handle        = 0xC0262322
+	ERROR_GRAPHICS_NO_RECOMMENDED_FUNCTIONAL_VIDPN                            Handle        = 0xC0262323
+	ERROR_GRAPHICS_MODE_ID_MUST_BE_UNIQUE                                     Handle        = 0xC0262324
+	ERROR_GRAPHICS_EMPTY_ADAPTER_MONITOR_MODE_SUPPORT_INTERSECTION            Handle        = 0xC0262325
+	ERROR_GRAPHICS_VIDEO_PRESENT_TARGETS_LESS_THAN_SOURCES                    Handle        = 0xC0262326
+	ERROR_GRAPHICS_PATH_NOT_IN_TOPOLOGY                                       Handle        = 0xC0262327
+	ERROR_GRAPHICS_ADAPTER_MUST_HAVE_AT_LEAST_ONE_SOURCE                      Handle        = 0xC0262328
+	ERROR_GRAPHICS_ADAPTER_MUST_HAVE_AT_LEAST_ONE_TARGET                      Handle        = 0xC0262329
+	ERROR_GRAPHICS_INVALID_MONITORDESCRIPTORSET                               Handle        = 0xC026232A
+	ERROR_GRAPHICS_INVALID_MONITORDESCRIPTOR                                  Handle        = 0xC026232B
+	ERROR_GRAPHICS_MONITORDESCRIPTOR_NOT_IN_SET                               Handle        = 0xC026232C
+	ERROR_GRAPHICS_MONITORDESCRIPTOR_ALREADY_IN_SET                           Handle        = 0xC026232D
+	ERROR_GRAPHICS_MONITORDESCRIPTOR_ID_MUST_BE_UNIQUE                        Handle        = 0xC026232E
+	ERROR_GRAPHICS_INVALID_VIDPN_TARGET_SUBSET_TYPE                           Handle        = 0xC026232F
+	ERROR_GRAPHICS_RESOURCES_NOT_RELATED                                      Handle        = 0xC0262330
+	ERROR_GRAPHICS_SOURCE_ID_MUST_BE_UNIQUE                                   Handle        = 0xC0262331
+	ERROR_GRAPHICS_TARGET_ID_MUST_BE_UNIQUE                                   Handle        = 0xC0262332
+	ERROR_GRAPHICS_NO_AVAILABLE_VIDPN_TARGET                                  Handle        = 0xC0262333
+	ERROR_GRAPHICS_MONITOR_COULD_NOT_BE_ASSOCIATED_WITH_ADAPTER               Handle        = 0xC0262334
+	ERROR_GRAPHICS_NO_VIDPNMGR                                                Handle        = 0xC0262335
+	ERROR_GRAPHICS_NO_ACTIVE_VIDPN                                            Handle        = 0xC0262336
+	ERROR_GRAPHICS_STALE_VIDPN_TOPOLOGY                                       Handle        = 0xC0262337
+	ERROR_GRAPHICS_MONITOR_NOT_CONNECTED                                      Handle        = 0xC0262338
+	ERROR_GRAPHICS_SOURCE_NOT_IN_TOPOLOGY                                     Handle        = 0xC0262339
+	ERROR_GRAPHICS_INVALID_PRIMARYSURFACE_SIZE                                Handle        = 0xC026233A
+	ERROR_GRAPHICS_INVALID_VISIBLEREGION_SIZE                                 Handle        = 0xC026233B
+	ERROR_GRAPHICS_INVALID_STRIDE                                             Handle        = 0xC026233C
+	ERROR_GRAPHICS_INVALID_PIXELFORMAT                                        Handle        = 0xC026233D
+	ERROR_GRAPHICS_INVALID_COLORBASIS                                         Handle        = 0xC026233E
+	ERROR_GRAPHICS_INVALID_PIXELVALUEACCESSMODE                               Handle        = 0xC026233F
+	ERROR_GRAPHICS_TARGET_NOT_IN_TOPOLOGY                                     Handle        = 0xC0262340
+	ERROR_GRAPHICS_NO_DISPLAY_MODE_MANAGEMENT_SUPPORT                         Handle        = 0xC0262341
+	ERROR_GRAPHICS_VIDPN_SOURCE_IN_USE                                        Handle        = 0xC0262342
+	ERROR_GRAPHICS_CANT_ACCESS_ACTIVE_VIDPN                                   Handle        = 0xC0262343
+	ERROR_GRAPHICS_INVALID_PATH_IMPORTANCE_ORDINAL                            Handle        = 0xC0262344
+	ERROR_GRAPHICS_INVALID_PATH_CONTENT_GEOMETRY_TRANSFORMATION               Handle        = 0xC0262345
+	ERROR_GRAPHICS_PATH_CONTENT_GEOMETRY_TRANSFORMATION_NOT_SUPPORTED         Handle        = 0xC0262346
+	ERROR_GRAPHICS_INVALID_GAMMA_RAMP                                         Handle        = 0xC0262347
+	ERROR_GRAPHICS_GAMMA_RAMP_NOT_SUPPORTED                                   Handle        = 0xC0262348
+	ERROR_GRAPHICS_MULTISAMPLING_NOT_SUPPORTED                                Handle        = 0xC0262349
+	ERROR_GRAPHICS_MODE_NOT_IN_MODESET                                        Handle        = 0xC026234A
+	ERROR_GRAPHICS_DATASET_IS_EMPTY                                           Handle        = 0x0026234B
+	ERROR_GRAPHICS_NO_MORE_ELEMENTS_IN_DATASET                                Handle        = 0x0026234C
+	ERROR_GRAPHICS_INVALID_VIDPN_TOPOLOGY_RECOMMENDATION_REASON               Handle        = 0xC026234D
+	ERROR_GRAPHICS_INVALID_PATH_CONTENT_TYPE                                  Handle        = 0xC026234E
+	ERROR_GRAPHICS_INVALID_COPYPROTECTION_TYPE                                Handle        = 0xC026234F
+	ERROR_GRAPHICS_UNASSIGNED_MODESET_ALREADY_EXISTS                          Handle        = 0xC0262350
+	ERROR_GRAPHICS_PATH_CONTENT_GEOMETRY_TRANSFORMATION_NOT_PINNED            Handle        = 0x00262351
+	ERROR_GRAPHICS_INVALID_SCANLINE_ORDERING                                  Handle        = 0xC0262352
+	ERROR_GRAPHICS_TOPOLOGY_CHANGES_NOT_ALLOWED                               Handle        = 0xC0262353
+	ERROR_GRAPHICS_NO_AVAILABLE_IMPORTANCE_ORDINALS                           Handle        = 0xC0262354
+	ERROR_GRAPHICS_INCOMPATIBLE_PRIVATE_FORMAT                                Handle        = 0xC0262355
+	ERROR_GRAPHICS_INVALID_MODE_PRUNING_ALGORITHM                             Handle        = 0xC0262356
+	ERROR_GRAPHICS_INVALID_MONITOR_CAPABILITY_ORIGIN                          Handle        = 0xC0262357
+	ERROR_GRAPHICS_INVALID_MONITOR_FREQUENCYRANGE_CONSTRAINT                  Handle        = 0xC0262358
+	ERROR_GRAPHICS_MAX_NUM_PATHS_REACHED                                      Handle        = 0xC0262359
+	ERROR_GRAPHICS_CANCEL_VIDPN_TOPOLOGY_AUGMENTATION                         Handle        = 0xC026235A
+	ERROR_GRAPHICS_INVALID_CLIENT_TYPE                                        Handle        = 0xC026235B
+	ERROR_GRAPHICS_CLIENTVIDPN_NOT_SET                                        Handle        = 0xC026235C
+	ERROR_GRAPHICS_SPECIFIED_CHILD_ALREADY_CONNECTED                          Handle        = 0xC0262400
+	ERROR_GRAPHICS_CHILD_DESCRIPTOR_NOT_SUPPORTED                             Handle        = 0xC0262401
+	ERROR_GRAPHICS_UNKNOWN_CHILD_STATUS                                       Handle        = 0x4026242F
+	ERROR_GRAPHICS_NOT_A_LINKED_ADAPTER                                       Handle        = 0xC0262430
+	ERROR_GRAPHICS_LEADLINK_NOT_ENUMERATED                                    Handle        = 0xC0262431
+	ERROR_GRAPHICS_CHAINLINKS_NOT_ENUMERATED                                  Handle        = 0xC0262432
+	ERROR_GRAPHICS_ADAPTER_CHAIN_NOT_READY                                    Handle        = 0xC0262433
+	ERROR_GRAPHICS_CHAINLINKS_NOT_STARTED                                     Handle        = 0xC0262434
+	ERROR_GRAPHICS_CHAINLINKS_NOT_POWERED_ON                                  Handle        = 0xC0262435
+	ERROR_GRAPHICS_INCONSISTENT_DEVICE_LINK_STATE                             Handle        = 0xC0262436
+	ERROR_GRAPHICS_LEADLINK_START_DEFERRED                                    Handle        = 0x40262437
+	ERROR_GRAPHICS_NOT_POST_DEVICE_DRIVER                                     Handle        = 0xC0262438
+	ERROR_GRAPHICS_POLLING_TOO_FREQUENTLY                                     Handle        = 0x40262439
+	ERROR_GRAPHICS_START_DEFERRED                                             Handle        = 0x4026243A
+	ERROR_GRAPHICS_ADAPTER_ACCESS_NOT_EXCLUDED                                Handle        = 0xC026243B
+	ERROR_GRAPHICS_DEPENDABLE_CHILD_STATUS                                    Handle        = 0x4026243C
+	ERROR_GRAPHICS_OPM_NOT_SUPPORTED                                          Handle        = 0xC0262500
+	ERROR_GRAPHICS_COPP_NOT_SUPPORTED                                         Handle        = 0xC0262501
+	ERROR_GRAPHICS_UAB_NOT_SUPPORTED                                          Handle        = 0xC0262502
+	ERROR_GRAPHICS_OPM_INVALID_ENCRYPTED_PARAMETERS                           Handle        = 0xC0262503
+	ERROR_GRAPHICS_OPM_NO_VIDEO_OUTPUTS_EXIST                                 Handle        = 0xC0262505
+	ERROR_GRAPHICS_OPM_INTERNAL_ERROR                                         Handle        = 0xC026250B
+	ERROR_GRAPHICS_OPM_INVALID_HANDLE                                         Handle        = 0xC026250C
+	ERROR_GRAPHICS_PVP_INVALID_CERTIFICATE_LENGTH                             Handle        = 0xC026250E
+	ERROR_GRAPHICS_OPM_SPANNING_MODE_ENABLED                                  Handle        = 0xC026250F
+	ERROR_GRAPHICS_OPM_THEATER_MODE_ENABLED                                   Handle        = 0xC0262510
+	ERROR_GRAPHICS_PVP_HFS_FAILED                                             Handle        = 0xC0262511
+	ERROR_GRAPHICS_OPM_INVALID_SRM                                            Handle        = 0xC0262512
+	ERROR_GRAPHICS_OPM_OUTPUT_DOES_NOT_SUPPORT_HDCP                           Handle        = 0xC0262513
+	ERROR_GRAPHICS_OPM_OUTPUT_DOES_NOT_SUPPORT_ACP                            Handle        = 0xC0262514
+	ERROR_GRAPHICS_OPM_OUTPUT_DOES_NOT_SUPPORT_CGMSA                          Handle        = 0xC0262515
+	ERROR_GRAPHICS_OPM_HDCP_SRM_NEVER_SET                                     Handle        = 0xC0262516
+	ERROR_GRAPHICS_OPM_RESOLUTION_TOO_HIGH                                    Handle        = 0xC0262517
+	ERROR_GRAPHICS_OPM_ALL_HDCP_HARDWARE_ALREADY_IN_USE                       Handle        = 0xC0262518
+	ERROR_GRAPHICS_OPM_VIDEO_OUTPUT_NO_LONGER_EXISTS                          Handle        = 0xC026251A
+	ERROR_GRAPHICS_OPM_SESSION_TYPE_CHANGE_IN_PROGRESS                        Handle        = 0xC026251B
+	ERROR_GRAPHICS_OPM_VIDEO_OUTPUT_DOES_NOT_HAVE_COPP_SEMANTICS              Handle        = 0xC026251C
+	ERROR_GRAPHICS_OPM_INVALID_INFORMATION_REQUEST                            Handle        = 0xC026251D
+	ERROR_GRAPHICS_OPM_DRIVER_INTERNAL_ERROR                                  Handle        = 0xC026251E
+	ERROR_GRAPHICS_OPM_VIDEO_OUTPUT_DOES_NOT_HAVE_OPM_SEMANTICS               Handle        = 0xC026251F
+	ERROR_GRAPHICS_OPM_SIGNALING_NOT_SUPPORTED                                Handle        = 0xC0262520
+	ERROR_GRAPHICS_OPM_INVALID_CONFIGURATION_REQUEST                          Handle        = 0xC0262521
+	ERROR_GRAPHICS_I2C_NOT_SUPPORTED                                          Handle        = 0xC0262580
+	ERROR_GRAPHICS_I2C_DEVICE_DOES_NOT_EXIST                                  Handle        = 0xC0262581
+	ERROR_GRAPHICS_I2C_ERROR_TRANSMITTING_DATA                                Handle        = 0xC0262582
+	ERROR_GRAPHICS_I2C_ERROR_RECEIVING_DATA                                   Handle        = 0xC0262583
+	ERROR_GRAPHICS_DDCCI_VCP_NOT_SUPPORTED                                    Handle        = 0xC0262584
+	ERROR_GRAPHICS_DDCCI_INVALID_DATA                                         Handle        = 0xC0262585
+	ERROR_GRAPHICS_DDCCI_MONITOR_RETURNED_INVALID_TIMING_STATUS_BYTE          Handle        = 0xC0262586
+	ERROR_GRAPHICS_MCA_INVALID_CAPABILITIES_STRING                            Handle        = 0xC0262587
+	ERROR_GRAPHICS_MCA_INTERNAL_ERROR                                         Handle        = 0xC0262588
+	ERROR_GRAPHICS_DDCCI_INVALID_MESSAGE_COMMAND                              Handle        = 0xC0262589
+	ERROR_GRAPHICS_DDCCI_INVALID_MESSAGE_LENGTH                               Handle        = 0xC026258A
+	ERROR_GRAPHICS_DDCCI_INVALID_MESSAGE_CHECKSUM                             Handle        = 0xC026258B
+	ERROR_GRAPHICS_INVALID_PHYSICAL_MONITOR_HANDLE                            Handle        = 0xC026258C
+	ERROR_GRAPHICS_MONITOR_NO_LONGER_EXISTS                                   Handle        = 0xC026258D
+	ERROR_GRAPHICS_DDCCI_CURRENT_CURRENT_VALUE_GREATER_THAN_MAXIMUM_VALUE     Handle        = 0xC02625D8
+	ERROR_GRAPHICS_MCA_INVALID_VCP_VERSION                                    Handle        = 0xC02625D9
+	ERROR_GRAPHICS_MCA_MONITOR_VIOLATES_MCCS_SPECIFICATION                    Handle        = 0xC02625DA
+	ERROR_GRAPHICS_MCA_MCCS_VERSION_MISMATCH                                  Handle        = 0xC02625DB
+	ERROR_GRAPHICS_MCA_UNSUPPORTED_MCCS_VERSION                               Handle        = 0xC02625DC
+	ERROR_GRAPHICS_MCA_INVALID_TECHNOLOGY_TYPE_RETURNED                       Handle        = 0xC02625DE
+	ERROR_GRAPHICS_MCA_UNSUPPORTED_COLOR_TEMPERATURE                          Handle        = 0xC02625DF
+	ERROR_GRAPHICS_ONLY_CONSOLE_SESSION_SUPPORTED                             Handle        = 0xC02625E0
+	ERROR_GRAPHICS_NO_DISPLAY_DEVICE_CORRESPONDS_TO_NAME                      Handle        = 0xC02625E1
+	ERROR_GRAPHICS_DISPLAY_DEVICE_NOT_ATTACHED_TO_DESKTOP                     Handle        = 0xC02625E2
+	ERROR_GRAPHICS_MIRRORING_DEVICES_NOT_SUPPORTED                            Handle        = 0xC02625E3
+	ERROR_GRAPHICS_INVALID_POINTER                                            Handle        = 0xC02625E4
+	ERROR_GRAPHICS_NO_MONITORS_CORRESPOND_TO_DISPLAY_DEVICE                   Handle        = 0xC02625E5
+	ERROR_GRAPHICS_PARAMETER_ARRAY_TOO_SMALL                                  Handle        = 0xC02625E6
+	ERROR_GRAPHICS_INTERNAL_ERROR                                             Handle        = 0xC02625E7
+	ERROR_GRAPHICS_SESSION_TYPE_CHANGE_IN_PROGRESS                            Handle        = 0xC02605E8
+	NAP_E_INVALID_PACKET                                                      Handle        = 0x80270001
+	NAP_E_MISSING_SOH                                                         Handle        = 0x80270002
+	NAP_E_CONFLICTING_ID                                                      Handle        = 0x80270003
+	NAP_E_NO_CACHED_SOH                                                       Handle        = 0x80270004
+	NAP_E_STILL_BOUND                                                         Handle        = 0x80270005
+	NAP_E_NOT_REGISTERED                                                      Handle        = 0x80270006
+	NAP_E_NOT_INITIALIZED                                                     Handle        = 0x80270007
+	NAP_E_MISMATCHED_ID                                                       Handle        = 0x80270008
+	NAP_E_NOT_PENDING                                                         Handle        = 0x80270009
+	NAP_E_ID_NOT_FOUND                                                        Handle        = 0x8027000A
+	NAP_E_MAXSIZE_TOO_SMALL                                                   Handle        = 0x8027000B
+	NAP_E_SERVICE_NOT_RUNNING                                                 Handle        = 0x8027000C
+	NAP_S_CERT_ALREADY_PRESENT                                                Handle        = 0x0027000D
+	NAP_E_ENTITY_DISABLED                                                     Handle        = 0x8027000E
+	NAP_E_NETSH_GROUPPOLICY_ERROR                                             Handle        = 0x8027000F
+	NAP_E_TOO_MANY_CALLS                                                      Handle        = 0x80270010
+	NAP_E_SHV_CONFIG_EXISTED                                                  Handle        = 0x80270011
+	NAP_E_SHV_CONFIG_NOT_FOUND                                                Handle        = 0x80270012
+	NAP_E_SHV_TIMEOUT                                                         Handle        = 0x80270013
+	TPM_E_ERROR_MASK                                                          Handle        = 0x80280000
+	TPM_E_AUTHFAIL                                                            Handle        = 0x80280001
+	TPM_E_BADINDEX                                                            Handle        = 0x80280002
+	TPM_E_BAD_PARAMETER                                                       Handle        = 0x80280003
+	TPM_E_AUDITFAILURE                                                        Handle        = 0x80280004
+	TPM_E_CLEAR_DISABLED                                                      Handle        = 0x80280005
+	TPM_E_DEACTIVATED                                                         Handle        = 0x80280006
+	TPM_E_DISABLED                                                            Handle        = 0x80280007
+	TPM_E_DISABLED_CMD                                                        Handle        = 0x80280008
+	TPM_E_FAIL                                                                Handle        = 0x80280009
+	TPM_E_BAD_ORDINAL                                                         Handle        = 0x8028000A
+	TPM_E_INSTALL_DISABLED                                                    Handle        = 0x8028000B
+	TPM_E_INVALID_KEYHANDLE                                                   Handle        = 0x8028000C
+	TPM_E_KEYNOTFOUND                                                         Handle        = 0x8028000D
+	TPM_E_INAPPROPRIATE_ENC                                                   Handle        = 0x8028000E
+	TPM_E_MIGRATEFAIL                                                         Handle        = 0x8028000F
+	TPM_E_INVALID_PCR_INFO                                                    Handle        = 0x80280010
+	TPM_E_NOSPACE                                                             Handle        = 0x80280011
+	TPM_E_NOSRK                                                               Handle        = 0x80280012
+	TPM_E_NOTSEALED_BLOB                                                      Handle        = 0x80280013
+	TPM_E_OWNER_SET                                                           Handle        = 0x80280014
+	TPM_E_RESOURCES                                                           Handle        = 0x80280015
+	TPM_E_SHORTRANDOM                                                         Handle        = 0x80280016
+	TPM_E_SIZE                                                                Handle        = 0x80280017
+	TPM_E_WRONGPCRVAL                                                         Handle        = 0x80280018
+	TPM_E_BAD_PARAM_SIZE                                                      Handle        = 0x80280019
+	TPM_E_SHA_THREAD                                                          Handle        = 0x8028001A
+	TPM_E_SHA_ERROR                                                           Handle        = 0x8028001B
+	TPM_E_FAILEDSELFTEST                                                      Handle        = 0x8028001C
+	TPM_E_AUTH2FAIL                                                           Handle        = 0x8028001D
+	TPM_E_BADTAG                                                              Handle        = 0x8028001E
+	TPM_E_IOERROR                                                             Handle        = 0x8028001F
+	TPM_E_ENCRYPT_ERROR                                                       Handle        = 0x80280020
+	TPM_E_DECRYPT_ERROR                                                       Handle        = 0x80280021
+	TPM_E_INVALID_AUTHHANDLE                                                  Handle        = 0x80280022
+	TPM_E_NO_ENDORSEMENT                                                      Handle        = 0x80280023
+	TPM_E_INVALID_KEYUSAGE                                                    Handle        = 0x80280024
+	TPM_E_WRONG_ENTITYTYPE                                                    Handle        = 0x80280025
+	TPM_E_INVALID_POSTINIT                                                    Handle        = 0x80280026
+	TPM_E_INAPPROPRIATE_SIG                                                   Handle        = 0x80280027
+	TPM_E_BAD_KEY_PROPERTY                                                    Handle        = 0x80280028
+	TPM_E_BAD_MIGRATION                                                       Handle        = 0x80280029
+	TPM_E_BAD_SCHEME                                                          Handle        = 0x8028002A
+	TPM_E_BAD_DATASIZE                                                        Handle        = 0x8028002B
+	TPM_E_BAD_MODE                                                            Handle        = 0x8028002C
+	TPM_E_BAD_PRESENCE                                                        Handle        = 0x8028002D
+	TPM_E_BAD_VERSION                                                         Handle        = 0x8028002E
+	TPM_E_NO_WRAP_TRANSPORT                                                   Handle        = 0x8028002F
+	TPM_E_AUDITFAIL_UNSUCCESSFUL                                              Handle        = 0x80280030
+	TPM_E_AUDITFAIL_SUCCESSFUL                                                Handle        = 0x80280031
+	TPM_E_NOTRESETABLE                                                        Handle        = 0x80280032
+	TPM_E_NOTLOCAL                                                            Handle        = 0x80280033
+	TPM_E_BAD_TYPE                                                            Handle        = 0x80280034
+	TPM_E_INVALID_RESOURCE                                                    Handle        = 0x80280035
+	TPM_E_NOTFIPS                                                             Handle        = 0x80280036
+	TPM_E_INVALID_FAMILY                                                      Handle        = 0x80280037
+	TPM_E_NO_NV_PERMISSION                                                    Handle        = 0x80280038
+	TPM_E_REQUIRES_SIGN                                                       Handle        = 0x80280039
+	TPM_E_KEY_NOTSUPPORTED                                                    Handle        = 0x8028003A
+	TPM_E_AUTH_CONFLICT                                                       Handle        = 0x8028003B
+	TPM_E_AREA_LOCKED                                                         Handle        = 0x8028003C
+	TPM_E_BAD_LOCALITY                                                        Handle        = 0x8028003D
+	TPM_E_READ_ONLY                                                           Handle        = 0x8028003E
+	TPM_E_PER_NOWRITE                                                         Handle        = 0x8028003F
+	TPM_E_FAMILYCOUNT                                                         Handle        = 0x80280040
+	TPM_E_WRITE_LOCKED                                                        Handle        = 0x80280041
+	TPM_E_BAD_ATTRIBUTES                                                      Handle        = 0x80280042
+	TPM_E_INVALID_STRUCTURE                                                   Handle        = 0x80280043
+	TPM_E_KEY_OWNER_CONTROL                                                   Handle        = 0x80280044
+	TPM_E_BAD_COUNTER                                                         Handle        = 0x80280045
+	TPM_E_NOT_FULLWRITE                                                       Handle        = 0x80280046
+	TPM_E_CONTEXT_GAP                                                         Handle        = 0x80280047
+	TPM_E_MAXNVWRITES                                                         Handle        = 0x80280048
+	TPM_E_NOOPERATOR                                                          Handle        = 0x80280049
+	TPM_E_RESOURCEMISSING                                                     Handle        = 0x8028004A
+	TPM_E_DELEGATE_LOCK                                                       Handle        = 0x8028004B
+	TPM_E_DELEGATE_FAMILY                                                     Handle        = 0x8028004C
+	TPM_E_DELEGATE_ADMIN                                                      Handle        = 0x8028004D
+	TPM_E_TRANSPORT_NOTEXCLUSIVE                                              Handle        = 0x8028004E
+	TPM_E_OWNER_CONTROL                                                       Handle        = 0x8028004F
+	TPM_E_DAA_RESOURCES                                                       Handle        = 0x80280050
+	TPM_E_DAA_INPUT_DATA0                                                     Handle        = 0x80280051
+	TPM_E_DAA_INPUT_DATA1                                                     Handle        = 0x80280052
+	TPM_E_DAA_ISSUER_SETTINGS                                                 Handle        = 0x80280053
+	TPM_E_DAA_TPM_SETTINGS                                                    Handle        = 0x80280054
+	TPM_E_DAA_STAGE                                                           Handle        = 0x80280055
+	TPM_E_DAA_ISSUER_VALIDITY                                                 Handle        = 0x80280056
+	TPM_E_DAA_WRONG_W                                                         Handle        = 0x80280057
+	TPM_E_BAD_HANDLE                                                          Handle        = 0x80280058
+	TPM_E_BAD_DELEGATE                                                        Handle        = 0x80280059
+	TPM_E_BADCONTEXT                                                          Handle        = 0x8028005A
+	TPM_E_TOOMANYCONTEXTS                                                     Handle        = 0x8028005B
+	TPM_E_MA_TICKET_SIGNATURE                                                 Handle        = 0x8028005C
+	TPM_E_MA_DESTINATION                                                      Handle        = 0x8028005D
+	TPM_E_MA_SOURCE                                                           Handle        = 0x8028005E
+	TPM_E_MA_AUTHORITY                                                        Handle        = 0x8028005F
+	TPM_E_PERMANENTEK                                                         Handle        = 0x80280061
+	TPM_E_BAD_SIGNATURE                                                       Handle        = 0x80280062
+	TPM_E_NOCONTEXTSPACE                                                      Handle        = 0x80280063
+	TPM_20_E_ASYMMETRIC                                                       Handle        = 0x80280081
+	TPM_20_E_ATTRIBUTES                                                       Handle        = 0x80280082
+	TPM_20_E_HASH                                                             Handle        = 0x80280083
+	TPM_20_E_VALUE                                                            Handle        = 0x80280084
+	TPM_20_E_HIERARCHY                                                        Handle        = 0x80280085
+	TPM_20_E_KEY_SIZE                                                         Handle        = 0x80280087
+	TPM_20_E_MGF                                                              Handle        = 0x80280088
+	TPM_20_E_MODE                                                             Handle        = 0x80280089
+	TPM_20_E_TYPE                                                             Handle        = 0x8028008A
+	TPM_20_E_HANDLE                                                           Handle        = 0x8028008B
+	TPM_20_E_KDF                                                              Handle        = 0x8028008C
+	TPM_20_E_RANGE                                                            Handle        = 0x8028008D
+	TPM_20_E_AUTH_FAIL                                                        Handle        = 0x8028008E
+	TPM_20_E_NONCE                                                            Handle        = 0x8028008F
+	TPM_20_E_PP                                                               Handle        = 0x80280090
+	TPM_20_E_SCHEME                                                           Handle        = 0x80280092
+	TPM_20_E_SIZE                                                             Handle        = 0x80280095
+	TPM_20_E_SYMMETRIC                                                        Handle        = 0x80280096
+	TPM_20_E_TAG                                                              Handle        = 0x80280097
+	TPM_20_E_SELECTOR                                                         Handle        = 0x80280098
+	TPM_20_E_INSUFFICIENT                                                     Handle        = 0x8028009A
+	TPM_20_E_SIGNATURE                                                        Handle        = 0x8028009B
+	TPM_20_E_KEY                                                              Handle        = 0x8028009C
+	TPM_20_E_POLICY_FAIL                                                      Handle        = 0x8028009D
+	TPM_20_E_INTEGRITY                                                        Handle        = 0x8028009F
+	TPM_20_E_TICKET                                                           Handle        = 0x802800A0
+	TPM_20_E_RESERVED_BITS                                                    Handle        = 0x802800A1
+	TPM_20_E_BAD_AUTH                                                         Handle        = 0x802800A2
+	TPM_20_E_EXPIRED                                                          Handle        = 0x802800A3
+	TPM_20_E_POLICY_CC                                                        Handle        = 0x802800A4
+	TPM_20_E_BINDING                                                          Handle        = 0x802800A5
+	TPM_20_E_CURVE                                                            Handle        = 0x802800A6
+	TPM_20_E_ECC_POINT                                                        Handle        = 0x802800A7
+	TPM_20_E_INITIALIZE                                                       Handle        = 0x80280100
+	TPM_20_E_FAILURE                                                          Handle        = 0x80280101
+	TPM_20_E_SEQUENCE                                                         Handle        = 0x80280103
+	TPM_20_E_PRIVATE                                                          Handle        = 0x8028010B
+	TPM_20_E_HMAC                                                             Handle        = 0x80280119
+	TPM_20_E_DISABLED                                                         Handle        = 0x80280120
+	TPM_20_E_EXCLUSIVE                                                        Handle        = 0x80280121
+	TPM_20_E_ECC_CURVE                                                        Handle        = 0x80280123
+	TPM_20_E_AUTH_TYPE                                                        Handle        = 0x80280124
+	TPM_20_E_AUTH_MISSING                                                     Handle        = 0x80280125
+	TPM_20_E_POLICY                                                           Handle        = 0x80280126
+	TPM_20_E_PCR                                                              Handle        = 0x80280127
+	TPM_20_E_PCR_CHANGED                                                      Handle        = 0x80280128
+	TPM_20_E_UPGRADE                                                          Handle        = 0x8028012D
+	TPM_20_E_TOO_MANY_CONTEXTS                                                Handle        = 0x8028012E
+	TPM_20_E_AUTH_UNAVAILABLE                                                 Handle        = 0x8028012F
+	TPM_20_E_REBOOT                                                           Handle        = 0x80280130
+	TPM_20_E_UNBALANCED                                                       Handle        = 0x80280131
+	TPM_20_E_COMMAND_SIZE                                                     Handle        = 0x80280142
+	TPM_20_E_COMMAND_CODE                                                     Handle        = 0x80280143
+	TPM_20_E_AUTHSIZE                                                         Handle        = 0x80280144
+	TPM_20_E_AUTH_CONTEXT                                                     Handle        = 0x80280145
+	TPM_20_E_NV_RANGE                                                         Handle        = 0x80280146
+	TPM_20_E_NV_SIZE                                                          Handle        = 0x80280147
+	TPM_20_E_NV_LOCKED                                                        Handle        = 0x80280148
+	TPM_20_E_NV_AUTHORIZATION                                                 Handle        = 0x80280149
+	TPM_20_E_NV_UNINITIALIZED                                                 Handle        = 0x8028014A
+	TPM_20_E_NV_SPACE                                                         Handle        = 0x8028014B
+	TPM_20_E_NV_DEFINED                                                       Handle        = 0x8028014C
+	TPM_20_E_BAD_CONTEXT                                                      Handle        = 0x80280150
+	TPM_20_E_CPHASH                                                           Handle        = 0x80280151
+	TPM_20_E_PARENT                                                           Handle        = 0x80280152
+	TPM_20_E_NEEDS_TEST                                                       Handle        = 0x80280153
+	TPM_20_E_NO_RESULT                                                        Handle        = 0x80280154
+	TPM_20_E_SENSITIVE                                                        Handle        = 0x80280155
+	TPM_E_COMMAND_BLOCKED                                                     Handle        = 0x80280400
+	TPM_E_INVALID_HANDLE                                                      Handle        = 0x80280401
+	TPM_E_DUPLICATE_VHANDLE                                                   Handle        = 0x80280402
+	TPM_E_EMBEDDED_COMMAND_BLOCKED                                            Handle        = 0x80280403
+	TPM_E_EMBEDDED_COMMAND_UNSUPPORTED                                        Handle        = 0x80280404
+	TPM_E_RETRY                                                               Handle        = 0x80280800
+	TPM_E_NEEDS_SELFTEST                                                      Handle        = 0x80280801
+	TPM_E_DOING_SELFTEST                                                      Handle        = 0x80280802
+	TPM_E_DEFEND_LOCK_RUNNING                                                 Handle        = 0x80280803
+	TPM_20_E_CONTEXT_GAP                                                      Handle        = 0x80280901
+	TPM_20_E_OBJECT_MEMORY                                                    Handle        = 0x80280902
+	TPM_20_E_SESSION_MEMORY                                                   Handle        = 0x80280903
+	TPM_20_E_MEMORY                                                           Handle        = 0x80280904
+	TPM_20_E_SESSION_HANDLES                                                  Handle        = 0x80280905
+	TPM_20_E_OBJECT_HANDLES                                                   Handle        = 0x80280906
+	TPM_20_E_LOCALITY                                                         Handle        = 0x80280907
+	TPM_20_E_YIELDED                                                          Handle        = 0x80280908
+	TPM_20_E_CANCELED                                                         Handle        = 0x80280909
+	TPM_20_E_TESTING                                                          Handle        = 0x8028090A
+	TPM_20_E_NV_RATE                                                          Handle        = 0x80280920
+	TPM_20_E_LOCKOUT                                                          Handle        = 0x80280921
+	TPM_20_E_RETRY                                                            Handle        = 0x80280922
+	TPM_20_E_NV_UNAVAILABLE                                                   Handle        = 0x80280923
+	TBS_E_INTERNAL_ERROR                                                      Handle        = 0x80284001
+	TBS_E_BAD_PARAMETER                                                       Handle        = 0x80284002
+	TBS_E_INVALID_OUTPUT_POINTER                                              Handle        = 0x80284003
+	TBS_E_INVALID_CONTEXT                                                     Handle        = 0x80284004
+	TBS_E_INSUFFICIENT_BUFFER                                                 Handle        = 0x80284005
+	TBS_E_IOERROR                                                             Handle        = 0x80284006
+	TBS_E_INVALID_CONTEXT_PARAM                                               Handle        = 0x80284007
+	TBS_E_SERVICE_NOT_RUNNING                                                 Handle        = 0x80284008
+	TBS_E_TOO_MANY_TBS_CONTEXTS                                               Handle        = 0x80284009
+	TBS_E_TOO_MANY_RESOURCES                                                  Handle        = 0x8028400A
+	TBS_E_SERVICE_START_PENDING                                               Handle        = 0x8028400B
+	TBS_E_PPI_NOT_SUPPORTED                                                   Handle        = 0x8028400C
+	TBS_E_COMMAND_CANCELED                                                    Handle        = 0x8028400D
+	TBS_E_BUFFER_TOO_LARGE                                                    Handle        = 0x8028400E
+	TBS_E_TPM_NOT_FOUND                                                       Handle        = 0x8028400F
+	TBS_E_SERVICE_DISABLED                                                    Handle        = 0x80284010
+	TBS_E_NO_EVENT_LOG                                                        Handle        = 0x80284011
+	TBS_E_ACCESS_DENIED                                                       Handle        = 0x80284012
+	TBS_E_PROVISIONING_NOT_ALLOWED                                            Handle        = 0x80284013
+	TBS_E_PPI_FUNCTION_UNSUPPORTED                                            Handle        = 0x80284014
+	TBS_E_OWNERAUTH_NOT_FOUND                                                 Handle        = 0x80284015
+	TBS_E_PROVISIONING_INCOMPLETE                                             Handle        = 0x80284016
+	TPMAPI_E_INVALID_STATE                                                    Handle        = 0x80290100
+	TPMAPI_E_NOT_ENOUGH_DATA                                                  Handle        = 0x80290101
+	TPMAPI_E_TOO_MUCH_DATA                                                    Handle        = 0x80290102
+	TPMAPI_E_INVALID_OUTPUT_POINTER                                           Handle        = 0x80290103
+	TPMAPI_E_INVALID_PARAMETER                                                Handle        = 0x80290104
+	TPMAPI_E_OUT_OF_MEMORY                                                    Handle        = 0x80290105
+	TPMAPI_E_BUFFER_TOO_SMALL                                                 Handle        = 0x80290106
+	TPMAPI_E_INTERNAL_ERROR                                                   Handle        = 0x80290107
+	TPMAPI_E_ACCESS_DENIED                                                    Handle        = 0x80290108
+	TPMAPI_E_AUTHORIZATION_FAILED                                             Handle        = 0x80290109
+	TPMAPI_E_INVALID_CONTEXT_HANDLE                                           Handle        = 0x8029010A
+	TPMAPI_E_TBS_COMMUNICATION_ERROR                                          Handle        = 0x8029010B
+	TPMAPI_E_TPM_COMMAND_ERROR                                                Handle        = 0x8029010C
+	TPMAPI_E_MESSAGE_TOO_LARGE                                                Handle        = 0x8029010D
+	TPMAPI_E_INVALID_ENCODING                                                 Handle        = 0x8029010E
+	TPMAPI_E_INVALID_KEY_SIZE                                                 Handle        = 0x8029010F
+	TPMAPI_E_ENCRYPTION_FAILED                                                Handle        = 0x80290110
+	TPMAPI_E_INVALID_KEY_PARAMS                                               Handle        = 0x80290111
+	TPMAPI_E_INVALID_MIGRATION_AUTHORIZATION_BLOB                             Handle        = 0x80290112
+	TPMAPI_E_INVALID_PCR_INDEX                                                Handle        = 0x80290113
+	TPMAPI_E_INVALID_DELEGATE_BLOB                                            Handle        = 0x80290114
+	TPMAPI_E_INVALID_CONTEXT_PARAMS                                           Handle        = 0x80290115
+	TPMAPI_E_INVALID_KEY_BLOB                                                 Handle        = 0x80290116
+	TPMAPI_E_INVALID_PCR_DATA                                                 Handle        = 0x80290117
+	TPMAPI_E_INVALID_OWNER_AUTH                                               Handle        = 0x80290118
+	TPMAPI_E_FIPS_RNG_CHECK_FAILED                                            Handle        = 0x80290119
+	TPMAPI_E_EMPTY_TCG_LOG                                                    Handle        = 0x8029011A
+	TPMAPI_E_INVALID_TCG_LOG_ENTRY                                            Handle        = 0x8029011B
+	TPMAPI_E_TCG_SEPARATOR_ABSENT                                             Handle        = 0x8029011C
+	TPMAPI_E_TCG_INVALID_DIGEST_ENTRY                                         Handle        = 0x8029011D
+	TPMAPI_E_POLICY_DENIES_OPERATION                                          Handle        = 0x8029011E
+	TPMAPI_E_NV_BITS_NOT_DEFINED                                              Handle        = 0x8029011F
+	TPMAPI_E_NV_BITS_NOT_READY                                                Handle        = 0x80290120
+	TPMAPI_E_SEALING_KEY_NOT_AVAILABLE                                        Handle        = 0x80290121
+	TPMAPI_E_NO_AUTHORIZATION_CHAIN_FOUND                                     Handle        = 0x80290122
+	TPMAPI_E_SVN_COUNTER_NOT_AVAILABLE                                        Handle        = 0x80290123
+	TPMAPI_E_OWNER_AUTH_NOT_NULL                                              Handle        = 0x80290124
+	TPMAPI_E_ENDORSEMENT_AUTH_NOT_NULL                                        Handle        = 0x80290125
+	TPMAPI_E_AUTHORIZATION_REVOKED                                            Handle        = 0x80290126
+	TPMAPI_E_MALFORMED_AUTHORIZATION_KEY                                      Handle        = 0x80290127
+	TPMAPI_E_AUTHORIZING_KEY_NOT_SUPPORTED                                    Handle        = 0x80290128
+	TPMAPI_E_INVALID_AUTHORIZATION_SIGNATURE                                  Handle        = 0x80290129
+	TPMAPI_E_MALFORMED_AUTHORIZATION_POLICY                                   Handle        = 0x8029012A
+	TPMAPI_E_MALFORMED_AUTHORIZATION_OTHER                                    Handle        = 0x8029012B
+	TPMAPI_E_SEALING_KEY_CHANGED                                              Handle        = 0x8029012C
+	TBSIMP_E_BUFFER_TOO_SMALL                                                 Handle        = 0x80290200
+	TBSIMP_E_CLEANUP_FAILED                                                   Handle        = 0x80290201
+	TBSIMP_E_INVALID_CONTEXT_HANDLE                                           Handle        = 0x80290202
+	TBSIMP_E_INVALID_CONTEXT_PARAM                                            Handle        = 0x80290203
+	TBSIMP_E_TPM_ERROR                                                        Handle        = 0x80290204
+	TBSIMP_E_HASH_BAD_KEY                                                     Handle        = 0x80290205
+	TBSIMP_E_DUPLICATE_VHANDLE                                                Handle        = 0x80290206
+	TBSIMP_E_INVALID_OUTPUT_POINTER                                           Handle        = 0x80290207
+	TBSIMP_E_INVALID_PARAMETER                                                Handle        = 0x80290208
+	TBSIMP_E_RPC_INIT_FAILED                                                  Handle        = 0x80290209
+	TBSIMP_E_SCHEDULER_NOT_RUNNING                                            Handle        = 0x8029020A
+	TBSIMP_E_COMMAND_CANCELED                                                 Handle        = 0x8029020B
+	TBSIMP_E_OUT_OF_MEMORY                                                    Handle        = 0x8029020C
+	TBSIMP_E_LIST_NO_MORE_ITEMS                                               Handle        = 0x8029020D
+	TBSIMP_E_LIST_NOT_FOUND                                                   Handle        = 0x8029020E
+	TBSIMP_E_NOT_ENOUGH_SPACE                                                 Handle        = 0x8029020F
+	TBSIMP_E_NOT_ENOUGH_TPM_CONTEXTS                                          Handle        = 0x80290210
+	TBSIMP_E_COMMAND_FAILED                                                   Handle        = 0x80290211
+	TBSIMP_E_UNKNOWN_ORDINAL                                                  Handle        = 0x80290212
+	TBSIMP_E_RESOURCE_EXPIRED                                                 Handle        = 0x80290213
+	TBSIMP_E_INVALID_RESOURCE                                                 Handle        = 0x80290214
+	TBSIMP_E_NOTHING_TO_UNLOAD                                                Handle        = 0x80290215
+	TBSIMP_E_HASH_TABLE_FULL                                                  Handle        = 0x80290216
+	TBSIMP_E_TOO_MANY_TBS_CONTEXTS                                            Handle        = 0x80290217
+	TBSIMP_E_TOO_MANY_RESOURCES                                               Handle        = 0x80290218
+	TBSIMP_E_PPI_NOT_SUPPORTED                                                Handle        = 0x80290219
+	TBSIMP_E_TPM_INCOMPATIBLE                                                 Handle        = 0x8029021A
+	TBSIMP_E_NO_EVENT_LOG                                                     Handle        = 0x8029021B
+	TPM_E_PPI_ACPI_FAILURE                                                    Handle        = 0x80290300
+	TPM_E_PPI_USER_ABORT                                                      Handle        = 0x80290301
+	TPM_E_PPI_BIOS_FAILURE                                                    Handle        = 0x80290302
+	TPM_E_PPI_NOT_SUPPORTED                                                   Handle        = 0x80290303
+	TPM_E_PPI_BLOCKED_IN_BIOS                                                 Handle        = 0x80290304
+	TPM_E_PCP_ERROR_MASK                                                      Handle        = 0x80290400
+	TPM_E_PCP_DEVICE_NOT_READY                                                Handle        = 0x80290401
+	TPM_E_PCP_INVALID_HANDLE                                                  Handle        = 0x80290402
+	TPM_E_PCP_INVALID_PARAMETER                                               Handle        = 0x80290403
+	TPM_E_PCP_FLAG_NOT_SUPPORTED                                              Handle        = 0x80290404
+	TPM_E_PCP_NOT_SUPPORTED                                                   Handle        = 0x80290405
+	TPM_E_PCP_BUFFER_TOO_SMALL                                                Handle        = 0x80290406
+	TPM_E_PCP_INTERNAL_ERROR                                                  Handle        = 0x80290407
+	TPM_E_PCP_AUTHENTICATION_FAILED                                           Handle        = 0x80290408
+	TPM_E_PCP_AUTHENTICATION_IGNORED                                          Handle        = 0x80290409
+	TPM_E_PCP_POLICY_NOT_FOUND                                                Handle        = 0x8029040A
+	TPM_E_PCP_PROFILE_NOT_FOUND                                               Handle        = 0x8029040B
+	TPM_E_PCP_VALIDATION_FAILED                                               Handle        = 0x8029040C
+	TPM_E_PCP_WRONG_PARENT                                                    Handle        = 0x8029040E
+	TPM_E_KEY_NOT_LOADED                                                      Handle        = 0x8029040F
+	TPM_E_NO_KEY_CERTIFICATION                                                Handle        = 0x80290410
+	TPM_E_KEY_NOT_FINALIZED                                                   Handle        = 0x80290411
+	TPM_E_ATTESTATION_CHALLENGE_NOT_SET                                       Handle        = 0x80290412
+	TPM_E_NOT_PCR_BOUND                                                       Handle        = 0x80290413
+	TPM_E_KEY_ALREADY_FINALIZED                                               Handle        = 0x80290414
+	TPM_E_KEY_USAGE_POLICY_NOT_SUPPORTED                                      Handle        = 0x80290415
+	TPM_E_KEY_USAGE_POLICY_INVALID                                            Handle        = 0x80290416
+	TPM_E_SOFT_KEY_ERROR                                                      Handle        = 0x80290417
+	TPM_E_KEY_NOT_AUTHENTICATED                                               Handle        = 0x80290418
+	TPM_E_PCP_KEY_NOT_AIK                                                     Handle        = 0x80290419
+	TPM_E_KEY_NOT_SIGNING_KEY                                                 Handle        = 0x8029041A
+	TPM_E_LOCKED_OUT                                                          Handle        = 0x8029041B
+	TPM_E_CLAIM_TYPE_NOT_SUPPORTED                                            Handle        = 0x8029041C
+	TPM_E_VERSION_NOT_SUPPORTED                                               Handle        = 0x8029041D
+	TPM_E_BUFFER_LENGTH_MISMATCH                                              Handle        = 0x8029041E
+	TPM_E_PCP_IFX_RSA_KEY_CREATION_BLOCKED                                    Handle        = 0x8029041F
+	TPM_E_PCP_TICKET_MISSING                                                  Handle        = 0x80290420
+	TPM_E_PCP_RAW_POLICY_NOT_SUPPORTED                                        Handle        = 0x80290421
+	TPM_E_PCP_KEY_HANDLE_INVALIDATED                                          Handle        = 0x80290422
+	TPM_E_PCP_UNSUPPORTED_PSS_SALT                                            Handle        = 0x40290423
+	TPM_E_ZERO_EXHAUST_ENABLED                                                Handle        = 0x80290500
+	PLA_E_DCS_NOT_FOUND                                                       Handle        = 0x80300002
+	PLA_E_DCS_IN_USE                                                          Handle        = 0x803000AA
+	PLA_E_TOO_MANY_FOLDERS                                                    Handle        = 0x80300045
+	PLA_E_NO_MIN_DISK                                                         Handle        = 0x80300070
+	PLA_E_DCS_ALREADY_EXISTS                                                  Handle        = 0x803000B7
+	PLA_S_PROPERTY_IGNORED                                                    Handle        = 0x00300100
+	PLA_E_PROPERTY_CONFLICT                                                   Handle        = 0x80300101
+	PLA_E_DCS_SINGLETON_REQUIRED                                              Handle        = 0x80300102
+	PLA_E_CREDENTIALS_REQUIRED                                                Handle        = 0x80300103
+	PLA_E_DCS_NOT_RUNNING                                                     Handle        = 0x80300104
+	PLA_E_CONFLICT_INCL_EXCL_API                                              Handle        = 0x80300105
+	PLA_E_NETWORK_EXE_NOT_VALID                                               Handle        = 0x80300106
+	PLA_E_EXE_ALREADY_CONFIGURED                                              Handle        = 0x80300107
+	PLA_E_EXE_PATH_NOT_VALID                                                  Handle        = 0x80300108
+	PLA_E_DC_ALREADY_EXISTS                                                   Handle        = 0x80300109
+	PLA_E_DCS_START_WAIT_TIMEOUT                                              Handle        = 0x8030010A
+	PLA_E_DC_START_WAIT_TIMEOUT                                               Handle        = 0x8030010B
+	PLA_E_REPORT_WAIT_TIMEOUT                                                 Handle        = 0x8030010C
+	PLA_E_NO_DUPLICATES                                                       Handle        = 0x8030010D
+	PLA_E_EXE_FULL_PATH_REQUIRED                                              Handle        = 0x8030010E
+	PLA_E_INVALID_SESSION_NAME                                                Handle        = 0x8030010F
+	PLA_E_PLA_CHANNEL_NOT_ENABLED                                             Handle        = 0x80300110
+	PLA_E_TASKSCHED_CHANNEL_NOT_ENABLED                                       Handle        = 0x80300111
+	PLA_E_RULES_MANAGER_FAILED                                                Handle        = 0x80300112
+	PLA_E_CABAPI_FAILURE                                                      Handle        = 0x80300113
+	FVE_E_LOCKED_VOLUME                                                       Handle        = 0x80310000
+	FVE_E_NOT_ENCRYPTED                                                       Handle        = 0x80310001
+	FVE_E_NO_TPM_BIOS                                                         Handle        = 0x80310002
+	FVE_E_NO_MBR_METRIC                                                       Handle        = 0x80310003
+	FVE_E_NO_BOOTSECTOR_METRIC                                                Handle        = 0x80310004
+	FVE_E_NO_BOOTMGR_METRIC                                                   Handle        = 0x80310005
+	FVE_E_WRONG_BOOTMGR                                                       Handle        = 0x80310006
+	FVE_E_SECURE_KEY_REQUIRED                                                 Handle        = 0x80310007
+	FVE_E_NOT_ACTIVATED                                                       Handle        = 0x80310008
+	FVE_E_ACTION_NOT_ALLOWED                                                  Handle        = 0x80310009
+	FVE_E_AD_SCHEMA_NOT_INSTALLED                                             Handle        = 0x8031000A
+	FVE_E_AD_INVALID_DATATYPE                                                 Handle        = 0x8031000B
+	FVE_E_AD_INVALID_DATASIZE                                                 Handle        = 0x8031000C
+	FVE_E_AD_NO_VALUES                                                        Handle        = 0x8031000D
+	FVE_E_AD_ATTR_NOT_SET                                                     Handle        = 0x8031000E
+	FVE_E_AD_GUID_NOT_FOUND                                                   Handle        = 0x8031000F
+	FVE_E_BAD_INFORMATION                                                     Handle        = 0x80310010
+	FVE_E_TOO_SMALL                                                           Handle        = 0x80310011
+	FVE_E_SYSTEM_VOLUME                                                       Handle        = 0x80310012
+	FVE_E_FAILED_WRONG_FS                                                     Handle        = 0x80310013
+	FVE_E_BAD_PARTITION_SIZE                                                  Handle        = 0x80310014
+	FVE_E_NOT_SUPPORTED                                                       Handle        = 0x80310015
+	FVE_E_BAD_DATA                                                            Handle        = 0x80310016
+	FVE_E_VOLUME_NOT_BOUND                                                    Handle        = 0x80310017
+	FVE_E_TPM_NOT_OWNED                                                       Handle        = 0x80310018
+	FVE_E_NOT_DATA_VOLUME                                                     Handle        = 0x80310019
+	FVE_E_AD_INSUFFICIENT_BUFFER                                              Handle        = 0x8031001A
+	FVE_E_CONV_READ                                                           Handle        = 0x8031001B
+	FVE_E_CONV_WRITE                                                          Handle        = 0x8031001C
+	FVE_E_KEY_REQUIRED                                                        Handle        = 0x8031001D
+	FVE_E_CLUSTERING_NOT_SUPPORTED                                            Handle        = 0x8031001E
+	FVE_E_VOLUME_BOUND_ALREADY                                                Handle        = 0x8031001F
+	FVE_E_OS_NOT_PROTECTED                                                    Handle        = 0x80310020
+	FVE_E_PROTECTION_DISABLED                                                 Handle        = 0x80310021
+	FVE_E_RECOVERY_KEY_REQUIRED                                               Handle        = 0x80310022
+	FVE_E_FOREIGN_VOLUME                                                      Handle        = 0x80310023
+	FVE_E_OVERLAPPED_UPDATE                                                   Handle        = 0x80310024
+	FVE_E_TPM_SRK_AUTH_NOT_ZERO                                               Handle        = 0x80310025
+	FVE_E_FAILED_SECTOR_SIZE                                                  Handle        = 0x80310026
+	FVE_E_FAILED_AUTHENTICATION                                               Handle        = 0x80310027
+	FVE_E_NOT_OS_VOLUME                                                       Handle        = 0x80310028
+	FVE_E_AUTOUNLOCK_ENABLED                                                  Handle        = 0x80310029
+	FVE_E_WRONG_BOOTSECTOR                                                    Handle        = 0x8031002A
+	FVE_E_WRONG_SYSTEM_FS                                                     Handle        = 0x8031002B
+	FVE_E_POLICY_PASSWORD_REQUIRED                                            Handle        = 0x8031002C
+	FVE_E_CANNOT_SET_FVEK_ENCRYPTED                                           Handle        = 0x8031002D
+	FVE_E_CANNOT_ENCRYPT_NO_KEY                                               Handle        = 0x8031002E
+	FVE_E_BOOTABLE_CDDVD                                                      Handle        = 0x80310030
+	FVE_E_PROTECTOR_EXISTS                                                    Handle        = 0x80310031
+	FVE_E_RELATIVE_PATH                                                       Handle        = 0x80310032
+	FVE_E_PROTECTOR_NOT_FOUND                                                 Handle        = 0x80310033
+	FVE_E_INVALID_KEY_FORMAT                                                  Handle        = 0x80310034
+	FVE_E_INVALID_PASSWORD_FORMAT                                             Handle        = 0x80310035
+	FVE_E_FIPS_RNG_CHECK_FAILED                                               Handle        = 0x80310036
+	FVE_E_FIPS_PREVENTS_RECOVERY_PASSWORD                                     Handle        = 0x80310037
+	FVE_E_FIPS_PREVENTS_EXTERNAL_KEY_EXPORT                                   Handle        = 0x80310038
+	FVE_E_NOT_DECRYPTED                                                       Handle        = 0x80310039
+	FVE_E_INVALID_PROTECTOR_TYPE                                              Handle        = 0x8031003A
+	FVE_E_NO_PROTECTORS_TO_TEST                                               Handle        = 0x8031003B
+	FVE_E_KEYFILE_NOT_FOUND                                                   Handle        = 0x8031003C
+	FVE_E_KEYFILE_INVALID                                                     Handle        = 0x8031003D
+	FVE_E_KEYFILE_NO_VMK                                                      Handle        = 0x8031003E
+	FVE_E_TPM_DISABLED                                                        Handle        = 0x8031003F
+	FVE_E_NOT_ALLOWED_IN_SAFE_MODE                                            Handle        = 0x80310040
+	FVE_E_TPM_INVALID_PCR                                                     Handle        = 0x80310041
+	FVE_E_TPM_NO_VMK                                                          Handle        = 0x80310042
+	FVE_E_PIN_INVALID                                                         Handle        = 0x80310043
+	FVE_E_AUTH_INVALID_APPLICATION                                            Handle        = 0x80310044
+	FVE_E_AUTH_INVALID_CONFIG                                                 Handle        = 0x80310045
+	FVE_E_FIPS_DISABLE_PROTECTION_NOT_ALLOWED                                 Handle        = 0x80310046
+	FVE_E_FS_NOT_EXTENDED                                                     Handle        = 0x80310047
+	FVE_E_FIRMWARE_TYPE_NOT_SUPPORTED                                         Handle        = 0x80310048
+	FVE_E_NO_LICENSE                                                          Handle        = 0x80310049
+	FVE_E_NOT_ON_STACK                                                        Handle        = 0x8031004A
+	FVE_E_FS_MOUNTED                                                          Handle        = 0x8031004B
+	FVE_E_TOKEN_NOT_IMPERSONATED                                              Handle        = 0x8031004C
+	FVE_E_DRY_RUN_FAILED                                                      Handle        = 0x8031004D
+	FVE_E_REBOOT_REQUIRED                                                     Handle        = 0x8031004E
+	FVE_E_DEBUGGER_ENABLED                                                    Handle        = 0x8031004F
+	FVE_E_RAW_ACCESS                                                          Handle        = 0x80310050
+	FVE_E_RAW_BLOCKED                                                         Handle        = 0x80310051
+	FVE_E_BCD_APPLICATIONS_PATH_INCORRECT                                     Handle        = 0x80310052
+	FVE_E_NOT_ALLOWED_IN_VERSION                                              Handle        = 0x80310053
+	FVE_E_NO_AUTOUNLOCK_MASTER_KEY                                            Handle        = 0x80310054
+	FVE_E_MOR_FAILED                                                          Handle        = 0x80310055
+	FVE_E_HIDDEN_VOLUME                                                       Handle        = 0x80310056
+	FVE_E_TRANSIENT_STATE                                                     Handle        = 0x80310057
+	FVE_E_PUBKEY_NOT_ALLOWED                                                  Handle        = 0x80310058
+	FVE_E_VOLUME_HANDLE_OPEN                                                  Handle        = 0x80310059
+	FVE_E_NO_FEATURE_LICENSE                                                  Handle        = 0x8031005A
+	FVE_E_INVALID_STARTUP_OPTIONS                                             Handle        = 0x8031005B
+	FVE_E_POLICY_RECOVERY_PASSWORD_NOT_ALLOWED                                Handle        = 0x8031005C
+	FVE_E_POLICY_RECOVERY_PASSWORD_REQUIRED                                   Handle        = 0x8031005D
+	FVE_E_POLICY_RECOVERY_KEY_NOT_ALLOWED                                     Handle        = 0x8031005E
+	FVE_E_POLICY_RECOVERY_KEY_REQUIRED                                        Handle        = 0x8031005F
+	FVE_E_POLICY_STARTUP_PIN_NOT_ALLOWED                                      Handle        = 0x80310060
+	FVE_E_POLICY_STARTUP_PIN_REQUIRED                                         Handle        = 0x80310061
+	FVE_E_POLICY_STARTUP_KEY_NOT_ALLOWED                                      Handle        = 0x80310062
+	FVE_E_POLICY_STARTUP_KEY_REQUIRED                                         Handle        = 0x80310063
+	FVE_E_POLICY_STARTUP_PIN_KEY_NOT_ALLOWED                                  Handle        = 0x80310064
+	FVE_E_POLICY_STARTUP_PIN_KEY_REQUIRED                                     Handle        = 0x80310065
+	FVE_E_POLICY_STARTUP_TPM_NOT_ALLOWED                                      Handle        = 0x80310066
+	FVE_E_POLICY_STARTUP_TPM_REQUIRED                                         Handle        = 0x80310067
+	FVE_E_POLICY_INVALID_PIN_LENGTH                                           Handle        = 0x80310068
+	FVE_E_KEY_PROTECTOR_NOT_SUPPORTED                                         Handle        = 0x80310069
+	FVE_E_POLICY_PASSPHRASE_NOT_ALLOWED                                       Handle        = 0x8031006A
+	FVE_E_POLICY_PASSPHRASE_REQUIRED                                          Handle        = 0x8031006B
+	FVE_E_FIPS_PREVENTS_PASSPHRASE                                            Handle        = 0x8031006C
+	FVE_E_OS_VOLUME_PASSPHRASE_NOT_ALLOWED                                    Handle        = 0x8031006D
+	FVE_E_INVALID_BITLOCKER_OID                                               Handle        = 0x8031006E
+	FVE_E_VOLUME_TOO_SMALL                                                    Handle        = 0x8031006F
+	FVE_E_DV_NOT_SUPPORTED_ON_FS                                              Handle        = 0x80310070
+	FVE_E_DV_NOT_ALLOWED_BY_GP                                                Handle        = 0x80310071
+	FVE_E_POLICY_USER_CERTIFICATE_NOT_ALLOWED                                 Handle        = 0x80310072
+	FVE_E_POLICY_USER_CERTIFICATE_REQUIRED                                    Handle        = 0x80310073
+	FVE_E_POLICY_USER_CERT_MUST_BE_HW                                         Handle        = 0x80310074
+	FVE_E_POLICY_USER_CONFIGURE_FDV_AUTOUNLOCK_NOT_ALLOWED                    Handle        = 0x80310075
+	FVE_E_POLICY_USER_CONFIGURE_RDV_AUTOUNLOCK_NOT_ALLOWED                    Handle        = 0x80310076
+	FVE_E_POLICY_USER_CONFIGURE_RDV_NOT_ALLOWED                               Handle        = 0x80310077
+	FVE_E_POLICY_USER_ENABLE_RDV_NOT_ALLOWED                                  Handle        = 0x80310078
+	FVE_E_POLICY_USER_DISABLE_RDV_NOT_ALLOWED                                 Handle        = 0x80310079
+	FVE_E_POLICY_INVALID_PASSPHRASE_LENGTH                                    Handle        = 0x80310080
+	FVE_E_POLICY_PASSPHRASE_TOO_SIMPLE                                        Handle        = 0x80310081
+	FVE_E_RECOVERY_PARTITION                                                  Handle        = 0x80310082
+	FVE_E_POLICY_CONFLICT_FDV_RK_OFF_AUK_ON                                   Handle        = 0x80310083
+	FVE_E_POLICY_CONFLICT_RDV_RK_OFF_AUK_ON                                   Handle        = 0x80310084
+	FVE_E_NON_BITLOCKER_OID                                                   Handle        = 0x80310085
+	FVE_E_POLICY_PROHIBITS_SELFSIGNED                                         Handle        = 0x80310086
+	FVE_E_POLICY_CONFLICT_RO_AND_STARTUP_KEY_REQUIRED                         Handle        = 0x80310087
+	FVE_E_CONV_RECOVERY_FAILED                                                Handle        = 0x80310088
+	FVE_E_VIRTUALIZED_SPACE_TOO_BIG                                           Handle        = 0x80310089
+	FVE_E_POLICY_CONFLICT_OSV_RP_OFF_ADB_ON                                   Handle        = 0x80310090
+	FVE_E_POLICY_CONFLICT_FDV_RP_OFF_ADB_ON                                   Handle        = 0x80310091
+	FVE_E_POLICY_CONFLICT_RDV_RP_OFF_ADB_ON                                   Handle        = 0x80310092
+	FVE_E_NON_BITLOCKER_KU                                                    Handle        = 0x80310093
+	FVE_E_PRIVATEKEY_AUTH_FAILED                                              Handle        = 0x80310094
+	FVE_E_REMOVAL_OF_DRA_FAILED                                               Handle        = 0x80310095
+	FVE_E_OPERATION_NOT_SUPPORTED_ON_VISTA_VOLUME                             Handle        = 0x80310096
+	FVE_E_CANT_LOCK_AUTOUNLOCK_ENABLED_VOLUME                                 Handle        = 0x80310097
+	FVE_E_FIPS_HASH_KDF_NOT_ALLOWED                                           Handle        = 0x80310098
+	FVE_E_ENH_PIN_INVALID                                                     Handle        = 0x80310099
+	FVE_E_INVALID_PIN_CHARS                                                   Handle        = 0x8031009A
+	FVE_E_INVALID_DATUM_TYPE                                                  Handle        = 0x8031009B
+	FVE_E_EFI_ONLY                                                            Handle        = 0x8031009C
+	FVE_E_MULTIPLE_NKP_CERTS                                                  Handle        = 0x8031009D
+	FVE_E_REMOVAL_OF_NKP_FAILED                                               Handle        = 0x8031009E
+	FVE_E_INVALID_NKP_CERT                                                    Handle        = 0x8031009F
+	FVE_E_NO_EXISTING_PIN                                                     Handle        = 0x803100A0
+	FVE_E_PROTECTOR_CHANGE_PIN_MISMATCH                                       Handle        = 0x803100A1
+	FVE_E_PIN_PROTECTOR_CHANGE_BY_STD_USER_DISALLOWED                         Handle        = 0x803100A2
+	FVE_E_PROTECTOR_CHANGE_MAX_PIN_CHANGE_ATTEMPTS_REACHED                    Handle        = 0x803100A3
+	FVE_E_POLICY_PASSPHRASE_REQUIRES_ASCII                                    Handle        = 0x803100A4
+	FVE_E_FULL_ENCRYPTION_NOT_ALLOWED_ON_TP_STORAGE                           Handle        = 0x803100A5
+	FVE_E_WIPE_NOT_ALLOWED_ON_TP_STORAGE                                      Handle        = 0x803100A6
+	FVE_E_KEY_LENGTH_NOT_SUPPORTED_BY_EDRIVE                                  Handle        = 0x803100A7
+	FVE_E_NO_EXISTING_PASSPHRASE                                              Handle        = 0x803100A8
+	FVE_E_PROTECTOR_CHANGE_PASSPHRASE_MISMATCH                                Handle        = 0x803100A9
+	FVE_E_PASSPHRASE_TOO_LONG                                                 Handle        = 0x803100AA
+	FVE_E_NO_PASSPHRASE_WITH_TPM                                              Handle        = 0x803100AB
+	FVE_E_NO_TPM_WITH_PASSPHRASE                                              Handle        = 0x803100AC
+	FVE_E_NOT_ALLOWED_ON_CSV_STACK                                            Handle        = 0x803100AD
+	FVE_E_NOT_ALLOWED_ON_CLUSTER                                              Handle        = 0x803100AE
+	FVE_E_EDRIVE_NO_FAILOVER_TO_SW                                            Handle        = 0x803100AF
+	FVE_E_EDRIVE_BAND_IN_USE                                                  Handle        = 0x803100B0
+	FVE_E_EDRIVE_DISALLOWED_BY_GP                                             Handle        = 0x803100B1
+	FVE_E_EDRIVE_INCOMPATIBLE_VOLUME                                          Handle        = 0x803100B2
+	FVE_E_NOT_ALLOWED_TO_UPGRADE_WHILE_CONVERTING                             Handle        = 0x803100B3
+	FVE_E_EDRIVE_DV_NOT_SUPPORTED                                             Handle        = 0x803100B4
+	FVE_E_NO_PREBOOT_KEYBOARD_DETECTED                                        Handle        = 0x803100B5
+	FVE_E_NO_PREBOOT_KEYBOARD_OR_WINRE_DETECTED                               Handle        = 0x803100B6
+	FVE_E_POLICY_REQUIRES_STARTUP_PIN_ON_TOUCH_DEVICE                         Handle        = 0x803100B7
+	FVE_E_POLICY_REQUIRES_RECOVERY_PASSWORD_ON_TOUCH_DEVICE                   Handle        = 0x803100B8
+	FVE_E_WIPE_CANCEL_NOT_APPLICABLE                                          Handle        = 0x803100B9
+	FVE_E_SECUREBOOT_DISABLED                                                 Handle        = 0x803100BA
+	FVE_E_SECUREBOOT_CONFIGURATION_INVALID                                    Handle        = 0x803100BB
+	FVE_E_EDRIVE_DRY_RUN_FAILED                                               Handle        = 0x803100BC
+	FVE_E_SHADOW_COPY_PRESENT                                                 Handle        = 0x803100BD
+	FVE_E_POLICY_INVALID_ENHANCED_BCD_SETTINGS                                Handle        = 0x803100BE
+	FVE_E_EDRIVE_INCOMPATIBLE_FIRMWARE                                        Handle        = 0x803100BF
+	FVE_E_PROTECTOR_CHANGE_MAX_PASSPHRASE_CHANGE_ATTEMPTS_REACHED             Handle        = 0x803100C0
+	FVE_E_PASSPHRASE_PROTECTOR_CHANGE_BY_STD_USER_DISALLOWED                  Handle        = 0x803100C1
+	FVE_E_LIVEID_ACCOUNT_SUSPENDED                                            Handle        = 0x803100C2
+	FVE_E_LIVEID_ACCOUNT_BLOCKED                                              Handle        = 0x803100C3
+	FVE_E_NOT_PROVISIONED_ON_ALL_VOLUMES                                      Handle        = 0x803100C4
+	FVE_E_DE_FIXED_DATA_NOT_SUPPORTED                                         Handle        = 0x803100C5
+	FVE_E_DE_HARDWARE_NOT_COMPLIANT                                           Handle        = 0x803100C6
+	FVE_E_DE_WINRE_NOT_CONFIGURED                                             Handle        = 0x803100C7
+	FVE_E_DE_PROTECTION_SUSPENDED                                             Handle        = 0x803100C8
+	FVE_E_DE_OS_VOLUME_NOT_PROTECTED                                          Handle        = 0x803100C9
+	FVE_E_DE_DEVICE_LOCKEDOUT                                                 Handle        = 0x803100CA
+	FVE_E_DE_PROTECTION_NOT_YET_ENABLED                                       Handle        = 0x803100CB
+	FVE_E_INVALID_PIN_CHARS_DETAILED                                          Handle        = 0x803100CC
+	FVE_E_DEVICE_LOCKOUT_COUNTER_UNAVAILABLE                                  Handle        = 0x803100CD
+	FVE_E_DEVICELOCKOUT_COUNTER_MISMATCH                                      Handle        = 0x803100CE
+	FVE_E_BUFFER_TOO_LARGE                                                    Handle        = 0x803100CF
+	FVE_E_NO_SUCH_CAPABILITY_ON_TARGET                                        Handle        = 0x803100D0
+	FVE_E_DE_PREVENTED_FOR_OS                                                 Handle        = 0x803100D1
+	FVE_E_DE_VOLUME_OPTED_OUT                                                 Handle        = 0x803100D2
+	FVE_E_DE_VOLUME_NOT_SUPPORTED                                             Handle        = 0x803100D3
+	FVE_E_EOW_NOT_SUPPORTED_IN_VERSION                                        Handle        = 0x803100D4
+	FVE_E_ADBACKUP_NOT_ENABLED                                                Handle        = 0x803100D5
+	FVE_E_VOLUME_EXTEND_PREVENTS_EOW_DECRYPT                                  Handle        = 0x803100D6
+	FVE_E_NOT_DE_VOLUME                                                       Handle        = 0x803100D7
+	FVE_E_PROTECTION_CANNOT_BE_DISABLED                                       Handle        = 0x803100D8
+	FVE_E_OSV_KSR_NOT_ALLOWED                                                 Handle        = 0x803100D9
+	FWP_E_CALLOUT_NOT_FOUND                                                   Handle        = 0x80320001
+	FWP_E_CONDITION_NOT_FOUND                                                 Handle        = 0x80320002
+	FWP_E_FILTER_NOT_FOUND                                                    Handle        = 0x80320003
+	FWP_E_LAYER_NOT_FOUND                                                     Handle        = 0x80320004
+	FWP_E_PROVIDER_NOT_FOUND                                                  Handle        = 0x80320005
+	FWP_E_PROVIDER_CONTEXT_NOT_FOUND                                          Handle        = 0x80320006
+	FWP_E_SUBLAYER_NOT_FOUND                                                  Handle        = 0x80320007
+	FWP_E_NOT_FOUND                                                           Handle        = 0x80320008
+	FWP_E_ALREADY_EXISTS                                                      Handle        = 0x80320009
+	FWP_E_IN_USE                                                              Handle        = 0x8032000A
+	FWP_E_DYNAMIC_SESSION_IN_PROGRESS                                         Handle        = 0x8032000B
+	FWP_E_WRONG_SESSION                                                       Handle        = 0x8032000C
+	FWP_E_NO_TXN_IN_PROGRESS                                                  Handle        = 0x8032000D
+	FWP_E_TXN_IN_PROGRESS                                                     Handle        = 0x8032000E
+	FWP_E_TXN_ABORTED                                                         Handle        = 0x8032000F
+	FWP_E_SESSION_ABORTED                                                     Handle        = 0x80320010
+	FWP_E_INCOMPATIBLE_TXN                                                    Handle        = 0x80320011
+	FWP_E_TIMEOUT                                                             Handle        = 0x80320012
+	FWP_E_NET_EVENTS_DISABLED                                                 Handle        = 0x80320013
+	FWP_E_INCOMPATIBLE_LAYER                                                  Handle        = 0x80320014
+	FWP_E_KM_CLIENTS_ONLY                                                     Handle        = 0x80320015
+	FWP_E_LIFETIME_MISMATCH                                                   Handle        = 0x80320016
+	FWP_E_BUILTIN_OBJECT                                                      Handle        = 0x80320017
+	FWP_E_TOO_MANY_CALLOUTS                                                   Handle        = 0x80320018
+	FWP_E_NOTIFICATION_DROPPED                                                Handle        = 0x80320019
+	FWP_E_TRAFFIC_MISMATCH                                                    Handle        = 0x8032001A
+	FWP_E_INCOMPATIBLE_SA_STATE                                               Handle        = 0x8032001B
+	FWP_E_NULL_POINTER                                                        Handle        = 0x8032001C
+	FWP_E_INVALID_ENUMERATOR                                                  Handle        = 0x8032001D
+	FWP_E_INVALID_FLAGS                                                       Handle        = 0x8032001E
+	FWP_E_INVALID_NET_MASK                                                    Handle        = 0x8032001F
+	FWP_E_INVALID_RANGE                                                       Handle        = 0x80320020
+	FWP_E_INVALID_INTERVAL                                                    Handle        = 0x80320021
+	FWP_E_ZERO_LENGTH_ARRAY                                                   Handle        = 0x80320022
+	FWP_E_NULL_DISPLAY_NAME                                                   Handle        = 0x80320023
+	FWP_E_INVALID_ACTION_TYPE                                                 Handle        = 0x80320024
+	FWP_E_INVALID_WEIGHT                                                      Handle        = 0x80320025
+	FWP_E_MATCH_TYPE_MISMATCH                                                 Handle        = 0x80320026
+	FWP_E_TYPE_MISMATCH                                                       Handle        = 0x80320027
+	FWP_E_OUT_OF_BOUNDS                                                       Handle        = 0x80320028
+	FWP_E_RESERVED                                                            Handle        = 0x80320029
+	FWP_E_DUPLICATE_CONDITION                                                 Handle        = 0x8032002A
+	FWP_E_DUPLICATE_KEYMOD                                                    Handle        = 0x8032002B
+	FWP_E_ACTION_INCOMPATIBLE_WITH_LAYER                                      Handle        = 0x8032002C
+	FWP_E_ACTION_INCOMPATIBLE_WITH_SUBLAYER                                   Handle        = 0x8032002D
+	FWP_E_CONTEXT_INCOMPATIBLE_WITH_LAYER                                     Handle        = 0x8032002E
+	FWP_E_CONTEXT_INCOMPATIBLE_WITH_CALLOUT                                   Handle        = 0x8032002F
+	FWP_E_INCOMPATIBLE_AUTH_METHOD                                            Handle        = 0x80320030
+	FWP_E_INCOMPATIBLE_DH_GROUP                                               Handle        = 0x80320031
+	FWP_E_EM_NOT_SUPPORTED                                                    Handle        = 0x80320032
+	FWP_E_NEVER_MATCH                                                         Handle        = 0x80320033
+	FWP_E_PROVIDER_CONTEXT_MISMATCH                                           Handle        = 0x80320034
+	FWP_E_INVALID_PARAMETER                                                   Handle        = 0x80320035
+	FWP_E_TOO_MANY_SUBLAYERS                                                  Handle        = 0x80320036
+	FWP_E_CALLOUT_NOTIFICATION_FAILED                                         Handle        = 0x80320037
+	FWP_E_INVALID_AUTH_TRANSFORM                                              Handle        = 0x80320038
+	FWP_E_INVALID_CIPHER_TRANSFORM                                            Handle        = 0x80320039
+	FWP_E_INCOMPATIBLE_CIPHER_TRANSFORM                                       Handle        = 0x8032003A
+	FWP_E_INVALID_TRANSFORM_COMBINATION                                       Handle        = 0x8032003B
+	FWP_E_DUPLICATE_AUTH_METHOD                                               Handle        = 0x8032003C
+	FWP_E_INVALID_TUNNEL_ENDPOINT                                             Handle        = 0x8032003D
+	FWP_E_L2_DRIVER_NOT_READY                                                 Handle        = 0x8032003E
+	FWP_E_KEY_DICTATOR_ALREADY_REGISTERED                                     Handle        = 0x8032003F
+	FWP_E_KEY_DICTATION_INVALID_KEYING_MATERIAL                               Handle        = 0x80320040
+	FWP_E_CONNECTIONS_DISABLED                                                Handle        = 0x80320041
+	FWP_E_INVALID_DNS_NAME                                                    Handle        = 0x80320042
+	FWP_E_STILL_ON                                                            Handle        = 0x80320043
+	FWP_E_IKEEXT_NOT_RUNNING                                                  Handle        = 0x80320044
+	FWP_E_DROP_NOICMP                                                         Handle        = 0x80320104
+	WS_S_ASYNC                                                                Handle        = 0x003D0000
+	WS_S_END                                                                  Handle        = 0x003D0001
+	WS_E_INVALID_FORMAT                                                       Handle        = 0x803D0000
+	WS_E_OBJECT_FAULTED                                                       Handle        = 0x803D0001
+	WS_E_NUMERIC_OVERFLOW                                                     Handle        = 0x803D0002
+	WS_E_INVALID_OPERATION                                                    Handle        = 0x803D0003
+	WS_E_OPERATION_ABORTED                                                    Handle        = 0x803D0004
+	WS_E_ENDPOINT_ACCESS_DENIED                                               Handle        = 0x803D0005
+	WS_E_OPERATION_TIMED_OUT                                                  Handle        = 0x803D0006
+	WS_E_OPERATION_ABANDONED                                                  Handle        = 0x803D0007
+	WS_E_QUOTA_EXCEEDED                                                       Handle        = 0x803D0008
+	WS_E_NO_TRANSLATION_AVAILABLE                                             Handle        = 0x803D0009
+	WS_E_SECURITY_VERIFICATION_FAILURE                                        Handle        = 0x803D000A
+	WS_E_ADDRESS_IN_USE                                                       Handle        = 0x803D000B
+	WS_E_ADDRESS_NOT_AVAILABLE                                                Handle        = 0x803D000C
+	WS_E_ENDPOINT_NOT_FOUND                                                   Handle        = 0x803D000D
+	WS_E_ENDPOINT_NOT_AVAILABLE                                               Handle        = 0x803D000E
+	WS_E_ENDPOINT_FAILURE                                                     Handle        = 0x803D000F
+	WS_E_ENDPOINT_UNREACHABLE                                                 Handle        = 0x803D0010
+	WS_E_ENDPOINT_ACTION_NOT_SUPPORTED                                        Handle        = 0x803D0011
+	WS_E_ENDPOINT_TOO_BUSY                                                    Handle        = 0x803D0012
+	WS_E_ENDPOINT_FAULT_RECEIVED                                              Handle        = 0x803D0013
+	WS_E_ENDPOINT_DISCONNECTED                                                Handle        = 0x803D0014
+	WS_E_PROXY_FAILURE                                                        Handle        = 0x803D0015
+	WS_E_PROXY_ACCESS_DENIED                                                  Handle        = 0x803D0016
+	WS_E_NOT_SUPPORTED                                                        Handle        = 0x803D0017
+	WS_E_PROXY_REQUIRES_BASIC_AUTH                                            Handle        = 0x803D0018
+	WS_E_PROXY_REQUIRES_DIGEST_AUTH                                           Handle        = 0x803D0019
+	WS_E_PROXY_REQUIRES_NTLM_AUTH                                             Handle        = 0x803D001A
+	WS_E_PROXY_REQUIRES_NEGOTIATE_AUTH                                        Handle        = 0x803D001B
+	WS_E_SERVER_REQUIRES_BASIC_AUTH                                           Handle        = 0x803D001C
+	WS_E_SERVER_REQUIRES_DIGEST_AUTH                                          Handle        = 0x803D001D
+	WS_E_SERVER_REQUIRES_NTLM_AUTH                                            Handle        = 0x803D001E
+	WS_E_SERVER_REQUIRES_NEGOTIATE_AUTH                                       Handle        = 0x803D001F
+	WS_E_INVALID_ENDPOINT_URL                                                 Handle        = 0x803D0020
+	WS_E_OTHER                                                                Handle        = 0x803D0021
+	WS_E_SECURITY_TOKEN_EXPIRED                                               Handle        = 0x803D0022
+	WS_E_SECURITY_SYSTEM_FAILURE                                              Handle        = 0x803D0023
+	ERROR_NDIS_INTERFACE_CLOSING                                              syscall.Errno = 0x80340002
+	ERROR_NDIS_BAD_VERSION                                                    syscall.Errno = 0x80340004
+	ERROR_NDIS_BAD_CHARACTERISTICS                                            syscall.Errno = 0x80340005
+	ERROR_NDIS_ADAPTER_NOT_FOUND                                              syscall.Errno = 0x80340006
+	ERROR_NDIS_OPEN_FAILED                                                    syscall.Errno = 0x80340007
+	ERROR_NDIS_DEVICE_FAILED                                                  syscall.Errno = 0x80340008
+	ERROR_NDIS_MULTICAST_FULL                                                 syscall.Errno = 0x80340009
+	ERROR_NDIS_MULTICAST_EXISTS                                               syscall.Errno = 0x8034000A
+	ERROR_NDIS_MULTICAST_NOT_FOUND                                            syscall.Errno = 0x8034000B
+	ERROR_NDIS_REQUEST_ABORTED                                                syscall.Errno = 0x8034000C
+	ERROR_NDIS_RESET_IN_PROGRESS                                              syscall.Errno = 0x8034000D
+	ERROR_NDIS_NOT_SUPPORTED                                                  syscall.Errno = 0x803400BB
+	ERROR_NDIS_INVALID_PACKET                                                 syscall.Errno = 0x8034000F
+	ERROR_NDIS_ADAPTER_NOT_READY                                              syscall.Errno = 0x80340011
+	ERROR_NDIS_INVALID_LENGTH                                                 syscall.Errno = 0x80340014
+	ERROR_NDIS_INVALID_DATA                                                   syscall.Errno = 0x80340015
+	ERROR_NDIS_BUFFER_TOO_SHORT                                               syscall.Errno = 0x80340016
+	ERROR_NDIS_INVALID_OID                                                    syscall.Errno = 0x80340017
+	ERROR_NDIS_ADAPTER_REMOVED                                                syscall.Errno = 0x80340018
+	ERROR_NDIS_UNSUPPORTED_MEDIA                                              syscall.Errno = 0x80340019
+	ERROR_NDIS_GROUP_ADDRESS_IN_USE                                           syscall.Errno = 0x8034001A
+	ERROR_NDIS_FILE_NOT_FOUND                                                 syscall.Errno = 0x8034001B
+	ERROR_NDIS_ERROR_READING_FILE                                             syscall.Errno = 0x8034001C
+	ERROR_NDIS_ALREADY_MAPPED                                                 syscall.Errno = 0x8034001D
+	ERROR_NDIS_RESOURCE_CONFLICT                                              syscall.Errno = 0x8034001E
+	ERROR_NDIS_MEDIA_DISCONNECTED                                             syscall.Errno = 0x8034001F
+	ERROR_NDIS_INVALID_ADDRESS                                                syscall.Errno = 0x80340022
+	ERROR_NDIS_INVALID_DEVICE_REQUEST                                         syscall.Errno = 0x80340010
+	ERROR_NDIS_PAUSED                                                         syscall.Errno = 0x8034002A
+	ERROR_NDIS_INTERFACE_NOT_FOUND                                            syscall.Errno = 0x8034002B
+	ERROR_NDIS_UNSUPPORTED_REVISION                                           syscall.Errno = 0x8034002C
+	ERROR_NDIS_INVALID_PORT                                                   syscall.Errno = 0x8034002D
+	ERROR_NDIS_INVALID_PORT_STATE                                             syscall.Errno = 0x8034002E
+	ERROR_NDIS_LOW_POWER_STATE                                                syscall.Errno = 0x8034002F
+	ERROR_NDIS_REINIT_REQUIRED                                                syscall.Errno = 0x80340030
+	ERROR_NDIS_NO_QUEUES                                                      syscall.Errno = 0x80340031
+	ERROR_NDIS_DOT11_AUTO_CONFIG_ENABLED                                      syscall.Errno = 0x80342000
+	ERROR_NDIS_DOT11_MEDIA_IN_USE                                             syscall.Errno = 0x80342001
+	ERROR_NDIS_DOT11_POWER_STATE_INVALID                                      syscall.Errno = 0x80342002
+	ERROR_NDIS_PM_WOL_PATTERN_LIST_FULL                                       syscall.Errno = 0x80342003
+	ERROR_NDIS_PM_PROTOCOL_OFFLOAD_LIST_FULL                                  syscall.Errno = 0x80342004
+	ERROR_NDIS_DOT11_AP_CHANNEL_CURRENTLY_NOT_AVAILABLE                       syscall.Errno = 0x80342005
+	ERROR_NDIS_DOT11_AP_BAND_CURRENTLY_NOT_AVAILABLE                          syscall.Errno = 0x80342006
+	ERROR_NDIS_DOT11_AP_CHANNEL_NOT_ALLOWED                                   syscall.Errno = 0x80342007
+	ERROR_NDIS_DOT11_AP_BAND_NOT_ALLOWED                                      syscall.Errno = 0x80342008
+	ERROR_NDIS_INDICATION_REQUIRED                                            syscall.Errno = 0x00340001
+	ERROR_NDIS_OFFLOAD_POLICY                                                 syscall.Errno = 0xC034100F
+	ERROR_NDIS_OFFLOAD_CONNECTION_REJECTED                                    syscall.Errno = 0xC0341012
+	ERROR_NDIS_OFFLOAD_PATH_REJECTED                                          syscall.Errno = 0xC0341013
+	ERROR_HV_INVALID_HYPERCALL_CODE                                           syscall.Errno = 0xC0350002
+	ERROR_HV_INVALID_HYPERCALL_INPUT                                          syscall.Errno = 0xC0350003
+	ERROR_HV_INVALID_ALIGNMENT                                                syscall.Errno = 0xC0350004
+	ERROR_HV_INVALID_PARAMETER                                                syscall.Errno = 0xC0350005
+	ERROR_HV_ACCESS_DENIED                                                    syscall.Errno = 0xC0350006
+	ERROR_HV_INVALID_PARTITION_STATE                                          syscall.Errno = 0xC0350007
+	ERROR_HV_OPERATION_DENIED                                                 syscall.Errno = 0xC0350008
+	ERROR_HV_UNKNOWN_PROPERTY                                                 syscall.Errno = 0xC0350009
+	ERROR_HV_PROPERTY_VALUE_OUT_OF_RANGE                                      syscall.Errno = 0xC035000A
+	ERROR_HV_INSUFFICIENT_MEMORY                                              syscall.Errno = 0xC035000B
+	ERROR_HV_PARTITION_TOO_DEEP                                               syscall.Errno = 0xC035000C
+	ERROR_HV_INVALID_PARTITION_ID                                             syscall.Errno = 0xC035000D
+	ERROR_HV_INVALID_VP_INDEX                                                 syscall.Errno = 0xC035000E
+	ERROR_HV_INVALID_PORT_ID                                                  syscall.Errno = 0xC0350011
+	ERROR_HV_INVALID_CONNECTION_ID                                            syscall.Errno = 0xC0350012
+	ERROR_HV_INSUFFICIENT_BUFFERS                                             syscall.Errno = 0xC0350013
+	ERROR_HV_NOT_ACKNOWLEDGED                                                 syscall.Errno = 0xC0350014
+	ERROR_HV_INVALID_VP_STATE                                                 syscall.Errno = 0xC0350015
+	ERROR_HV_ACKNOWLEDGED                                                     syscall.Errno = 0xC0350016
+	ERROR_HV_INVALID_SAVE_RESTORE_STATE                                       syscall.Errno = 0xC0350017
+	ERROR_HV_INVALID_SYNIC_STATE                                              syscall.Errno = 0xC0350018
+	ERROR_HV_OBJECT_IN_USE                                                    syscall.Errno = 0xC0350019
+	ERROR_HV_INVALID_PROXIMITY_DOMAIN_INFO                                    syscall.Errno = 0xC035001A
+	ERROR_HV_NO_DATA                                                          syscall.Errno = 0xC035001B
+	ERROR_HV_INACTIVE                                                         syscall.Errno = 0xC035001C
+	ERROR_HV_NO_RESOURCES                                                     syscall.Errno = 0xC035001D
+	ERROR_HV_FEATURE_UNAVAILABLE                                              syscall.Errno = 0xC035001E
+	ERROR_HV_INSUFFICIENT_BUFFER                                              syscall.Errno = 0xC0350033
+	ERROR_HV_INSUFFICIENT_DEVICE_DOMAINS                                      syscall.Errno = 0xC0350038
+	ERROR_HV_CPUID_FEATURE_VALIDATION                                         syscall.Errno = 0xC035003C
+	ERROR_HV_CPUID_XSAVE_FEATURE_VALIDATION                                   syscall.Errno = 0xC035003D
+	ERROR_HV_PROCESSOR_STARTUP_TIMEOUT                                        syscall.Errno = 0xC035003E
+	ERROR_HV_SMX_ENABLED                                                      syscall.Errno = 0xC035003F
+	ERROR_HV_INVALID_LP_INDEX                                                 syscall.Errno = 0xC0350041
+	ERROR_HV_INVALID_REGISTER_VALUE                                           syscall.Errno = 0xC0350050
+	ERROR_HV_INVALID_VTL_STATE                                                syscall.Errno = 0xC0350051
+	ERROR_HV_NX_NOT_DETECTED                                                  syscall.Errno = 0xC0350055
+	ERROR_HV_INVALID_DEVICE_ID                                                syscall.Errno = 0xC0350057
+	ERROR_HV_INVALID_DEVICE_STATE                                             syscall.Errno = 0xC0350058
+	ERROR_HV_PENDING_PAGE_REQUESTS                                            syscall.Errno = 0x00350059
+	ERROR_HV_PAGE_REQUEST_INVALID                                             syscall.Errno = 0xC0350060
+	ERROR_HV_INVALID_CPU_GROUP_ID                                             syscall.Errno = 0xC035006F
+	ERROR_HV_INVALID_CPU_GROUP_STATE                                          syscall.Errno = 0xC0350070
+	ERROR_HV_OPERATION_FAILED                                                 syscall.Errno = 0xC0350071
+	ERROR_HV_NOT_ALLOWED_WITH_NESTED_VIRT_ACTIVE                              syscall.Errno = 0xC0350072
+	ERROR_HV_INSUFFICIENT_ROOT_MEMORY                                         syscall.Errno = 0xC0350073
+	ERROR_HV_NOT_PRESENT                                                      syscall.Errno = 0xC0351000
+	ERROR_VID_DUPLICATE_HANDLER                                               syscall.Errno = 0xC0370001
+	ERROR_VID_TOO_MANY_HANDLERS                                               syscall.Errno = 0xC0370002
+	ERROR_VID_QUEUE_FULL                                                      syscall.Errno = 0xC0370003
+	ERROR_VID_HANDLER_NOT_PRESENT                                             syscall.Errno = 0xC0370004
+	ERROR_VID_INVALID_OBJECT_NAME                                             syscall.Errno = 0xC0370005
+	ERROR_VID_PARTITION_NAME_TOO_LONG                                         syscall.Errno = 0xC0370006
+	ERROR_VID_MESSAGE_QUEUE_NAME_TOO_LONG                                     syscall.Errno = 0xC0370007
+	ERROR_VID_PARTITION_ALREADY_EXISTS                                        syscall.Errno = 0xC0370008
+	ERROR_VID_PARTITION_DOES_NOT_EXIST                                        syscall.Errno = 0xC0370009
+	ERROR_VID_PARTITION_NAME_NOT_FOUND                                        syscall.Errno = 0xC037000A
+	ERROR_VID_MESSAGE_QUEUE_ALREADY_EXISTS                                    syscall.Errno = 0xC037000B
+	ERROR_VID_EXCEEDED_MBP_ENTRY_MAP_LIMIT                                    syscall.Errno = 0xC037000C
+	ERROR_VID_MB_STILL_REFERENCED                                             syscall.Errno = 0xC037000D
+	ERROR_VID_CHILD_GPA_PAGE_SET_CORRUPTED                                    syscall.Errno = 0xC037000E
+	ERROR_VID_INVALID_NUMA_SETTINGS                                           syscall.Errno = 0xC037000F
+	ERROR_VID_INVALID_NUMA_NODE_INDEX                                         syscall.Errno = 0xC0370010
+	ERROR_VID_NOTIFICATION_QUEUE_ALREADY_ASSOCIATED                           syscall.Errno = 0xC0370011
+	ERROR_VID_INVALID_MEMORY_BLOCK_HANDLE                                     syscall.Errno = 0xC0370012
+	ERROR_VID_PAGE_RANGE_OVERFLOW                                             syscall.Errno = 0xC0370013
+	ERROR_VID_INVALID_MESSAGE_QUEUE_HANDLE                                    syscall.Errno = 0xC0370014
+	ERROR_VID_INVALID_GPA_RANGE_HANDLE                                        syscall.Errno = 0xC0370015
+	ERROR_VID_NO_MEMORY_BLOCK_NOTIFICATION_QUEUE                              syscall.Errno = 0xC0370016
+	ERROR_VID_MEMORY_BLOCK_LOCK_COUNT_EXCEEDED                                syscall.Errno = 0xC0370017
+	ERROR_VID_INVALID_PPM_HANDLE                                              syscall.Errno = 0xC0370018
+	ERROR_VID_MBPS_ARE_LOCKED                                                 syscall.Errno = 0xC0370019
+	ERROR_VID_MESSAGE_QUEUE_CLOSED                                            syscall.Errno = 0xC037001A
+	ERROR_VID_VIRTUAL_PROCESSOR_LIMIT_EXCEEDED                                syscall.Errno = 0xC037001B
+	ERROR_VID_STOP_PENDING                                                    syscall.Errno = 0xC037001C
+	ERROR_VID_INVALID_PROCESSOR_STATE                                         syscall.Errno = 0xC037001D
+	ERROR_VID_EXCEEDED_KM_CONTEXT_COUNT_LIMIT                                 syscall.Errno = 0xC037001E
+	ERROR_VID_KM_INTERFACE_ALREADY_INITIALIZED                                syscall.Errno = 0xC037001F
+	ERROR_VID_MB_PROPERTY_ALREADY_SET_RESET                                   syscall.Errno = 0xC0370020
+	ERROR_VID_MMIO_RANGE_DESTROYED                                            syscall.Errno = 0xC0370021
+	ERROR_VID_INVALID_CHILD_GPA_PAGE_SET                                      syscall.Errno = 0xC0370022
+	ERROR_VID_RESERVE_PAGE_SET_IS_BEING_USED                                  syscall.Errno = 0xC0370023
+	ERROR_VID_RESERVE_PAGE_SET_TOO_SMALL                                      syscall.Errno = 0xC0370024
+	ERROR_VID_MBP_ALREADY_LOCKED_USING_RESERVED_PAGE                          syscall.Errno = 0xC0370025
+	ERROR_VID_MBP_COUNT_EXCEEDED_LIMIT                                        syscall.Errno = 0xC0370026
+	ERROR_VID_SAVED_STATE_CORRUPT                                             syscall.Errno = 0xC0370027
+	ERROR_VID_SAVED_STATE_UNRECOGNIZED_ITEM                                   syscall.Errno = 0xC0370028
+	ERROR_VID_SAVED_STATE_INCOMPATIBLE                                        syscall.Errno = 0xC0370029
+	ERROR_VID_VTL_ACCESS_DENIED                                               syscall.Errno = 0xC037002A
+	ERROR_VMCOMPUTE_TERMINATED_DURING_START                                   syscall.Errno = 0xC0370100
+	ERROR_VMCOMPUTE_IMAGE_MISMATCH                                            syscall.Errno = 0xC0370101
+	ERROR_VMCOMPUTE_HYPERV_NOT_INSTALLED                                      syscall.Errno = 0xC0370102
+	ERROR_VMCOMPUTE_OPERATION_PENDING                                         syscall.Errno = 0xC0370103
+	ERROR_VMCOMPUTE_TOO_MANY_NOTIFICATIONS                                    syscall.Errno = 0xC0370104
+	ERROR_VMCOMPUTE_INVALID_STATE                                             syscall.Errno = 0xC0370105
+	ERROR_VMCOMPUTE_UNEXPECTED_EXIT                                           syscall.Errno = 0xC0370106
+	ERROR_VMCOMPUTE_TERMINATED                                                syscall.Errno = 0xC0370107
+	ERROR_VMCOMPUTE_CONNECT_FAILED                                            syscall.Errno = 0xC0370108
+	ERROR_VMCOMPUTE_TIMEOUT                                                   syscall.Errno = 0xC0370109
+	ERROR_VMCOMPUTE_CONNECTION_CLOSED                                         syscall.Errno = 0xC037010A
+	ERROR_VMCOMPUTE_UNKNOWN_MESSAGE                                           syscall.Errno = 0xC037010B
+	ERROR_VMCOMPUTE_UNSUPPORTED_PROTOCOL_VERSION                              syscall.Errno = 0xC037010C
+	ERROR_VMCOMPUTE_INVALID_JSON                                              syscall.Errno = 0xC037010D
+	ERROR_VMCOMPUTE_SYSTEM_NOT_FOUND                                          syscall.Errno = 0xC037010E
+	ERROR_VMCOMPUTE_SYSTEM_ALREADY_EXISTS                                     syscall.Errno = 0xC037010F
+	ERROR_VMCOMPUTE_SYSTEM_ALREADY_STOPPED                                    syscall.Errno = 0xC0370110
+	ERROR_VMCOMPUTE_PROTOCOL_ERROR                                            syscall.Errno = 0xC0370111
+	ERROR_VMCOMPUTE_INVALID_LAYER                                             syscall.Errno = 0xC0370112
+	ERROR_VMCOMPUTE_WINDOWS_INSIDER_REQUIRED                                  syscall.Errno = 0xC0370113
+	HCS_E_TERMINATED_DURING_START                                             Handle        = 0x80370100
+	HCS_E_IMAGE_MISMATCH                                                      Handle        = 0x80370101
+	HCS_E_HYPERV_NOT_INSTALLED                                                Handle        = 0x80370102
+	HCS_E_INVALID_STATE                                                       Handle        = 0x80370105
+	HCS_E_UNEXPECTED_EXIT                                                     Handle        = 0x80370106
+	HCS_E_TERMINATED                                                          Handle        = 0x80370107
+	HCS_E_CONNECT_FAILED                                                      Handle        = 0x80370108
+	HCS_E_CONNECTION_TIMEOUT                                                  Handle        = 0x80370109
+	HCS_E_CONNECTION_CLOSED                                                   Handle        = 0x8037010A
+	HCS_E_UNKNOWN_MESSAGE                                                     Handle        = 0x8037010B
+	HCS_E_UNSUPPORTED_PROTOCOL_VERSION                                        Handle        = 0x8037010C
+	HCS_E_INVALID_JSON                                                        Handle        = 0x8037010D
+	HCS_E_SYSTEM_NOT_FOUND                                                    Handle        = 0x8037010E
+	HCS_E_SYSTEM_ALREADY_EXISTS                                               Handle        = 0x8037010F
+	HCS_E_SYSTEM_ALREADY_STOPPED                                              Handle        = 0x80370110
+	HCS_E_PROTOCOL_ERROR                                                      Handle        = 0x80370111
+	HCS_E_INVALID_LAYER                                                       Handle        = 0x80370112
+	HCS_E_WINDOWS_INSIDER_REQUIRED                                            Handle        = 0x80370113
+	HCS_E_SERVICE_NOT_AVAILABLE                                               Handle        = 0x80370114
+	HCS_E_OPERATION_NOT_STARTED                                               Handle        = 0x80370115
+	HCS_E_OPERATION_ALREADY_STARTED                                           Handle        = 0x80370116
+	HCS_E_OPERATION_PENDING                                                   Handle        = 0x80370117
+	HCS_E_OPERATION_TIMEOUT                                                   Handle        = 0x80370118
+	HCS_E_OPERATION_SYSTEM_CALLBACK_ALREADY_SET                               Handle        = 0x80370119
+	HCS_E_OPERATION_RESULT_ALLOCATION_FAILED                                  Handle        = 0x8037011A
+	HCS_E_ACCESS_DENIED                                                       Handle        = 0x8037011B
+	HCS_E_GUEST_CRITICAL_ERROR                                                Handle        = 0x8037011C
+	ERROR_VNET_VIRTUAL_SWITCH_NAME_NOT_FOUND                                  syscall.Errno = 0xC0370200
+	ERROR_VID_REMOTE_NODE_PARENT_GPA_PAGES_USED                               syscall.Errno = 0x80370001
+	WHV_E_UNKNOWN_CAPABILITY                                                  Handle        = 0x80370300
+	WHV_E_INSUFFICIENT_BUFFER                                                 Handle        = 0x80370301
+	WHV_E_UNKNOWN_PROPERTY                                                    Handle        = 0x80370302
+	WHV_E_UNSUPPORTED_HYPERVISOR_CONFIG                                       Handle        = 0x80370303
+	WHV_E_INVALID_PARTITION_CONFIG                                            Handle        = 0x80370304
+	WHV_E_GPA_RANGE_NOT_FOUND                                                 Handle        = 0x80370305
+	WHV_E_VP_ALREADY_EXISTS                                                   Handle        = 0x80370306
+	WHV_E_VP_DOES_NOT_EXIST                                                   Handle        = 0x80370307
+	WHV_E_INVALID_VP_STATE                                                    Handle        = 0x80370308
+	WHV_E_INVALID_VP_REGISTER_NAME                                            Handle        = 0x80370309
+	ERROR_VSMB_SAVED_STATE_FILE_NOT_FOUND                                     syscall.Errno = 0xC0370400
+	ERROR_VSMB_SAVED_STATE_CORRUPT                                            syscall.Errno = 0xC0370401
+	ERROR_VOLMGR_INCOMPLETE_REGENERATION                                      syscall.Errno = 0x80380001
+	ERROR_VOLMGR_INCOMPLETE_DISK_MIGRATION                                    syscall.Errno = 0x80380002
+	ERROR_VOLMGR_DATABASE_FULL                                                syscall.Errno = 0xC0380001
+	ERROR_VOLMGR_DISK_CONFIGURATION_CORRUPTED                                 syscall.Errno = 0xC0380002
+	ERROR_VOLMGR_DISK_CONFIGURATION_NOT_IN_SYNC                               syscall.Errno = 0xC0380003
+	ERROR_VOLMGR_PACK_CONFIG_UPDATE_FAILED                                    syscall.Errno = 0xC0380004
+	ERROR_VOLMGR_DISK_CONTAINS_NON_SIMPLE_VOLUME                              syscall.Errno = 0xC0380005
+	ERROR_VOLMGR_DISK_DUPLICATE                                               syscall.Errno = 0xC0380006
+	ERROR_VOLMGR_DISK_DYNAMIC                                                 syscall.Errno = 0xC0380007
+	ERROR_VOLMGR_DISK_ID_INVALID                                              syscall.Errno = 0xC0380008
+	ERROR_VOLMGR_DISK_INVALID                                                 syscall.Errno = 0xC0380009
+	ERROR_VOLMGR_DISK_LAST_VOTER                                              syscall.Errno = 0xC038000A
+	ERROR_VOLMGR_DISK_LAYOUT_INVALID                                          syscall.Errno = 0xC038000B
+	ERROR_VOLMGR_DISK_LAYOUT_NON_BASIC_BETWEEN_BASIC_PARTITIONS               syscall.Errno = 0xC038000C
+	ERROR_VOLMGR_DISK_LAYOUT_NOT_CYLINDER_ALIGNED                             syscall.Errno = 0xC038000D
+	ERROR_VOLMGR_DISK_LAYOUT_PARTITIONS_TOO_SMALL                             syscall.Errno = 0xC038000E
+	ERROR_VOLMGR_DISK_LAYOUT_PRIMARY_BETWEEN_LOGICAL_PARTITIONS               syscall.Errno = 0xC038000F
+	ERROR_VOLMGR_DISK_LAYOUT_TOO_MANY_PARTITIONS                              syscall.Errno = 0xC0380010
+	ERROR_VOLMGR_DISK_MISSING                                                 syscall.Errno = 0xC0380011
+	ERROR_VOLMGR_DISK_NOT_EMPTY                                               syscall.Errno = 0xC0380012
+	ERROR_VOLMGR_DISK_NOT_ENOUGH_SPACE                                        syscall.Errno = 0xC0380013
+	ERROR_VOLMGR_DISK_REVECTORING_FAILED                                      syscall.Errno = 0xC0380014
+	ERROR_VOLMGR_DISK_SECTOR_SIZE_INVALID                                     syscall.Errno = 0xC0380015
+	ERROR_VOLMGR_DISK_SET_NOT_CONTAINED                                       syscall.Errno = 0xC0380016
+	ERROR_VOLMGR_DISK_USED_BY_MULTIPLE_MEMBERS                                syscall.Errno = 0xC0380017
+	ERROR_VOLMGR_DISK_USED_BY_MULTIPLE_PLEXES                                 syscall.Errno = 0xC0380018
+	ERROR_VOLMGR_DYNAMIC_DISK_NOT_SUPPORTED                                   syscall.Errno = 0xC0380019
+	ERROR_VOLMGR_EXTENT_ALREADY_USED                                          syscall.Errno = 0xC038001A
+	ERROR_VOLMGR_EXTENT_NOT_CONTIGUOUS                                        syscall.Errno = 0xC038001B
+	ERROR_VOLMGR_EXTENT_NOT_IN_PUBLIC_REGION                                  syscall.Errno = 0xC038001C
+	ERROR_VOLMGR_EXTENT_NOT_SECTOR_ALIGNED                                    syscall.Errno = 0xC038001D
+	ERROR_VOLMGR_EXTENT_OVERLAPS_EBR_PARTITION                                syscall.Errno = 0xC038001E
+	ERROR_VOLMGR_EXTENT_VOLUME_LENGTHS_DO_NOT_MATCH                           syscall.Errno = 0xC038001F
+	ERROR_VOLMGR_FAULT_TOLERANT_NOT_SUPPORTED                                 syscall.Errno = 0xC0380020
+	ERROR_VOLMGR_INTERLEAVE_LENGTH_INVALID                                    syscall.Errno = 0xC0380021
+	ERROR_VOLMGR_MAXIMUM_REGISTERED_USERS                                     syscall.Errno = 0xC0380022
+	ERROR_VOLMGR_MEMBER_IN_SYNC                                               syscall.Errno = 0xC0380023
+	ERROR_VOLMGR_MEMBER_INDEX_DUPLICATE                                       syscall.Errno = 0xC0380024
+	ERROR_VOLMGR_MEMBER_INDEX_INVALID                                         syscall.Errno = 0xC0380025
+	ERROR_VOLMGR_MEMBER_MISSING                                               syscall.Errno = 0xC0380026
+	ERROR_VOLMGR_MEMBER_NOT_DETACHED                                          syscall.Errno = 0xC0380027
+	ERROR_VOLMGR_MEMBER_REGENERATING                                          syscall.Errno = 0xC0380028
+	ERROR_VOLMGR_ALL_DISKS_FAILED                                             syscall.Errno = 0xC0380029
+	ERROR_VOLMGR_NO_REGISTERED_USERS                                          syscall.Errno = 0xC038002A
+	ERROR_VOLMGR_NO_SUCH_USER                                                 syscall.Errno = 0xC038002B
+	ERROR_VOLMGR_NOTIFICATION_RESET                                           syscall.Errno = 0xC038002C
+	ERROR_VOLMGR_NUMBER_OF_MEMBERS_INVALID                                    syscall.Errno = 0xC038002D
+	ERROR_VOLMGR_NUMBER_OF_PLEXES_INVALID                                     syscall.Errno = 0xC038002E
+	ERROR_VOLMGR_PACK_DUPLICATE                                               syscall.Errno = 0xC038002F
+	ERROR_VOLMGR_PACK_ID_INVALID                                              syscall.Errno = 0xC0380030
+	ERROR_VOLMGR_PACK_INVALID                                                 syscall.Errno = 0xC0380031
+	ERROR_VOLMGR_PACK_NAME_INVALID                                            syscall.Errno = 0xC0380032
+	ERROR_VOLMGR_PACK_OFFLINE                                                 syscall.Errno = 0xC0380033
+	ERROR_VOLMGR_PACK_HAS_QUORUM                                              syscall.Errno = 0xC0380034
+	ERROR_VOLMGR_PACK_WITHOUT_QUORUM                                          syscall.Errno = 0xC0380035
+	ERROR_VOLMGR_PARTITION_STYLE_INVALID                                      syscall.Errno = 0xC0380036
+	ERROR_VOLMGR_PARTITION_UPDATE_FAILED                                      syscall.Errno = 0xC0380037
+	ERROR_VOLMGR_PLEX_IN_SYNC                                                 syscall.Errno = 0xC0380038
+	ERROR_VOLMGR_PLEX_INDEX_DUPLICATE                                         syscall.Errno = 0xC0380039
+	ERROR_VOLMGR_PLEX_INDEX_INVALID                                           syscall.Errno = 0xC038003A
+	ERROR_VOLMGR_PLEX_LAST_ACTIVE                                             syscall.Errno = 0xC038003B
+	ERROR_VOLMGR_PLEX_MISSING                                                 syscall.Errno = 0xC038003C
+	ERROR_VOLMGR_PLEX_REGENERATING                                            syscall.Errno = 0xC038003D
+	ERROR_VOLMGR_PLEX_TYPE_INVALID                                            syscall.Errno = 0xC038003E
+	ERROR_VOLMGR_PLEX_NOT_RAID5                                               syscall.Errno = 0xC038003F
+	ERROR_VOLMGR_PLEX_NOT_SIMPLE                                              syscall.Errno = 0xC0380040
+	ERROR_VOLMGR_STRUCTURE_SIZE_INVALID                                       syscall.Errno = 0xC0380041
+	ERROR_VOLMGR_TOO_MANY_NOTIFICATION_REQUESTS                               syscall.Errno = 0xC0380042
+	ERROR_VOLMGR_TRANSACTION_IN_PROGRESS                                      syscall.Errno = 0xC0380043
+	ERROR_VOLMGR_UNEXPECTED_DISK_LAYOUT_CHANGE                                syscall.Errno = 0xC0380044
+	ERROR_VOLMGR_VOLUME_CONTAINS_MISSING_DISK                                 syscall.Errno = 0xC0380045
+	ERROR_VOLMGR_VOLUME_ID_INVALID                                            syscall.Errno = 0xC0380046
+	ERROR_VOLMGR_VOLUME_LENGTH_INVALID                                        syscall.Errno = 0xC0380047
+	ERROR_VOLMGR_VOLUME_LENGTH_NOT_SECTOR_SIZE_MULTIPLE                       syscall.Errno = 0xC0380048
+	ERROR_VOLMGR_VOLUME_NOT_MIRRORED                                          syscall.Errno = 0xC0380049
+	ERROR_VOLMGR_VOLUME_NOT_RETAINED                                          syscall.Errno = 0xC038004A
+	ERROR_VOLMGR_VOLUME_OFFLINE                                               syscall.Errno = 0xC038004B
+	ERROR_VOLMGR_VOLUME_RETAINED                                              syscall.Errno = 0xC038004C
+	ERROR_VOLMGR_NUMBER_OF_EXTENTS_INVALID                                    syscall.Errno = 0xC038004D
+	ERROR_VOLMGR_DIFFERENT_SECTOR_SIZE                                        syscall.Errno = 0xC038004E
+	ERROR_VOLMGR_BAD_BOOT_DISK                                                syscall.Errno = 0xC038004F
+	ERROR_VOLMGR_PACK_CONFIG_OFFLINE                                          syscall.Errno = 0xC0380050
+	ERROR_VOLMGR_PACK_CONFIG_ONLINE                                           syscall.Errno = 0xC0380051
+	ERROR_VOLMGR_NOT_PRIMARY_PACK                                             syscall.Errno = 0xC0380052
+	ERROR_VOLMGR_PACK_LOG_UPDATE_FAILED                                       syscall.Errno = 0xC0380053
+	ERROR_VOLMGR_NUMBER_OF_DISKS_IN_PLEX_INVALID                              syscall.Errno = 0xC0380054
+	ERROR_VOLMGR_NUMBER_OF_DISKS_IN_MEMBER_INVALID                            syscall.Errno = 0xC0380055
+	ERROR_VOLMGR_VOLUME_MIRRORED                                              syscall.Errno = 0xC0380056
+	ERROR_VOLMGR_PLEX_NOT_SIMPLE_SPANNED                                      syscall.Errno = 0xC0380057
+	ERROR_VOLMGR_NO_VALID_LOG_COPIES                                          syscall.Errno = 0xC0380058
+	ERROR_VOLMGR_PRIMARY_PACK_PRESENT                                         syscall.Errno = 0xC0380059
+	ERROR_VOLMGR_NUMBER_OF_DISKS_INVALID                                      syscall.Errno = 0xC038005A
+	ERROR_VOLMGR_MIRROR_NOT_SUPPORTED                                         syscall.Errno = 0xC038005B
+	ERROR_VOLMGR_RAID5_NOT_SUPPORTED                                          syscall.Errno = 0xC038005C
+	ERROR_BCD_NOT_ALL_ENTRIES_IMPORTED                                        syscall.Errno = 0x80390001
+	ERROR_BCD_TOO_MANY_ELEMENTS                                               syscall.Errno = 0xC0390002
+	ERROR_BCD_NOT_ALL_ENTRIES_SYNCHRONIZED                                    syscall.Errno = 0x80390003
+	ERROR_VHD_DRIVE_FOOTER_MISSING                                            syscall.Errno = 0xC03A0001
+	ERROR_VHD_DRIVE_FOOTER_CHECKSUM_MISMATCH                                  syscall.Errno = 0xC03A0002
+	ERROR_VHD_DRIVE_FOOTER_CORRUPT                                            syscall.Errno = 0xC03A0003
+	ERROR_VHD_FORMAT_UNKNOWN                                                  syscall.Errno = 0xC03A0004
+	ERROR_VHD_FORMAT_UNSUPPORTED_VERSION                                      syscall.Errno = 0xC03A0005
+	ERROR_VHD_SPARSE_HEADER_CHECKSUM_MISMATCH                                 syscall.Errno = 0xC03A0006
+	ERROR_VHD_SPARSE_HEADER_UNSUPPORTED_VERSION                               syscall.Errno = 0xC03A0007
+	ERROR_VHD_SPARSE_HEADER_CORRUPT                                           syscall.Errno = 0xC03A0008
+	ERROR_VHD_BLOCK_ALLOCATION_FAILURE                                        syscall.Errno = 0xC03A0009
+	ERROR_VHD_BLOCK_ALLOCATION_TABLE_CORRUPT                                  syscall.Errno = 0xC03A000A
+	ERROR_VHD_INVALID_BLOCK_SIZE                                              syscall.Errno = 0xC03A000B
+	ERROR_VHD_BITMAP_MISMATCH                                                 syscall.Errno = 0xC03A000C
+	ERROR_VHD_PARENT_VHD_NOT_FOUND                                            syscall.Errno = 0xC03A000D
+	ERROR_VHD_CHILD_PARENT_ID_MISMATCH                                        syscall.Errno = 0xC03A000E
+	ERROR_VHD_CHILD_PARENT_TIMESTAMP_MISMATCH                                 syscall.Errno = 0xC03A000F
+	ERROR_VHD_METADATA_READ_FAILURE                                           syscall.Errno = 0xC03A0010
+	ERROR_VHD_METADATA_WRITE_FAILURE                                          syscall.Errno = 0xC03A0011
+	ERROR_VHD_INVALID_SIZE                                                    syscall.Errno = 0xC03A0012
+	ERROR_VHD_INVALID_FILE_SIZE                                               syscall.Errno = 0xC03A0013
+	ERROR_VIRTDISK_PROVIDER_NOT_FOUND                                         syscall.Errno = 0xC03A0014
+	ERROR_VIRTDISK_NOT_VIRTUAL_DISK                                           syscall.Errno = 0xC03A0015
+	ERROR_VHD_PARENT_VHD_ACCESS_DENIED                                        syscall.Errno = 0xC03A0016
+	ERROR_VHD_CHILD_PARENT_SIZE_MISMATCH                                      syscall.Errno = 0xC03A0017
+	ERROR_VHD_DIFFERENCING_CHAIN_CYCLE_DETECTED                               syscall.Errno = 0xC03A0018
+	ERROR_VHD_DIFFERENCING_CHAIN_ERROR_IN_PARENT                              syscall.Errno = 0xC03A0019
+	ERROR_VIRTUAL_DISK_LIMITATION                                             syscall.Errno = 0xC03A001A
+	ERROR_VHD_INVALID_TYPE                                                    syscall.Errno = 0xC03A001B
+	ERROR_VHD_INVALID_STATE                                                   syscall.Errno = 0xC03A001C
+	ERROR_VIRTDISK_UNSUPPORTED_DISK_SECTOR_SIZE                               syscall.Errno = 0xC03A001D
+	ERROR_VIRTDISK_DISK_ALREADY_OWNED                                         syscall.Errno = 0xC03A001E
+	ERROR_VIRTDISK_DISK_ONLINE_AND_WRITABLE                                   syscall.Errno = 0xC03A001F
+	ERROR_CTLOG_TRACKING_NOT_INITIALIZED                                      syscall.Errno = 0xC03A0020
+	ERROR_CTLOG_LOGFILE_SIZE_EXCEEDED_MAXSIZE                                 syscall.Errno = 0xC03A0021
+	ERROR_CTLOG_VHD_CHANGED_OFFLINE                                           syscall.Errno = 0xC03A0022
+	ERROR_CTLOG_INVALID_TRACKING_STATE                                        syscall.Errno = 0xC03A0023
+	ERROR_CTLOG_INCONSISTENT_TRACKING_FILE                                    syscall.Errno = 0xC03A0024
+	ERROR_VHD_RESIZE_WOULD_TRUNCATE_DATA                                      syscall.Errno = 0xC03A0025
+	ERROR_VHD_COULD_NOT_COMPUTE_MINIMUM_VIRTUAL_SIZE                          syscall.Errno = 0xC03A0026
+	ERROR_VHD_ALREADY_AT_OR_BELOW_MINIMUM_VIRTUAL_SIZE                        syscall.Errno = 0xC03A0027
+	ERROR_VHD_METADATA_FULL                                                   syscall.Errno = 0xC03A0028
+	ERROR_VHD_INVALID_CHANGE_TRACKING_ID                                      syscall.Errno = 0xC03A0029
+	ERROR_VHD_CHANGE_TRACKING_DISABLED                                        syscall.Errno = 0xC03A002A
+	ERROR_VHD_MISSING_CHANGE_TRACKING_INFORMATION                             syscall.Errno = 0xC03A0030
+	ERROR_QUERY_STORAGE_ERROR                                                 syscall.Errno = 0x803A0001
+	HCN_E_NETWORK_NOT_FOUND                                                   Handle        = 0x803B0001
+	HCN_E_ENDPOINT_NOT_FOUND                                                  Handle        = 0x803B0002
+	HCN_E_LAYER_NOT_FOUND                                                     Handle        = 0x803B0003
+	HCN_E_SWITCH_NOT_FOUND                                                    Handle        = 0x803B0004
+	HCN_E_SUBNET_NOT_FOUND                                                    Handle        = 0x803B0005
+	HCN_E_ADAPTER_NOT_FOUND                                                   Handle        = 0x803B0006
+	HCN_E_PORT_NOT_FOUND                                                      Handle        = 0x803B0007
+	HCN_E_POLICY_NOT_FOUND                                                    Handle        = 0x803B0008
+	HCN_E_VFP_PORTSETTING_NOT_FOUND                                           Handle        = 0x803B0009
+	HCN_E_INVALID_NETWORK                                                     Handle        = 0x803B000A
+	HCN_E_INVALID_NETWORK_TYPE                                                Handle        = 0x803B000B
+	HCN_E_INVALID_ENDPOINT                                                    Handle        = 0x803B000C
+	HCN_E_INVALID_POLICY                                                      Handle        = 0x803B000D
+	HCN_E_INVALID_POLICY_TYPE                                                 Handle        = 0x803B000E
+	HCN_E_INVALID_REMOTE_ENDPOINT_OPERATION                                   Handle        = 0x803B000F
+	HCN_E_NETWORK_ALREADY_EXISTS                                              Handle        = 0x803B0010
+	HCN_E_LAYER_ALREADY_EXISTS                                                Handle        = 0x803B0011
+	HCN_E_POLICY_ALREADY_EXISTS                                               Handle        = 0x803B0012
+	HCN_E_PORT_ALREADY_EXISTS                                                 Handle        = 0x803B0013
+	HCN_E_ENDPOINT_ALREADY_ATTACHED                                           Handle        = 0x803B0014
+	HCN_E_REQUEST_UNSUPPORTED                                                 Handle        = 0x803B0015
+	HCN_E_MAPPING_NOT_SUPPORTED                                               Handle        = 0x803B0016
+	HCN_E_DEGRADED_OPERATION                                                  Handle        = 0x803B0017
+	HCN_E_SHARED_SWITCH_MODIFICATION                                          Handle        = 0x803B0018
+	HCN_E_GUID_CONVERSION_FAILURE                                             Handle        = 0x803B0019
+	HCN_E_REGKEY_FAILURE                                                      Handle        = 0x803B001A
+	HCN_E_INVALID_JSON                                                        Handle        = 0x803B001B
+	HCN_E_INVALID_JSON_REFERENCE                                              Handle        = 0x803B001C
+	HCN_E_ENDPOINT_SHARING_DISABLED                                           Handle        = 0x803B001D
+	HCN_E_INVALID_IP                                                          Handle        = 0x803B001E
+	HCN_E_SWITCH_EXTENSION_NOT_FOUND                                          Handle        = 0x803B001F
+	HCN_E_MANAGER_STOPPED                                                     Handle        = 0x803B0020
+	GCN_E_MODULE_NOT_FOUND                                                    Handle        = 0x803B0021
+	GCN_E_NO_REQUEST_HANDLERS                                                 Handle        = 0x803B0022
+	GCN_E_REQUEST_UNSUPPORTED                                                 Handle        = 0x803B0023
+	GCN_E_RUNTIMEKEYS_FAILED                                                  Handle        = 0x803B0024
+	GCN_E_NETADAPTER_TIMEOUT                                                  Handle        = 0x803B0025
+	GCN_E_NETADAPTER_NOT_FOUND                                                Handle        = 0x803B0026
+	GCN_E_NETCOMPARTMENT_NOT_FOUND                                            Handle        = 0x803B0027
+	GCN_E_NETINTERFACE_NOT_FOUND                                              Handle        = 0x803B0028
+	GCN_E_DEFAULTNAMESPACE_EXISTS                                             Handle        = 0x803B0029
+	SDIAG_E_CANCELLED                                                         syscall.Errno = 0x803C0100
+	SDIAG_E_SCRIPT                                                            syscall.Errno = 0x803C0101
+	SDIAG_E_POWERSHELL                                                        syscall.Errno = 0x803C0102
+	SDIAG_E_MANAGEDHOST                                                       syscall.Errno = 0x803C0103
+	SDIAG_E_NOVERIFIER                                                        syscall.Errno = 0x803C0104
+	SDIAG_S_CANNOTRUN                                                         syscall.Errno = 0x003C0105
+	SDIAG_E_DISABLED                                                          syscall.Errno = 0x803C0106
+	SDIAG_E_TRUST                                                             syscall.Errno = 0x803C0107
+	SDIAG_E_CANNOTRUN                                                         syscall.Errno = 0x803C0108
+	SDIAG_E_VERSION                                                           syscall.Errno = 0x803C0109
+	SDIAG_E_RESOURCE                                                          syscall.Errno = 0x803C010A
+	SDIAG_E_ROOTCAUSE                                                         syscall.Errno = 0x803C010B
+	WPN_E_CHANNEL_CLOSED                                                      Handle        = 0x803E0100
+	WPN_E_CHANNEL_REQUEST_NOT_COMPLETE                                        Handle        = 0x803E0101
+	WPN_E_INVALID_APP                                                         Handle        = 0x803E0102
+	WPN_E_OUTSTANDING_CHANNEL_REQUEST                                         Handle        = 0x803E0103
+	WPN_E_DUPLICATE_CHANNEL                                                   Handle        = 0x803E0104
+	WPN_E_PLATFORM_UNAVAILABLE                                                Handle        = 0x803E0105
+	WPN_E_NOTIFICATION_POSTED                                                 Handle        = 0x803E0106
+	WPN_E_NOTIFICATION_HIDDEN                                                 Handle        = 0x803E0107
+	WPN_E_NOTIFICATION_NOT_POSTED                                             Handle        = 0x803E0108
+	WPN_E_CLOUD_DISABLED                                                      Handle        = 0x803E0109
+	WPN_E_CLOUD_INCAPABLE                                                     Handle        = 0x803E0110
+	WPN_E_CLOUD_AUTH_UNAVAILABLE                                              Handle        = 0x803E011A
+	WPN_E_CLOUD_SERVICE_UNAVAILABLE                                           Handle        = 0x803E011B
+	WPN_E_FAILED_LOCK_SCREEN_UPDATE_INTIALIZATION                             Handle        = 0x803E011C
+	WPN_E_NOTIFICATION_DISABLED                                               Handle        = 0x803E0111
+	WPN_E_NOTIFICATION_INCAPABLE                                              Handle        = 0x803E0112
+	WPN_E_INTERNET_INCAPABLE                                                  Handle        = 0x803E0113
+	WPN_E_NOTIFICATION_TYPE_DISABLED                                          Handle        = 0x803E0114
+	WPN_E_NOTIFICATION_SIZE                                                   Handle        = 0x803E0115
+	WPN_E_TAG_SIZE                                                            Handle        = 0x803E0116
+	WPN_E_ACCESS_DENIED                                                       Handle        = 0x803E0117
+	WPN_E_DUPLICATE_REGISTRATION                                              Handle        = 0x803E0118
+	WPN_E_PUSH_NOTIFICATION_INCAPABLE                                         Handle        = 0x803E0119
+	WPN_E_DEV_ID_SIZE                                                         Handle        = 0x803E0120
+	WPN_E_TAG_ALPHANUMERIC                                                    Handle        = 0x803E012A
+	WPN_E_INVALID_HTTP_STATUS_CODE                                            Handle        = 0x803E012B
+	WPN_E_OUT_OF_SESSION                                                      Handle        = 0x803E0200
+	WPN_E_POWER_SAVE                                                          Handle        = 0x803E0201
+	WPN_E_IMAGE_NOT_FOUND_IN_CACHE                                            Handle        = 0x803E0202
+	WPN_E_ALL_URL_NOT_COMPLETED                                               Handle        = 0x803E0203
+	WPN_E_INVALID_CLOUD_IMAGE                                                 Handle        = 0x803E0204
+	WPN_E_NOTIFICATION_ID_MATCHED                                             Handle        = 0x803E0205
+	WPN_E_CALLBACK_ALREADY_REGISTERED                                         Handle        = 0x803E0206
+	WPN_E_TOAST_NOTIFICATION_DROPPED                                          Handle        = 0x803E0207
+	WPN_E_STORAGE_LOCKED                                                      Handle        = 0x803E0208
+	WPN_E_GROUP_SIZE                                                          Handle        = 0x803E0209
+	WPN_E_GROUP_ALPHANUMERIC                                                  Handle        = 0x803E020A
+	WPN_E_CLOUD_DISABLED_FOR_APP                                              Handle        = 0x803E020B
+	E_MBN_CONTEXT_NOT_ACTIVATED                                               Handle        = 0x80548201
+	E_MBN_BAD_SIM                                                             Handle        = 0x80548202
+	E_MBN_DATA_CLASS_NOT_AVAILABLE                                            Handle        = 0x80548203
+	E_MBN_INVALID_ACCESS_STRING                                               Handle        = 0x80548204
+	E_MBN_MAX_ACTIVATED_CONTEXTS                                              Handle        = 0x80548205
+	E_MBN_PACKET_SVC_DETACHED                                                 Handle        = 0x80548206
+	E_MBN_PROVIDER_NOT_VISIBLE                                                Handle        = 0x80548207
+	E_MBN_RADIO_POWER_OFF                                                     Handle        = 0x80548208
+	E_MBN_SERVICE_NOT_ACTIVATED                                               Handle        = 0x80548209
+	E_MBN_SIM_NOT_INSERTED                                                    Handle        = 0x8054820A
+	E_MBN_VOICE_CALL_IN_PROGRESS                                              Handle        = 0x8054820B
+	E_MBN_INVALID_CACHE                                                       Handle        = 0x8054820C
+	E_MBN_NOT_REGISTERED                                                      Handle        = 0x8054820D
+	E_MBN_PROVIDERS_NOT_FOUND                                                 Handle        = 0x8054820E
+	E_MBN_PIN_NOT_SUPPORTED                                                   Handle        = 0x8054820F
+	E_MBN_PIN_REQUIRED                                                        Handle        = 0x80548210
+	E_MBN_PIN_DISABLED                                                        Handle        = 0x80548211
+	E_MBN_FAILURE                                                             Handle        = 0x80548212
+	E_MBN_INVALID_PROFILE                                                     Handle        = 0x80548218
+	E_MBN_DEFAULT_PROFILE_EXIST                                               Handle        = 0x80548219
+	E_MBN_SMS_ENCODING_NOT_SUPPORTED                                          Handle        = 0x80548220
+	E_MBN_SMS_FILTER_NOT_SUPPORTED                                            Handle        = 0x80548221
+	E_MBN_SMS_INVALID_MEMORY_INDEX                                            Handle        = 0x80548222
+	E_MBN_SMS_LANG_NOT_SUPPORTED                                              Handle        = 0x80548223
+	E_MBN_SMS_MEMORY_FAILURE                                                  Handle        = 0x80548224
+	E_MBN_SMS_NETWORK_TIMEOUT                                                 Handle        = 0x80548225
+	E_MBN_SMS_UNKNOWN_SMSC_ADDRESS                                            Handle        = 0x80548226
+	E_MBN_SMS_FORMAT_NOT_SUPPORTED                                            Handle        = 0x80548227
+	E_MBN_SMS_OPERATION_NOT_ALLOWED                                           Handle        = 0x80548228
+	E_MBN_SMS_MEMORY_FULL                                                     Handle        = 0x80548229
+	PEER_E_IPV6_NOT_INSTALLED                                                 Handle        = 0x80630001
+	PEER_E_NOT_INITIALIZED                                                    Handle        = 0x80630002
+	PEER_E_CANNOT_START_SERVICE                                               Handle        = 0x80630003
+	PEER_E_NOT_LICENSED                                                       Handle        = 0x80630004
+	PEER_E_INVALID_GRAPH                                                      Handle        = 0x80630010
+	PEER_E_DBNAME_CHANGED                                                     Handle        = 0x80630011
+	PEER_E_DUPLICATE_GRAPH                                                    Handle        = 0x80630012
+	PEER_E_GRAPH_NOT_READY                                                    Handle        = 0x80630013
+	PEER_E_GRAPH_SHUTTING_DOWN                                                Handle        = 0x80630014
+	PEER_E_GRAPH_IN_USE                                                       Handle        = 0x80630015
+	PEER_E_INVALID_DATABASE                                                   Handle        = 0x80630016
+	PEER_E_TOO_MANY_ATTRIBUTES                                                Handle        = 0x80630017
+	PEER_E_CONNECTION_NOT_FOUND                                               Handle        = 0x80630103
+	PEER_E_CONNECT_SELF                                                       Handle        = 0x80630106
+	PEER_E_ALREADY_LISTENING                                                  Handle        = 0x80630107
+	PEER_E_NODE_NOT_FOUND                                                     Handle        = 0x80630108
+	PEER_E_CONNECTION_FAILED                                                  Handle        = 0x80630109
+	PEER_E_CONNECTION_NOT_AUTHENTICATED                                       Handle        = 0x8063010A
+	PEER_E_CONNECTION_REFUSED                                                 Handle        = 0x8063010B
+	PEER_E_CLASSIFIER_TOO_LONG                                                Handle        = 0x80630201
+	PEER_E_TOO_MANY_IDENTITIES                                                Handle        = 0x80630202
+	PEER_E_NO_KEY_ACCESS                                                      Handle        = 0x80630203
+	PEER_E_GROUPS_EXIST                                                       Handle        = 0x80630204
+	PEER_E_RECORD_NOT_FOUND                                                   Handle        = 0x80630301
+	PEER_E_DATABASE_ACCESSDENIED                                              Handle        = 0x80630302
+	PEER_E_DBINITIALIZATION_FAILED                                            Handle        = 0x80630303
+	PEER_E_MAX_RECORD_SIZE_EXCEEDED                                           Handle        = 0x80630304
+	PEER_E_DATABASE_ALREADY_PRESENT                                           Handle        = 0x80630305
+	PEER_E_DATABASE_NOT_PRESENT                                               Handle        = 0x80630306
+	PEER_E_IDENTITY_NOT_FOUND                                                 Handle        = 0x80630401
+	PEER_E_EVENT_HANDLE_NOT_FOUND                                             Handle        = 0x80630501
+	PEER_E_INVALID_SEARCH                                                     Handle        = 0x80630601
+	PEER_E_INVALID_ATTRIBUTES                                                 Handle        = 0x80630602
+	PEER_E_INVITATION_NOT_TRUSTED                                             Handle        = 0x80630701
+	PEER_E_CHAIN_TOO_LONG                                                     Handle        = 0x80630703
+	PEER_E_INVALID_TIME_PERIOD                                                Handle        = 0x80630705
+	PEER_E_CIRCULAR_CHAIN_DETECTED                                            Handle        = 0x80630706
+	PEER_E_CERT_STORE_CORRUPTED                                               Handle        = 0x80630801
+	PEER_E_NO_CLOUD                                                           Handle        = 0x80631001
+	PEER_E_CLOUD_NAME_AMBIGUOUS                                               Handle        = 0x80631005
+	PEER_E_INVALID_RECORD                                                     Handle        = 0x80632010
+	PEER_E_NOT_AUTHORIZED                                                     Handle        = 0x80632020
+	PEER_E_PASSWORD_DOES_NOT_MEET_POLICY                                      Handle        = 0x80632021
+	PEER_E_DEFERRED_VALIDATION                                                Handle        = 0x80632030
+	PEER_E_INVALID_GROUP_PROPERTIES                                           Handle        = 0x80632040
+	PEER_E_INVALID_PEER_NAME                                                  Handle        = 0x80632050
+	PEER_E_INVALID_CLASSIFIER                                                 Handle        = 0x80632060
+	PEER_E_INVALID_FRIENDLY_NAME                                              Handle        = 0x80632070
+	PEER_E_INVALID_ROLE_PROPERTY                                              Handle        = 0x80632071
+	PEER_E_INVALID_CLASSIFIER_PROPERTY                                        Handle        = 0x80632072
+	PEER_E_INVALID_RECORD_EXPIRATION                                          Handle        = 0x80632080
+	PEER_E_INVALID_CREDENTIAL_INFO                                            Handle        = 0x80632081
+	PEER_E_INVALID_CREDENTIAL                                                 Handle        = 0x80632082
+	PEER_E_INVALID_RECORD_SIZE                                                Handle        = 0x80632083
+	PEER_E_UNSUPPORTED_VERSION                                                Handle        = 0x80632090
+	PEER_E_GROUP_NOT_READY                                                    Handle        = 0x80632091
+	PEER_E_GROUP_IN_USE                                                       Handle        = 0x80632092
+	PEER_E_INVALID_GROUP                                                      Handle        = 0x80632093
+	PEER_E_NO_MEMBERS_FOUND                                                   Handle        = 0x80632094
+	PEER_E_NO_MEMBER_CONNECTIONS                                              Handle        = 0x80632095
+	PEER_E_UNABLE_TO_LISTEN                                                   Handle        = 0x80632096
+	PEER_E_IDENTITY_DELETED                                                   Handle        = 0x806320A0
+	PEER_E_SERVICE_NOT_AVAILABLE                                              Handle        = 0x806320A1
+	PEER_E_CONTACT_NOT_FOUND                                                  Handle        = 0x80636001
+	PEER_S_GRAPH_DATA_CREATED                                                 Handle        = 0x00630001
+	PEER_S_NO_EVENT_DATA                                                      Handle        = 0x00630002
+	PEER_S_ALREADY_CONNECTED                                                  Handle        = 0x00632000
+	PEER_S_SUBSCRIPTION_EXISTS                                                Handle        = 0x00636000
+	PEER_S_NO_CONNECTIVITY                                                    Handle        = 0x00630005
+	PEER_S_ALREADY_A_MEMBER                                                   Handle        = 0x00630006
+	PEER_E_CANNOT_CONVERT_PEER_NAME                                           Handle        = 0x80634001
+	PEER_E_INVALID_PEER_HOST_NAME                                             Handle        = 0x80634002
+	PEER_E_NO_MORE                                                            Handle        = 0x80634003
+	PEER_E_PNRP_DUPLICATE_PEER_NAME                                           Handle        = 0x80634005
+	PEER_E_INVITE_CANCELLED                                                   Handle        = 0x80637000
+	PEER_E_INVITE_RESPONSE_NOT_AVAILABLE                                      Handle        = 0x80637001
+	PEER_E_NOT_SIGNED_IN                                                      Handle        = 0x80637003
+	PEER_E_PRIVACY_DECLINED                                                   Handle        = 0x80637004
+	PEER_E_TIMEOUT                                                            Handle        = 0x80637005
+	PEER_E_INVALID_ADDRESS                                                    Handle        = 0x80637007
+	PEER_E_FW_EXCEPTION_DISABLED                                              Handle        = 0x80637008
+	PEER_E_FW_BLOCKED_BY_POLICY                                               Handle        = 0x80637009
+	PEER_E_FW_BLOCKED_BY_SHIELDS_UP                                           Handle        = 0x8063700A
+	PEER_E_FW_DECLINED                                                        Handle        = 0x8063700B
+	UI_E_CREATE_FAILED                                                        Handle        = 0x802A0001
+	UI_E_SHUTDOWN_CALLED                                                      Handle        = 0x802A0002
+	UI_E_ILLEGAL_REENTRANCY                                                   Handle        = 0x802A0003
+	UI_E_OBJECT_SEALED                                                        Handle        = 0x802A0004
+	UI_E_VALUE_NOT_SET                                                        Handle        = 0x802A0005
+	UI_E_VALUE_NOT_DETERMINED                                                 Handle        = 0x802A0006
+	UI_E_INVALID_OUTPUT                                                       Handle        = 0x802A0007
+	UI_E_BOOLEAN_EXPECTED                                                     Handle        = 0x802A0008
+	UI_E_DIFFERENT_OWNER                                                      Handle        = 0x802A0009
+	UI_E_AMBIGUOUS_MATCH                                                      Handle        = 0x802A000A
+	UI_E_FP_OVERFLOW                                                          Handle        = 0x802A000B
+	UI_E_WRONG_THREAD                                                         Handle        = 0x802A000C
+	UI_E_STORYBOARD_ACTIVE                                                    Handle        = 0x802A0101
+	UI_E_STORYBOARD_NOT_PLAYING                                               Handle        = 0x802A0102
+	UI_E_START_KEYFRAME_AFTER_END                                             Handle        = 0x802A0103
+	UI_E_END_KEYFRAME_NOT_DETERMINED                                          Handle        = 0x802A0104
+	UI_E_LOOPS_OVERLAP                                                        Handle        = 0x802A0105
+	UI_E_TRANSITION_ALREADY_USED                                              Handle        = 0x802A0106
+	UI_E_TRANSITION_NOT_IN_STORYBOARD                                         Handle        = 0x802A0107
+	UI_E_TRANSITION_ECLIPSED                                                  Handle        = 0x802A0108
+	UI_E_TIME_BEFORE_LAST_UPDATE                                              Handle        = 0x802A0109
+	UI_E_TIMER_CLIENT_ALREADY_CONNECTED                                       Handle        = 0x802A010A
+	UI_E_INVALID_DIMENSION                                                    Handle        = 0x802A010B
+	UI_E_PRIMITIVE_OUT_OF_BOUNDS                                              Handle        = 0x802A010C
+	UI_E_WINDOW_CLOSED                                                        Handle        = 0x802A0201
+	E_BLUETOOTH_ATT_INVALID_HANDLE                                            Handle        = 0x80650001
+	E_BLUETOOTH_ATT_READ_NOT_PERMITTED                                        Handle        = 0x80650002
+	E_BLUETOOTH_ATT_WRITE_NOT_PERMITTED                                       Handle        = 0x80650003
+	E_BLUETOOTH_ATT_INVALID_PDU                                               Handle        = 0x80650004
+	E_BLUETOOTH_ATT_INSUFFICIENT_AUTHENTICATION                               Handle        = 0x80650005
+	E_BLUETOOTH_ATT_REQUEST_NOT_SUPPORTED                                     Handle        = 0x80650006
+	E_BLUETOOTH_ATT_INVALID_OFFSET                                            Handle        = 0x80650007
+	E_BLUETOOTH_ATT_INSUFFICIENT_AUTHORIZATION                                Handle        = 0x80650008
+	E_BLUETOOTH_ATT_PREPARE_QUEUE_FULL                                        Handle        = 0x80650009
+	E_BLUETOOTH_ATT_ATTRIBUTE_NOT_FOUND                                       Handle        = 0x8065000A
+	E_BLUETOOTH_ATT_ATTRIBUTE_NOT_LONG                                        Handle        = 0x8065000B
+	E_BLUETOOTH_ATT_INSUFFICIENT_ENCRYPTION_KEY_SIZE                          Handle        = 0x8065000C
+	E_BLUETOOTH_ATT_INVALID_ATTRIBUTE_VALUE_LENGTH                            Handle        = 0x8065000D
+	E_BLUETOOTH_ATT_UNLIKELY                                                  Handle        = 0x8065000E
+	E_BLUETOOTH_ATT_INSUFFICIENT_ENCRYPTION                                   Handle        = 0x8065000F
+	E_BLUETOOTH_ATT_UNSUPPORTED_GROUP_TYPE                                    Handle        = 0x80650010
+	E_BLUETOOTH_ATT_INSUFFICIENT_RESOURCES                                    Handle        = 0x80650011
+	E_BLUETOOTH_ATT_UNKNOWN_ERROR                                             Handle        = 0x80651000
+	E_AUDIO_ENGINE_NODE_NOT_FOUND                                             Handle        = 0x80660001
+	E_HDAUDIO_EMPTY_CONNECTION_LIST                                           Handle        = 0x80660002
+	E_HDAUDIO_CONNECTION_LIST_NOT_SUPPORTED                                   Handle        = 0x80660003
+	E_HDAUDIO_NO_LOGICAL_DEVICES_CREATED                                      Handle        = 0x80660004
+	E_HDAUDIO_NULL_LINKED_LIST_ENTRY                                          Handle        = 0x80660005
+	STATEREPOSITORY_E_CONCURRENCY_LOCKING_FAILURE                             Handle        = 0x80670001
+	STATEREPOSITORY_E_STATEMENT_INPROGRESS                                    Handle        = 0x80670002
+	STATEREPOSITORY_E_CONFIGURATION_INVALID                                   Handle        = 0x80670003
+	STATEREPOSITORY_E_UNKNOWN_SCHEMA_VERSION                                  Handle        = 0x80670004
+	STATEREPOSITORY_ERROR_DICTIONARY_CORRUPTED                                Handle        = 0x80670005
+	STATEREPOSITORY_E_BLOCKED                                                 Handle        = 0x80670006
+	STATEREPOSITORY_E_BUSY_RETRY                                              Handle        = 0x80670007
+	STATEREPOSITORY_E_BUSY_RECOVERY_RETRY                                     Handle        = 0x80670008
+	STATEREPOSITORY_E_LOCKED_RETRY                                            Handle        = 0x80670009
+	STATEREPOSITORY_E_LOCKED_SHAREDCACHE_RETRY                                Handle        = 0x8067000A
+	STATEREPOSITORY_E_TRANSACTION_REQUIRED                                    Handle        = 0x8067000B
+	STATEREPOSITORY_E_BUSY_TIMEOUT_EXCEEDED                                   Handle        = 0x8067000C
+	STATEREPOSITORY_E_BUSY_RECOVERY_TIMEOUT_EXCEEDED                          Handle        = 0x8067000D
+	STATEREPOSITORY_E_LOCKED_TIMEOUT_EXCEEDED                                 Handle        = 0x8067000E
+	STATEREPOSITORY_E_LOCKED_SHAREDCACHE_TIMEOUT_EXCEEDED                     Handle        = 0x8067000F
+	STATEREPOSITORY_E_SERVICE_STOP_IN_PROGRESS                                Handle        = 0x80670010
+	STATEREPOSTORY_E_NESTED_TRANSACTION_NOT_SUPPORTED                         Handle        = 0x80670011
+	STATEREPOSITORY_ERROR_CACHE_CORRUPTED                                     Handle        = 0x80670012
+	STATEREPOSITORY_TRANSACTION_CALLER_ID_CHANGED                             Handle        = 0x00670013
+	STATEREPOSITORY_TRANSACTION_IN_PROGRESS                                   Handle        = 0x00670014
+	ERROR_SPACES_POOL_WAS_DELETED                                             Handle        = 0x00E70001
+	ERROR_SPACES_FAULT_DOMAIN_TYPE_INVALID                                    Handle        = 0x80E70001
+	ERROR_SPACES_INTERNAL_ERROR                                               Handle        = 0x80E70002
+	ERROR_SPACES_RESILIENCY_TYPE_INVALID                                      Handle        = 0x80E70003
+	ERROR_SPACES_DRIVE_SECTOR_SIZE_INVALID                                    Handle        = 0x80E70004
+	ERROR_SPACES_DRIVE_REDUNDANCY_INVALID                                     Handle        = 0x80E70006
+	ERROR_SPACES_NUMBER_OF_DATA_COPIES_INVALID                                Handle        = 0x80E70007
+	ERROR_SPACES_PARITY_LAYOUT_INVALID                                        Handle        = 0x80E70008
+	ERROR_SPACES_INTERLEAVE_LENGTH_INVALID                                    Handle        = 0x80E70009
+	ERROR_SPACES_NUMBER_OF_COLUMNS_INVALID                                    Handle        = 0x80E7000A
+	ERROR_SPACES_NOT_ENOUGH_DRIVES                                            Handle        = 0x80E7000B
+	ERROR_SPACES_EXTENDED_ERROR                                               Handle        = 0x80E7000C
+	ERROR_SPACES_PROVISIONING_TYPE_INVALID                                    Handle        = 0x80E7000D
+	ERROR_SPACES_ALLOCATION_SIZE_INVALID                                      Handle        = 0x80E7000E
+	ERROR_SPACES_ENCLOSURE_AWARE_INVALID                                      Handle        = 0x80E7000F
+	ERROR_SPACES_WRITE_CACHE_SIZE_INVALID                                     Handle        = 0x80E70010
+	ERROR_SPACES_NUMBER_OF_GROUPS_INVALID                                     Handle        = 0x80E70011
+	ERROR_SPACES_DRIVE_OPERATIONAL_STATE_INVALID                              Handle        = 0x80E70012
+	ERROR_SPACES_ENTRY_INCOMPLETE                                             Handle        = 0x80E70013
+	ERROR_SPACES_ENTRY_INVALID                                                Handle        = 0x80E70014
+	ERROR_VOLSNAP_BOOTFILE_NOT_VALID                                          Handle        = 0x80820001
+	ERROR_VOLSNAP_ACTIVATION_TIMEOUT                                          Handle        = 0x80820002
+	ERROR_TIERING_NOT_SUPPORTED_ON_VOLUME                                     Handle        = 0x80830001
+	ERROR_TIERING_VOLUME_DISMOUNT_IN_PROGRESS                                 Handle        = 0x80830002
+	ERROR_TIERING_STORAGE_TIER_NOT_FOUND                                      Handle        = 0x80830003
+	ERROR_TIERING_INVALID_FILE_ID                                             Handle        = 0x80830004
+	ERROR_TIERING_WRONG_CLUSTER_NODE                                          Handle        = 0x80830005
+	ERROR_TIERING_ALREADY_PROCESSING                                          Handle        = 0x80830006
+	ERROR_TIERING_CANNOT_PIN_OBJECT                                           Handle        = 0x80830007
+	ERROR_TIERING_FILE_IS_NOT_PINNED                                          Handle        = 0x80830008
+	ERROR_NOT_A_TIERED_VOLUME                                                 Handle        = 0x80830009
+	ERROR_ATTRIBUTE_NOT_PRESENT                                               Handle        = 0x8083000A
+	ERROR_SECCORE_INVALID_COMMAND                                             Handle        = 0xC0E80000
+	ERROR_NO_APPLICABLE_APP_LICENSES_FOUND                                    Handle        = 0xC0EA0001
+	ERROR_CLIP_LICENSE_NOT_FOUND                                              Handle        = 0xC0EA0002
+	ERROR_CLIP_DEVICE_LICENSE_MISSING                                         Handle        = 0xC0EA0003
+	ERROR_CLIP_LICENSE_INVALID_SIGNATURE                                      Handle        = 0xC0EA0004
+	ERROR_CLIP_KEYHOLDER_LICENSE_MISSING_OR_INVALID                           Handle        = 0xC0EA0005
+	ERROR_CLIP_LICENSE_EXPIRED                                                Handle        = 0xC0EA0006
+	ERROR_CLIP_LICENSE_SIGNED_BY_UNKNOWN_SOURCE                               Handle        = 0xC0EA0007
+	ERROR_CLIP_LICENSE_NOT_SIGNED                                             Handle        = 0xC0EA0008
+	ERROR_CLIP_LICENSE_HARDWARE_ID_OUT_OF_TOLERANCE                           Handle        = 0xC0EA0009
+	ERROR_CLIP_LICENSE_DEVICE_ID_MISMATCH                                     Handle        = 0xC0EA000A
+	DXGI_STATUS_OCCLUDED                                                      Handle        = 0x087A0001
+	DXGI_STATUS_CLIPPED                                                       Handle        = 0x087A0002
+	DXGI_STATUS_NO_REDIRECTION                                                Handle        = 0x087A0004
+	DXGI_STATUS_NO_DESKTOP_ACCESS                                             Handle        = 0x087A0005
+	DXGI_STATUS_GRAPHICS_VIDPN_SOURCE_IN_USE                                  Handle        = 0x087A0006
+	DXGI_STATUS_MODE_CHANGED                                                  Handle        = 0x087A0007
+	DXGI_STATUS_MODE_CHANGE_IN_PROGRESS                                       Handle        = 0x087A0008
+	DXGI_ERROR_INVALID_CALL                                                   Handle        = 0x887A0001
+	DXGI_ERROR_NOT_FOUND                                                      Handle        = 0x887A0002
+	DXGI_ERROR_MORE_DATA                                                      Handle        = 0x887A0003
+	DXGI_ERROR_UNSUPPORTED                                                    Handle        = 0x887A0004
+	DXGI_ERROR_DEVICE_REMOVED                                                 Handle        = 0x887A0005
+	DXGI_ERROR_DEVICE_HUNG                                                    Handle        = 0x887A0006
+	DXGI_ERROR_DEVICE_RESET                                                   Handle        = 0x887A0007
+	DXGI_ERROR_WAS_STILL_DRAWING                                              Handle        = 0x887A000A
+	DXGI_ERROR_FRAME_STATISTICS_DISJOINT                                      Handle        = 0x887A000B
+	DXGI_ERROR_GRAPHICS_VIDPN_SOURCE_IN_USE                                   Handle        = 0x887A000C
+	DXGI_ERROR_DRIVER_INTERNAL_ERROR                                          Handle        = 0x887A0020
+	DXGI_ERROR_NONEXCLUSIVE                                                   Handle        = 0x887A0021
+	DXGI_ERROR_NOT_CURRENTLY_AVAILABLE                                        Handle        = 0x887A0022
+	DXGI_ERROR_REMOTE_CLIENT_DISCONNECTED                                     Handle        = 0x887A0023
+	DXGI_ERROR_REMOTE_OUTOFMEMORY                                             Handle        = 0x887A0024
+	DXGI_ERROR_ACCESS_LOST                                                    Handle        = 0x887A0026
+	DXGI_ERROR_WAIT_TIMEOUT                                                   Handle        = 0x887A0027
+	DXGI_ERROR_SESSION_DISCONNECTED                                           Handle        = 0x887A0028
+	DXGI_ERROR_RESTRICT_TO_OUTPUT_STALE                                       Handle        = 0x887A0029
+	DXGI_ERROR_CANNOT_PROTECT_CONTENT                                         Handle        = 0x887A002A
+	DXGI_ERROR_ACCESS_DENIED                                                  Handle        = 0x887A002B
+	DXGI_ERROR_NAME_ALREADY_EXISTS                                            Handle        = 0x887A002C
+	DXGI_ERROR_SDK_COMPONENT_MISSING                                          Handle        = 0x887A002D
+	DXGI_ERROR_NOT_CURRENT                                                    Handle        = 0x887A002E
+	DXGI_ERROR_HW_PROTECTION_OUTOFMEMORY                                      Handle        = 0x887A0030
+	DXGI_ERROR_DYNAMIC_CODE_POLICY_VIOLATION                                  Handle        = 0x887A0031
+	DXGI_ERROR_NON_COMPOSITED_UI                                              Handle        = 0x887A0032
+	DXGI_STATUS_UNOCCLUDED                                                    Handle        = 0x087A0009
+	DXGI_STATUS_DDA_WAS_STILL_DRAWING                                         Handle        = 0x087A000A
+	DXGI_ERROR_MODE_CHANGE_IN_PROGRESS                                        Handle        = 0x887A0025
+	DXGI_STATUS_PRESENT_REQUIRED                                              Handle        = 0x087A002F
+	DXGI_ERROR_CACHE_CORRUPT                                                  Handle        = 0x887A0033
+	DXGI_ERROR_CACHE_FULL                                                     Handle        = 0x887A0034
+	DXGI_ERROR_CACHE_HASH_COLLISION                                           Handle        = 0x887A0035
+	DXGI_ERROR_ALREADY_EXISTS                                                 Handle        = 0x887A0036
+	DXGI_DDI_ERR_WASSTILLDRAWING                                              Handle        = 0x887B0001
+	DXGI_DDI_ERR_UNSUPPORTED                                                  Handle        = 0x887B0002
+	DXGI_DDI_ERR_NONEXCLUSIVE                                                 Handle        = 0x887B0003
+	D3D10_ERROR_TOO_MANY_UNIQUE_STATE_OBJECTS                                 Handle        = 0x88790001
+	D3D10_ERROR_FILE_NOT_FOUND                                                Handle        = 0x88790002
+	D3D11_ERROR_TOO_MANY_UNIQUE_STATE_OBJECTS                                 Handle        = 0x887C0001
+	D3D11_ERROR_FILE_NOT_FOUND                                                Handle        = 0x887C0002
+	D3D11_ERROR_TOO_MANY_UNIQUE_VIEW_OBJECTS                                  Handle        = 0x887C0003
+	D3D11_ERROR_DEFERRED_CONTEXT_MAP_WITHOUT_INITIAL_DISCARD                  Handle        = 0x887C0004
+	D3D12_ERROR_ADAPTER_NOT_FOUND                                             Handle        = 0x887E0001
+	D3D12_ERROR_DRIVER_VERSION_MISMATCH                                       Handle        = 0x887E0002
+	D2DERR_WRONG_STATE                                                        Handle        = 0x88990001
+	D2DERR_NOT_INITIALIZED                                                    Handle        = 0x88990002
+	D2DERR_UNSUPPORTED_OPERATION                                              Handle        = 0x88990003
+	D2DERR_SCANNER_FAILED                                                     Handle        = 0x88990004
+	D2DERR_SCREEN_ACCESS_DENIED                                               Handle        = 0x88990005
+	D2DERR_DISPLAY_STATE_INVALID                                              Handle        = 0x88990006
+	D2DERR_ZERO_VECTOR                                                        Handle        = 0x88990007
+	D2DERR_INTERNAL_ERROR                                                     Handle        = 0x88990008
+	D2DERR_DISPLAY_FORMAT_NOT_SUPPORTED                                       Handle        = 0x88990009
+	D2DERR_INVALID_CALL                                                       Handle        = 0x8899000A
+	D2DERR_NO_HARDWARE_DEVICE                                                 Handle        = 0x8899000B
+	D2DERR_RECREATE_TARGET                                                    Handle        = 0x8899000C
+	D2DERR_TOO_MANY_SHADER_ELEMENTS                                           Handle        = 0x8899000D
+	D2DERR_SHADER_COMPILE_FAILED                                              Handle        = 0x8899000E
+	D2DERR_MAX_TEXTURE_SIZE_EXCEEDED                                          Handle        = 0x8899000F
+	D2DERR_UNSUPPORTED_VERSION                                                Handle        = 0x88990010
+	D2DERR_BAD_NUMBER                                                         Handle        = 0x88990011
+	D2DERR_WRONG_FACTORY                                                      Handle        = 0x88990012
+	D2DERR_LAYER_ALREADY_IN_USE                                               Handle        = 0x88990013
+	D2DERR_POP_CALL_DID_NOT_MATCH_PUSH                                        Handle        = 0x88990014
+	D2DERR_WRONG_RESOURCE_DOMAIN                                              Handle        = 0x88990015
+	D2DERR_PUSH_POP_UNBALANCED                                                Handle        = 0x88990016
+	D2DERR_RENDER_TARGET_HAS_LAYER_OR_CLIPRECT                                Handle        = 0x88990017
+	D2DERR_INCOMPATIBLE_BRUSH_TYPES                                           Handle        = 0x88990018
+	D2DERR_WIN32_ERROR                                                        Handle        = 0x88990019
+	D2DERR_TARGET_NOT_GDI_COMPATIBLE                                          Handle        = 0x8899001A
+	D2DERR_TEXT_EFFECT_IS_WRONG_TYPE                                          Handle        = 0x8899001B
+	D2DERR_TEXT_RENDERER_NOT_RELEASED                                         Handle        = 0x8899001C
+	D2DERR_EXCEEDS_MAX_BITMAP_SIZE                                            Handle        = 0x8899001D
+	D2DERR_INVALID_GRAPH_CONFIGURATION                                        Handle        = 0x8899001E
+	D2DERR_INVALID_INTERNAL_GRAPH_CONFIGURATION                               Handle        = 0x8899001F
+	D2DERR_CYCLIC_GRAPH                                                       Handle        = 0x88990020
+	D2DERR_BITMAP_CANNOT_DRAW                                                 Handle        = 0x88990021
+	D2DERR_OUTSTANDING_BITMAP_REFERENCES                                      Handle        = 0x88990022
+	D2DERR_ORIGINAL_TARGET_NOT_BOUND                                          Handle        = 0x88990023
+	D2DERR_INVALID_TARGET                                                     Handle        = 0x88990024
+	D2DERR_BITMAP_BOUND_AS_TARGET                                             Handle        = 0x88990025
+	D2DERR_INSUFFICIENT_DEVICE_CAPABILITIES                                   Handle        = 0x88990026
+	D2DERR_INTERMEDIATE_TOO_LARGE                                             Handle        = 0x88990027
+	D2DERR_EFFECT_IS_NOT_REGISTERED                                           Handle        = 0x88990028
+	D2DERR_INVALID_PROPERTY                                                   Handle        = 0x88990029
+	D2DERR_NO_SUBPROPERTIES                                                   Handle        = 0x8899002A
+	D2DERR_PRINT_JOB_CLOSED                                                   Handle        = 0x8899002B
+	D2DERR_PRINT_FORMAT_NOT_SUPPORTED                                         Handle        = 0x8899002C
+	D2DERR_TOO_MANY_TRANSFORM_INPUTS                                          Handle        = 0x8899002D
+	D2DERR_INVALID_GLYPH_IMAGE                                                Handle        = 0x8899002E
+	DWRITE_E_FILEFORMAT                                                       Handle        = 0x88985000
+	DWRITE_E_UNEXPECTED                                                       Handle        = 0x88985001
+	DWRITE_E_NOFONT                                                           Handle        = 0x88985002
+	DWRITE_E_FILENOTFOUND                                                     Handle        = 0x88985003
+	DWRITE_E_FILEACCESS                                                       Handle        = 0x88985004
+	DWRITE_E_FONTCOLLECTIONOBSOLETE                                           Handle        = 0x88985005
+	DWRITE_E_ALREADYREGISTERED                                                Handle        = 0x88985006
+	DWRITE_E_CACHEFORMAT                                                      Handle        = 0x88985007
+	DWRITE_E_CACHEVERSION                                                     Handle        = 0x88985008
+	DWRITE_E_UNSUPPORTEDOPERATION                                             Handle        = 0x88985009
+	DWRITE_E_TEXTRENDERERINCOMPATIBLE                                         Handle        = 0x8898500A
+	DWRITE_E_FLOWDIRECTIONCONFLICTS                                           Handle        = 0x8898500B
+	DWRITE_E_NOCOLOR                                                          Handle        = 0x8898500C
+	DWRITE_E_REMOTEFONT                                                       Handle        = 0x8898500D
+	DWRITE_E_DOWNLOADCANCELLED                                                Handle        = 0x8898500E
+	DWRITE_E_DOWNLOADFAILED                                                   Handle        = 0x8898500F
+	DWRITE_E_TOOMANYDOWNLOADS                                                 Handle        = 0x88985010
+	WINCODEC_ERR_WRONGSTATE                                                   Handle        = 0x88982F04
+	WINCODEC_ERR_VALUEOUTOFRANGE                                              Handle        = 0x88982F05
+	WINCODEC_ERR_UNKNOWNIMAGEFORMAT                                           Handle        = 0x88982F07
+	WINCODEC_ERR_UNSUPPORTEDVERSION                                           Handle        = 0x88982F0B
+	WINCODEC_ERR_NOTINITIALIZED                                               Handle        = 0x88982F0C
+	WINCODEC_ERR_ALREADYLOCKED                                                Handle        = 0x88982F0D
+	WINCODEC_ERR_PROPERTYNOTFOUND                                             Handle        = 0x88982F40
+	WINCODEC_ERR_PROPERTYNOTSUPPORTED                                         Handle        = 0x88982F41
+	WINCODEC_ERR_PROPERTYSIZE                                                 Handle        = 0x88982F42
+	WINCODEC_ERR_CODECPRESENT                                                 Handle        = 0x88982F43
+	WINCODEC_ERR_CODECNOTHUMBNAIL                                             Handle        = 0x88982F44
+	WINCODEC_ERR_PALETTEUNAVAILABLE                                           Handle        = 0x88982F45
+	WINCODEC_ERR_CODECTOOMANYSCANLINES                                        Handle        = 0x88982F46
+	WINCODEC_ERR_INTERNALERROR                                                Handle        = 0x88982F48
+	WINCODEC_ERR_SOURCERECTDOESNOTMATCHDIMENSIONS                             Handle        = 0x88982F49
+	WINCODEC_ERR_COMPONENTNOTFOUND                                            Handle        = 0x88982F50
+	WINCODEC_ERR_IMAGESIZEOUTOFRANGE                                          Handle        = 0x88982F51
+	WINCODEC_ERR_TOOMUCHMETADATA                                              Handle        = 0x88982F52
+	WINCODEC_ERR_BADIMAGE                                                     Handle        = 0x88982F60
+	WINCODEC_ERR_BADHEADER                                                    Handle        = 0x88982F61
+	WINCODEC_ERR_FRAMEMISSING                                                 Handle        = 0x88982F62
+	WINCODEC_ERR_BADMETADATAHEADER                                            Handle        = 0x88982F63
+	WINCODEC_ERR_BADSTREAMDATA                                                Handle        = 0x88982F70
+	WINCODEC_ERR_STREAMWRITE                                                  Handle        = 0x88982F71
+	WINCODEC_ERR_STREAMREAD                                                   Handle        = 0x88982F72
+	WINCODEC_ERR_STREAMNOTAVAILABLE                                           Handle        = 0x88982F73
+	WINCODEC_ERR_UNSUPPORTEDPIXELFORMAT                                       Handle        = 0x88982F80
+	WINCODEC_ERR_UNSUPPORTEDOPERATION                                         Handle        = 0x88982F81
+	WINCODEC_ERR_INVALIDREGISTRATION                                          Handle        = 0x88982F8A
+	WINCODEC_ERR_COMPONENTINITIALIZEFAILURE                                   Handle        = 0x88982F8B
+	WINCODEC_ERR_INSUFFICIENTBUFFER                                           Handle        = 0x88982F8C
+	WINCODEC_ERR_DUPLICATEMETADATAPRESENT                                     Handle        = 0x88982F8D
+	WINCODEC_ERR_PROPERTYUNEXPECTEDTYPE                                       Handle        = 0x88982F8E
+	WINCODEC_ERR_UNEXPECTEDSIZE                                               Handle        = 0x88982F8F
+	WINCODEC_ERR_INVALIDQUERYREQUEST                                          Handle        = 0x88982F90
+	WINCODEC_ERR_UNEXPECTEDMETADATATYPE                                       Handle        = 0x88982F91
+	WINCODEC_ERR_REQUESTONLYVALIDATMETADATAROOT                               Handle        = 0x88982F92
+	WINCODEC_ERR_INVALIDQUERYCHARACTER                                        Handle        = 0x88982F93
+	WINCODEC_ERR_WIN32ERROR                                                   Handle        = 0x88982F94
+	WINCODEC_ERR_INVALIDPROGRESSIVELEVEL                                      Handle        = 0x88982F95
+	WINCODEC_ERR_INVALIDJPEGSCANINDEX                                         Handle        = 0x88982F96
+	MILERR_OBJECTBUSY                                                         Handle        = 0x88980001
+	MILERR_INSUFFICIENTBUFFER                                                 Handle        = 0x88980002
+	MILERR_WIN32ERROR                                                         Handle        = 0x88980003
+	MILERR_SCANNER_FAILED                                                     Handle        = 0x88980004
+	MILERR_SCREENACCESSDENIED                                                 Handle        = 0x88980005
+	MILERR_DISPLAYSTATEINVALID                                                Handle        = 0x88980006
+	MILERR_NONINVERTIBLEMATRIX                                                Handle        = 0x88980007
+	MILERR_ZEROVECTOR                                                         Handle        = 0x88980008
+	MILERR_TERMINATED                                                         Handle        = 0x88980009
+	MILERR_BADNUMBER                                                          Handle        = 0x8898000A
+	MILERR_INTERNALERROR                                                      Handle        = 0x88980080
+	MILERR_DISPLAYFORMATNOTSUPPORTED                                          Handle        = 0x88980084
+	MILERR_INVALIDCALL                                                        Handle        = 0x88980085
+	MILERR_ALREADYLOCKED                                                      Handle        = 0x88980086
+	MILERR_NOTLOCKED                                                          Handle        = 0x88980087
+	MILERR_DEVICECANNOTRENDERTEXT                                             Handle        = 0x88980088
+	MILERR_GLYPHBITMAPMISSED                                                  Handle        = 0x88980089
+	MILERR_MALFORMEDGLYPHCACHE                                                Handle        = 0x8898008A
+	MILERR_GENERIC_IGNORE                                                     Handle        = 0x8898008B
+	MILERR_MALFORMED_GUIDELINE_DATA                                           Handle        = 0x8898008C
+	MILERR_NO_HARDWARE_DEVICE                                                 Handle        = 0x8898008D
+	MILERR_NEED_RECREATE_AND_PRESENT                                          Handle        = 0x8898008E
+	MILERR_ALREADY_INITIALIZED                                                Handle        = 0x8898008F
+	MILERR_MISMATCHED_SIZE                                                    Handle        = 0x88980090
+	MILERR_NO_REDIRECTION_SURFACE_AVAILABLE                                   Handle        = 0x88980091
+	MILERR_REMOTING_NOT_SUPPORTED                                             Handle        = 0x88980092
+	MILERR_QUEUED_PRESENT_NOT_SUPPORTED                                       Handle        = 0x88980093
+	MILERR_NOT_QUEUING_PRESENTS                                               Handle        = 0x88980094
+	MILERR_NO_REDIRECTION_SURFACE_RETRY_LATER                                 Handle        = 0x88980095
+	MILERR_TOOMANYSHADERELEMNTS                                               Handle        = 0x88980096
+	MILERR_MROW_READLOCK_FAILED                                               Handle        = 0x88980097
+	MILERR_MROW_UPDATE_FAILED                                                 Handle        = 0x88980098
+	MILERR_SHADER_COMPILE_FAILED                                              Handle        = 0x88980099
+	MILERR_MAX_TEXTURE_SIZE_EXCEEDED                                          Handle        = 0x8898009A
+	MILERR_QPC_TIME_WENT_BACKWARD                                             Handle        = 0x8898009B
+	MILERR_DXGI_ENUMERATION_OUT_OF_SYNC                                       Handle        = 0x8898009D
+	MILERR_ADAPTER_NOT_FOUND                                                  Handle        = 0x8898009E
+	MILERR_COLORSPACE_NOT_SUPPORTED                                           Handle        = 0x8898009F
+	MILERR_PREFILTER_NOT_SUPPORTED                                            Handle        = 0x889800A0
+	MILERR_DISPLAYID_ACCESS_DENIED                                            Handle        = 0x889800A1
+	UCEERR_INVALIDPACKETHEADER                                                Handle        = 0x88980400
+	UCEERR_UNKNOWNPACKET                                                      Handle        = 0x88980401
+	UCEERR_ILLEGALPACKET                                                      Handle        = 0x88980402
+	UCEERR_MALFORMEDPACKET                                                    Handle        = 0x88980403
+	UCEERR_ILLEGALHANDLE                                                      Handle        = 0x88980404
+	UCEERR_HANDLELOOKUPFAILED                                                 Handle        = 0x88980405
+	UCEERR_RENDERTHREADFAILURE                                                Handle        = 0x88980406
+	UCEERR_CTXSTACKFRSTTARGETNULL                                             Handle        = 0x88980407
+	UCEERR_CONNECTIONIDLOOKUPFAILED                                           Handle        = 0x88980408
+	UCEERR_BLOCKSFULL                                                         Handle        = 0x88980409
+	UCEERR_MEMORYFAILURE                                                      Handle        = 0x8898040A
+	UCEERR_PACKETRECORDOUTOFRANGE                                             Handle        = 0x8898040B
+	UCEERR_ILLEGALRECORDTYPE                                                  Handle        = 0x8898040C
+	UCEERR_OUTOFHANDLES                                                       Handle        = 0x8898040D
+	UCEERR_UNCHANGABLE_UPDATE_ATTEMPTED                                       Handle        = 0x8898040E
+	UCEERR_NO_MULTIPLE_WORKER_THREADS                                         Handle        = 0x8898040F
+	UCEERR_REMOTINGNOTSUPPORTED                                               Handle        = 0x88980410
+	UCEERR_MISSINGENDCOMMAND                                                  Handle        = 0x88980411
+	UCEERR_MISSINGBEGINCOMMAND                                                Handle        = 0x88980412
+	UCEERR_CHANNELSYNCTIMEDOUT                                                Handle        = 0x88980413
+	UCEERR_CHANNELSYNCABANDONED                                               Handle        = 0x88980414
+	UCEERR_UNSUPPORTEDTRANSPORTVERSION                                        Handle        = 0x88980415
+	UCEERR_TRANSPORTUNAVAILABLE                                               Handle        = 0x88980416
+	UCEERR_FEEDBACK_UNSUPPORTED                                               Handle        = 0x88980417
+	UCEERR_COMMANDTRANSPORTDENIED                                             Handle        = 0x88980418
+	UCEERR_GRAPHICSSTREAMUNAVAILABLE                                          Handle        = 0x88980419
+	UCEERR_GRAPHICSSTREAMALREADYOPEN                                          Handle        = 0x88980420
+	UCEERR_TRANSPORTDISCONNECTED                                              Handle        = 0x88980421
+	UCEERR_TRANSPORTOVERLOADED                                                Handle        = 0x88980422
+	UCEERR_PARTITION_ZOMBIED                                                  Handle        = 0x88980423
+	MILAVERR_NOCLOCK                                                          Handle        = 0x88980500
+	MILAVERR_NOMEDIATYPE                                                      Handle        = 0x88980501
+	MILAVERR_NOVIDEOMIXER                                                     Handle        = 0x88980502
+	MILAVERR_NOVIDEOPRESENTER                                                 Handle        = 0x88980503
+	MILAVERR_NOREADYFRAMES                                                    Handle        = 0x88980504
+	MILAVERR_MODULENOTLOADED                                                  Handle        = 0x88980505
+	MILAVERR_WMPFACTORYNOTREGISTERED                                          Handle        = 0x88980506
+	MILAVERR_INVALIDWMPVERSION                                                Handle        = 0x88980507
+	MILAVERR_INSUFFICIENTVIDEORESOURCES                                       Handle        = 0x88980508
+	MILAVERR_VIDEOACCELERATIONNOTAVAILABLE                                    Handle        = 0x88980509
+	MILAVERR_REQUESTEDTEXTURETOOBIG                                           Handle        = 0x8898050A
+	MILAVERR_SEEKFAILED                                                       Handle        = 0x8898050B
+	MILAVERR_UNEXPECTEDWMPFAILURE                                             Handle        = 0x8898050C
+	MILAVERR_MEDIAPLAYERCLOSED                                                Handle        = 0x8898050D
+	MILAVERR_UNKNOWNHARDWAREERROR                                             Handle        = 0x8898050E
+	MILEFFECTSERR_UNKNOWNPROPERTY                                             Handle        = 0x8898060E
+	MILEFFECTSERR_EFFECTNOTPARTOFGROUP                                        Handle        = 0x8898060F
+	MILEFFECTSERR_NOINPUTSOURCEATTACHED                                       Handle        = 0x88980610
+	MILEFFECTSERR_CONNECTORNOTCONNECTED                                       Handle        = 0x88980611
+	MILEFFECTSERR_CONNECTORNOTASSOCIATEDWITHEFFECT                            Handle        = 0x88980612
+	MILEFFECTSERR_RESERVED                                                    Handle        = 0x88980613
+	MILEFFECTSERR_CYCLEDETECTED                                               Handle        = 0x88980614
+	MILEFFECTSERR_EFFECTINMORETHANONEGRAPH                                    Handle        = 0x88980615
+	MILEFFECTSERR_EFFECTALREADYINAGRAPH                                       Handle        = 0x88980616
+	MILEFFECTSERR_EFFECTHASNOCHILDREN                                         Handle        = 0x88980617
+	MILEFFECTSERR_ALREADYATTACHEDTOLISTENER                                   Handle        = 0x88980618
+	MILEFFECTSERR_NOTAFFINETRANSFORM                                          Handle        = 0x88980619
+	MILEFFECTSERR_EMPTYBOUNDS                                                 Handle        = 0x8898061A
+	MILEFFECTSERR_OUTPUTSIZETOOLARGE                                          Handle        = 0x8898061B
+	DWMERR_STATE_TRANSITION_FAILED                                            Handle        = 0x88980700
+	DWMERR_THEME_FAILED                                                       Handle        = 0x88980701
+	DWMERR_CATASTROPHIC_FAILURE                                               Handle        = 0x88980702
+	DCOMPOSITION_ERROR_WINDOW_ALREADY_COMPOSED                                Handle        = 0x88980800
+	DCOMPOSITION_ERROR_SURFACE_BEING_RENDERED                                 Handle        = 0x88980801
+	DCOMPOSITION_ERROR_SURFACE_NOT_BEING_RENDERED                             Handle        = 0x88980802
+	ONL_E_INVALID_AUTHENTICATION_TARGET                                       Handle        = 0x80860001
+	ONL_E_ACCESS_DENIED_BY_TOU                                                Handle        = 0x80860002
+	ONL_E_INVALID_APPLICATION                                                 Handle        = 0x80860003
+	ONL_E_PASSWORD_UPDATE_REQUIRED                                            Handle        = 0x80860004
+	ONL_E_ACCOUNT_UPDATE_REQUIRED                                             Handle        = 0x80860005
+	ONL_E_FORCESIGNIN                                                         Handle        = 0x80860006
+	ONL_E_ACCOUNT_LOCKED                                                      Handle        = 0x80860007
+	ONL_E_PARENTAL_CONSENT_REQUIRED                                           Handle        = 0x80860008
+	ONL_E_EMAIL_VERIFICATION_REQUIRED                                         Handle        = 0x80860009
+	ONL_E_ACCOUNT_SUSPENDED_COMPROIMISE                                       Handle        = 0x8086000A
+	ONL_E_ACCOUNT_SUSPENDED_ABUSE                                             Handle        = 0x8086000B
+	ONL_E_ACTION_REQUIRED                                                     Handle        = 0x8086000C
+	ONL_CONNECTION_COUNT_LIMIT                                                Handle        = 0x8086000D
+	ONL_E_CONNECTED_ACCOUNT_CAN_NOT_SIGNOUT                                   Handle        = 0x8086000E
+	ONL_E_USER_AUTHENTICATION_REQUIRED                                        Handle        = 0x8086000F
+	ONL_E_REQUEST_THROTTLED                                                   Handle        = 0x80860010
+	FA_E_MAX_PERSISTED_ITEMS_REACHED                                          Handle        = 0x80270220
+	FA_E_HOMEGROUP_NOT_AVAILABLE                                              Handle        = 0x80270222
+	E_MONITOR_RESOLUTION_TOO_LOW                                              Handle        = 0x80270250
+	E_ELEVATED_ACTIVATION_NOT_SUPPORTED                                       Handle        = 0x80270251
+	E_UAC_DISABLED                                                            Handle        = 0x80270252
+	E_FULL_ADMIN_NOT_SUPPORTED                                                Handle        = 0x80270253
+	E_APPLICATION_NOT_REGISTERED                                              Handle        = 0x80270254
+	E_MULTIPLE_EXTENSIONS_FOR_APPLICATION                                     Handle        = 0x80270255
+	E_MULTIPLE_PACKAGES_FOR_FAMILY                                            Handle        = 0x80270256
+	E_APPLICATION_MANAGER_NOT_RUNNING                                         Handle        = 0x80270257
+	S_STORE_LAUNCHED_FOR_REMEDIATION                                          Handle        = 0x00270258
+	S_APPLICATION_ACTIVATION_ERROR_HANDLED_BY_DIALOG                          Handle        = 0x00270259
+	E_APPLICATION_ACTIVATION_TIMED_OUT                                        Handle        = 0x8027025A
+	E_APPLICATION_ACTIVATION_EXEC_FAILURE                                     Handle        = 0x8027025B
+	E_APPLICATION_TEMPORARY_LICENSE_ERROR                                     Handle        = 0x8027025C
+	E_APPLICATION_TRIAL_LICENSE_EXPIRED                                       Handle        = 0x8027025D
+	E_SKYDRIVE_ROOT_TARGET_FILE_SYSTEM_NOT_SUPPORTED                          Handle        = 0x80270260
+	E_SKYDRIVE_ROOT_TARGET_OVERLAP                                            Handle        = 0x80270261
+	E_SKYDRIVE_ROOT_TARGET_CANNOT_INDEX                                       Handle        = 0x80270262
+	E_SKYDRIVE_FILE_NOT_UPLOADED                                              Handle        = 0x80270263
+	E_SKYDRIVE_UPDATE_AVAILABILITY_FAIL                                       Handle        = 0x80270264
+	E_SKYDRIVE_ROOT_TARGET_VOLUME_ROOT_NOT_SUPPORTED                          Handle        = 0x80270265
+	E_SYNCENGINE_FILE_SIZE_OVER_LIMIT                                         Handle        = 0x8802B001
+	E_SYNCENGINE_FILE_SIZE_EXCEEDS_REMAINING_QUOTA                            Handle        = 0x8802B002
+	E_SYNCENGINE_UNSUPPORTED_FILE_NAME                                        Handle        = 0x8802B003
+	E_SYNCENGINE_FOLDER_ITEM_COUNT_LIMIT_EXCEEDED                             Handle        = 0x8802B004
+	E_SYNCENGINE_FILE_SYNC_PARTNER_ERROR                                      Handle        = 0x8802B005
+	E_SYNCENGINE_SYNC_PAUSED_BY_SERVICE                                       Handle        = 0x8802B006
+	E_SYNCENGINE_FILE_IDENTIFIER_UNKNOWN                                      Handle        = 0x8802C002
+	E_SYNCENGINE_SERVICE_AUTHENTICATION_FAILED                                Handle        = 0x8802C003
+	E_SYNCENGINE_UNKNOWN_SERVICE_ERROR                                        Handle        = 0x8802C004
+	E_SYNCENGINE_SERVICE_RETURNED_UNEXPECTED_SIZE                             Handle        = 0x8802C005
+	E_SYNCENGINE_REQUEST_BLOCKED_BY_SERVICE                                   Handle        = 0x8802C006
+	E_SYNCENGINE_REQUEST_BLOCKED_DUE_TO_CLIENT_ERROR                          Handle        = 0x8802C007
+	E_SYNCENGINE_FOLDER_INACCESSIBLE                                          Handle        = 0x8802D001
+	E_SYNCENGINE_UNSUPPORTED_FOLDER_NAME                                      Handle        = 0x8802D002
+	E_SYNCENGINE_UNSUPPORTED_MARKET                                           Handle        = 0x8802D003
+	E_SYNCENGINE_PATH_LENGTH_LIMIT_EXCEEDED                                   Handle        = 0x8802D004
+	E_SYNCENGINE_REMOTE_PATH_LENGTH_LIMIT_EXCEEDED                            Handle        = 0x8802D005
+	E_SYNCENGINE_CLIENT_UPDATE_NEEDED                                         Handle        = 0x8802D006
+	E_SYNCENGINE_PROXY_AUTHENTICATION_REQUIRED                                Handle        = 0x8802D007
+	E_SYNCENGINE_STORAGE_SERVICE_PROVISIONING_FAILED                          Handle        = 0x8802D008
+	E_SYNCENGINE_UNSUPPORTED_REPARSE_POINT                                    Handle        = 0x8802D009
+	E_SYNCENGINE_STORAGE_SERVICE_BLOCKED                                      Handle        = 0x8802D00A
+	E_SYNCENGINE_FOLDER_IN_REDIRECTION                                        Handle        = 0x8802D00B
+	EAS_E_POLICY_NOT_MANAGED_BY_OS                                            Handle        = 0x80550001
+	EAS_E_POLICY_COMPLIANT_WITH_ACTIONS                                       Handle        = 0x80550002
+	EAS_E_REQUESTED_POLICY_NOT_ENFORCEABLE                                    Handle        = 0x80550003
+	EAS_E_CURRENT_USER_HAS_BLANK_PASSWORD                                     Handle        = 0x80550004
+	EAS_E_REQUESTED_POLICY_PASSWORD_EXPIRATION_INCOMPATIBLE                   Handle        = 0x80550005
+	EAS_E_USER_CANNOT_CHANGE_PASSWORD                                         Handle        = 0x80550006
+	EAS_E_ADMINS_HAVE_BLANK_PASSWORD                                          Handle        = 0x80550007
+	EAS_E_ADMINS_CANNOT_CHANGE_PASSWORD                                       Handle        = 0x80550008
+	EAS_E_LOCAL_CONTROLLED_USERS_CANNOT_CHANGE_PASSWORD                       Handle        = 0x80550009
+	EAS_E_PASSWORD_POLICY_NOT_ENFORCEABLE_FOR_CONNECTED_ADMINS                Handle        = 0x8055000A
+	EAS_E_CONNECTED_ADMINS_NEED_TO_CHANGE_PASSWORD                            Handle        = 0x8055000B
+	EAS_E_PASSWORD_POLICY_NOT_ENFORCEABLE_FOR_CURRENT_CONNECTED_USER          Handle        = 0x8055000C
+	EAS_E_CURRENT_CONNECTED_USER_NEED_TO_CHANGE_PASSWORD                      Handle        = 0x8055000D
+	WEB_E_UNSUPPORTED_FORMAT                                                  Handle        = 0x83750001
+	WEB_E_INVALID_XML                                                         Handle        = 0x83750002
+	WEB_E_MISSING_REQUIRED_ELEMENT                                            Handle        = 0x83750003
+	WEB_E_MISSING_REQUIRED_ATTRIBUTE                                          Handle        = 0x83750004
+	WEB_E_UNEXPECTED_CONTENT                                                  Handle        = 0x83750005
+	WEB_E_RESOURCE_TOO_LARGE                                                  Handle        = 0x83750006
+	WEB_E_INVALID_JSON_STRING                                                 Handle        = 0x83750007
+	WEB_E_INVALID_JSON_NUMBER                                                 Handle        = 0x83750008
+	WEB_E_JSON_VALUE_NOT_FOUND                                                Handle        = 0x83750009
+	HTTP_E_STATUS_UNEXPECTED                                                  Handle        = 0x80190001
+	HTTP_E_STATUS_UNEXPECTED_REDIRECTION                                      Handle        = 0x80190003
+	HTTP_E_STATUS_UNEXPECTED_CLIENT_ERROR                                     Handle        = 0x80190004
+	HTTP_E_STATUS_UNEXPECTED_SERVER_ERROR                                     Handle        = 0x80190005
+	HTTP_E_STATUS_AMBIGUOUS                                                   Handle        = 0x8019012C
+	HTTP_E_STATUS_MOVED                                                       Handle        = 0x8019012D
+	HTTP_E_STATUS_REDIRECT                                                    Handle        = 0x8019012E
+	HTTP_E_STATUS_REDIRECT_METHOD                                             Handle        = 0x8019012F
+	HTTP_E_STATUS_NOT_MODIFIED                                                Handle        = 0x80190130
+	HTTP_E_STATUS_USE_PROXY                                                   Handle        = 0x80190131
+	HTTP_E_STATUS_REDIRECT_KEEP_VERB                                          Handle        = 0x80190133
+	HTTP_E_STATUS_BAD_REQUEST                                                 Handle        = 0x80190190
+	HTTP_E_STATUS_DENIED                                                      Handle        = 0x80190191
+	HTTP_E_STATUS_PAYMENT_REQ                                                 Handle        = 0x80190192
+	HTTP_E_STATUS_FORBIDDEN                                                   Handle        = 0x80190193
+	HTTP_E_STATUS_NOT_FOUND                                                   Handle        = 0x80190194
+	HTTP_E_STATUS_BAD_METHOD                                                  Handle        = 0x80190195
+	HTTP_E_STATUS_NONE_ACCEPTABLE                                             Handle        = 0x80190196
+	HTTP_E_STATUS_PROXY_AUTH_REQ                                              Handle        = 0x80190197
+	HTTP_E_STATUS_REQUEST_TIMEOUT                                             Handle        = 0x80190198
+	HTTP_E_STATUS_CONFLICT                                                    Handle        = 0x80190199
+	HTTP_E_STATUS_GONE                                                        Handle        = 0x8019019A
+	HTTP_E_STATUS_LENGTH_REQUIRED                                             Handle        = 0x8019019B
+	HTTP_E_STATUS_PRECOND_FAILED                                              Handle        = 0x8019019C
+	HTTP_E_STATUS_REQUEST_TOO_LARGE                                           Handle        = 0x8019019D
+	HTTP_E_STATUS_URI_TOO_LONG                                                Handle        = 0x8019019E
+	HTTP_E_STATUS_UNSUPPORTED_MEDIA                                           Handle        = 0x8019019F
+	HTTP_E_STATUS_RANGE_NOT_SATISFIABLE                                       Handle        = 0x801901A0
+	HTTP_E_STATUS_EXPECTATION_FAILED                                          Handle        = 0x801901A1
+	HTTP_E_STATUS_SERVER_ERROR                                                Handle        = 0x801901F4
+	HTTP_E_STATUS_NOT_SUPPORTED                                               Handle        = 0x801901F5
+	HTTP_E_STATUS_BAD_GATEWAY                                                 Handle        = 0x801901F6
+	HTTP_E_STATUS_SERVICE_UNAVAIL                                             Handle        = 0x801901F7
+	HTTP_E_STATUS_GATEWAY_TIMEOUT                                             Handle        = 0x801901F8
+	HTTP_E_STATUS_VERSION_NOT_SUP                                             Handle        = 0x801901F9
+	E_INVALID_PROTOCOL_OPERATION                                              Handle        = 0x83760001
+	E_INVALID_PROTOCOL_FORMAT                                                 Handle        = 0x83760002
+	E_PROTOCOL_EXTENSIONS_NOT_SUPPORTED                                       Handle        = 0x83760003
+	E_SUBPROTOCOL_NOT_SUPPORTED                                               Handle        = 0x83760004
+	E_PROTOCOL_VERSION_NOT_SUPPORTED                                          Handle        = 0x83760005
+	INPUT_E_OUT_OF_ORDER                                                      Handle        = 0x80400000
+	INPUT_E_REENTRANCY                                                        Handle        = 0x80400001
+	INPUT_E_MULTIMODAL                                                        Handle        = 0x80400002
+	INPUT_E_PACKET                                                            Handle        = 0x80400003
+	INPUT_E_FRAME                                                             Handle        = 0x80400004
+	INPUT_E_HISTORY                                                           Handle        = 0x80400005
+	INPUT_E_DEVICE_INFO                                                       Handle        = 0x80400006
+	INPUT_E_TRANSFORM                                                         Handle        = 0x80400007
+	INPUT_E_DEVICE_PROPERTY                                                   Handle        = 0x80400008
+	INET_E_INVALID_URL                                                        Handle        = 0x800C0002
+	INET_E_NO_SESSION                                                         Handle        = 0x800C0003
+	INET_E_CANNOT_CONNECT                                                     Handle        = 0x800C0004
+	INET_E_RESOURCE_NOT_FOUND                                                 Handle        = 0x800C0005
+	INET_E_OBJECT_NOT_FOUND                                                   Handle        = 0x800C0006
+	INET_E_DATA_NOT_AVAILABLE                                                 Handle        = 0x800C0007
+	INET_E_DOWNLOAD_FAILURE                                                   Handle        = 0x800C0008
+	INET_E_AUTHENTICATION_REQUIRED                                            Handle        = 0x800C0009
+	INET_E_NO_VALID_MEDIA                                                     Handle        = 0x800C000A
+	INET_E_CONNECTION_TIMEOUT                                                 Handle        = 0x800C000B
+	INET_E_INVALID_REQUEST                                                    Handle        = 0x800C000C
+	INET_E_UNKNOWN_PROTOCOL                                                   Handle        = 0x800C000D
+	INET_E_SECURITY_PROBLEM                                                   Handle        = 0x800C000E
+	INET_E_CANNOT_LOAD_DATA                                                   Handle        = 0x800C000F
+	INET_E_CANNOT_INSTANTIATE_OBJECT                                          Handle        = 0x800C0010
+	INET_E_INVALID_CERTIFICATE                                                Handle        = 0x800C0019
+	INET_E_REDIRECT_FAILED                                                    Handle        = 0x800C0014
+	INET_E_REDIRECT_TO_DIR                                                    Handle        = 0x800C0015
+	ERROR_DBG_CREATE_PROCESS_FAILURE_LOCKDOWN                                 Handle        = 0x80B00001
+	ERROR_DBG_ATTACH_PROCESS_FAILURE_LOCKDOWN                                 Handle        = 0x80B00002
+	ERROR_DBG_CONNECT_SERVER_FAILURE_LOCKDOWN                                 Handle        = 0x80B00003
+	ERROR_DBG_START_SERVER_FAILURE_LOCKDOWN                                   Handle        = 0x80B00004
+	ERROR_IO_PREEMPTED                                                        Handle        = 0x89010001
+	JSCRIPT_E_CANTEXECUTE                                                     Handle        = 0x89020001
+	WEP_E_NOT_PROVISIONED_ON_ALL_VOLUMES                                      Handle        = 0x88010001
+	WEP_E_FIXED_DATA_NOT_SUPPORTED                                            Handle        = 0x88010002
+	WEP_E_HARDWARE_NOT_COMPLIANT                                              Handle        = 0x88010003
+	WEP_E_LOCK_NOT_CONFIGURED                                                 Handle        = 0x88010004
+	WEP_E_PROTECTION_SUSPENDED                                                Handle        = 0x88010005
+	WEP_E_NO_LICENSE                                                          Handle        = 0x88010006
+	WEP_E_OS_NOT_PROTECTED                                                    Handle        = 0x88010007
+	WEP_E_UNEXPECTED_FAIL                                                     Handle        = 0x88010008
+	WEP_E_BUFFER_TOO_LARGE                                                    Handle        = 0x88010009
+	ERROR_SVHDX_ERROR_STORED                                                  Handle        = 0xC05C0000
+	ERROR_SVHDX_ERROR_NOT_AVAILABLE                                           Handle        = 0xC05CFF00
+	ERROR_SVHDX_UNIT_ATTENTION_AVAILABLE                                      Handle        = 0xC05CFF01
+	ERROR_SVHDX_UNIT_ATTENTION_CAPACITY_DATA_CHANGED                          Handle        = 0xC05CFF02
+	ERROR_SVHDX_UNIT_ATTENTION_RESERVATIONS_PREEMPTED                         Handle        = 0xC05CFF03
+	ERROR_SVHDX_UNIT_ATTENTION_RESERVATIONS_RELEASED                          Handle        = 0xC05CFF04
+	ERROR_SVHDX_UNIT_ATTENTION_REGISTRATIONS_PREEMPTED                        Handle        = 0xC05CFF05
+	ERROR_SVHDX_UNIT_ATTENTION_OPERATING_DEFINITION_CHANGED                   Handle        = 0xC05CFF06
+	ERROR_SVHDX_RESERVATION_CONFLICT                                          Handle        = 0xC05CFF07
+	ERROR_SVHDX_WRONG_FILE_TYPE                                               Handle        = 0xC05CFF08
+	ERROR_SVHDX_VERSION_MISMATCH                                              Handle        = 0xC05CFF09
+	ERROR_VHD_SHARED                                                          Handle        = 0xC05CFF0A
+	ERROR_SVHDX_NO_INITIATOR                                                  Handle        = 0xC05CFF0B
+	ERROR_VHDSET_BACKING_STORAGE_NOT_FOUND                                    Handle        = 0xC05CFF0C
+	ERROR_SMB_NO_PREAUTH_INTEGRITY_HASH_OVERLAP                               Handle        = 0xC05D0000
+	ERROR_SMB_BAD_CLUSTER_DIALECT                                             Handle        = 0xC05D0001
+	WININET_E_OUT_OF_HANDLES                                                  Handle        = 0x80072EE1
+	WININET_E_TIMEOUT                                                         Handle        = 0x80072EE2
+	WININET_E_EXTENDED_ERROR                                                  Handle        = 0x80072EE3
+	WININET_E_INTERNAL_ERROR                                                  Handle        = 0x80072EE4
+	WININET_E_INVALID_URL                                                     Handle        = 0x80072EE5
+	WININET_E_UNRECOGNIZED_SCHEME                                             Handle        = 0x80072EE6
+	WININET_E_NAME_NOT_RESOLVED                                               Handle        = 0x80072EE7
+	WININET_E_PROTOCOL_NOT_FOUND                                              Handle        = 0x80072EE8
+	WININET_E_INVALID_OPTION                                                  Handle        = 0x80072EE9
+	WININET_E_BAD_OPTION_LENGTH                                               Handle        = 0x80072EEA
+	WININET_E_OPTION_NOT_SETTABLE                                             Handle        = 0x80072EEB
+	WININET_E_SHUTDOWN                                                        Handle        = 0x80072EEC
+	WININET_E_INCORRECT_USER_NAME                                             Handle        = 0x80072EED
+	WININET_E_INCORRECT_PASSWORD                                              Handle        = 0x80072EEE
+	WININET_E_LOGIN_FAILURE                                                   Handle        = 0x80072EEF
+	WININET_E_INVALID_OPERATION                                               Handle        = 0x80072EF0
+	WININET_E_OPERATION_CANCELLED                                             Handle        = 0x80072EF1
+	WININET_E_INCORRECT_HANDLE_TYPE                                           Handle        = 0x80072EF2
+	WININET_E_INCORRECT_HANDLE_STATE                                          Handle        = 0x80072EF3
+	WININET_E_NOT_PROXY_REQUEST                                               Handle        = 0x80072EF4
+	WININET_E_REGISTRY_VALUE_NOT_FOUND                                        Handle        = 0x80072EF5
+	WININET_E_BAD_REGISTRY_PARAMETER                                          Handle        = 0x80072EF6
+	WININET_E_NO_DIRECT_ACCESS                                                Handle        = 0x80072EF7
+	WININET_E_NO_CONTEXT                                                      Handle        = 0x80072EF8
+	WININET_E_NO_CALLBACK                                                     Handle        = 0x80072EF9
+	WININET_E_REQUEST_PENDING                                                 Handle        = 0x80072EFA
+	WININET_E_INCORRECT_FORMAT                                                Handle        = 0x80072EFB
+	WININET_E_ITEM_NOT_FOUND                                                  Handle        = 0x80072EFC
+	WININET_E_CANNOT_CONNECT                                                  Handle        = 0x80072EFD
+	WININET_E_CONNECTION_ABORTED                                              Handle        = 0x80072EFE
+	WININET_E_CONNECTION_RESET                                                Handle        = 0x80072EFF
+	WININET_E_FORCE_RETRY                                                     Handle        = 0x80072F00
+	WININET_E_INVALID_PROXY_REQUEST                                           Handle        = 0x80072F01
+	WININET_E_NEED_UI                                                         Handle        = 0x80072F02
+	WININET_E_HANDLE_EXISTS                                                   Handle        = 0x80072F04
+	WININET_E_SEC_CERT_DATE_INVALID                                           Handle        = 0x80072F05
+	WININET_E_SEC_CERT_CN_INVALID                                             Handle        = 0x80072F06
+	WININET_E_HTTP_TO_HTTPS_ON_REDIR                                          Handle        = 0x80072F07
+	WININET_E_HTTPS_TO_HTTP_ON_REDIR                                          Handle        = 0x80072F08
+	WININET_E_MIXED_SECURITY                                                  Handle        = 0x80072F09
+	WININET_E_CHG_POST_IS_NON_SECURE                                          Handle        = 0x80072F0A
+	WININET_E_POST_IS_NON_SECURE                                              Handle        = 0x80072F0B
+	WININET_E_CLIENT_AUTH_CERT_NEEDED                                         Handle        = 0x80072F0C
+	WININET_E_INVALID_CA                                                      Handle        = 0x80072F0D
+	WININET_E_CLIENT_AUTH_NOT_SETUP                                           Handle        = 0x80072F0E
+	WININET_E_ASYNC_THREAD_FAILED                                             Handle        = 0x80072F0F
+	WININET_E_REDIRECT_SCHEME_CHANGE                                          Handle        = 0x80072F10
+	WININET_E_DIALOG_PENDING                                                  Handle        = 0x80072F11
+	WININET_E_RETRY_DIALOG                                                    Handle        = 0x80072F12
+	WININET_E_NO_NEW_CONTAINERS                                               Handle        = 0x80072F13
+	WININET_E_HTTPS_HTTP_SUBMIT_REDIR                                         Handle        = 0x80072F14
+	WININET_E_SEC_CERT_ERRORS                                                 Handle        = 0x80072F17
+	WININET_E_SEC_CERT_REV_FAILED                                             Handle        = 0x80072F19
+	WININET_E_HEADER_NOT_FOUND                                                Handle        = 0x80072F76
+	WININET_E_DOWNLEVEL_SERVER                                                Handle        = 0x80072F77
+	WININET_E_INVALID_SERVER_RESPONSE                                         Handle        = 0x80072F78
+	WININET_E_INVALID_HEADER                                                  Handle        = 0x80072F79
+	WININET_E_INVALID_QUERY_REQUEST                                           Handle        = 0x80072F7A
+	WININET_E_HEADER_ALREADY_EXISTS                                           Handle        = 0x80072F7B
+	WININET_E_REDIRECT_FAILED                                                 Handle        = 0x80072F7C
+	WININET_E_SECURITY_CHANNEL_ERROR                                          Handle        = 0x80072F7D
+	WININET_E_UNABLE_TO_CACHE_FILE                                            Handle        = 0x80072F7E
+	WININET_E_TCPIP_NOT_INSTALLED                                             Handle        = 0x80072F7F
+	WININET_E_DISCONNECTED                                                    Handle        = 0x80072F83
+	WININET_E_SERVER_UNREACHABLE                                              Handle        = 0x80072F84
+	WININET_E_PROXY_SERVER_UNREACHABLE                                        Handle        = 0x80072F85
+	WININET_E_BAD_AUTO_PROXY_SCRIPT                                           Handle        = 0x80072F86
+	WININET_E_UNABLE_TO_DOWNLOAD_SCRIPT                                       Handle        = 0x80072F87
+	WININET_E_SEC_INVALID_CERT                                                Handle        = 0x80072F89
+	WININET_E_SEC_CERT_REVOKED                                                Handle        = 0x80072F8A
+	WININET_E_FAILED_DUETOSECURITYCHECK                                       Handle        = 0x80072F8B
+	WININET_E_NOT_INITIALIZED                                                 Handle        = 0x80072F8C
+	WININET_E_LOGIN_FAILURE_DISPLAY_ENTITY_BODY                               Handle        = 0x80072F8E
+	WININET_E_DECODING_FAILED                                                 Handle        = 0x80072F8F
+	WININET_E_NOT_REDIRECTED                                                  Handle        = 0x80072F80
+	WININET_E_COOKIE_NEEDS_CONFIRMATION                                       Handle        = 0x80072F81
+	WININET_E_COOKIE_DECLINED                                                 Handle        = 0x80072F82
+	WININET_E_REDIRECT_NEEDS_CONFIRMATION                                     Handle        = 0x80072F88
+	SQLITE_E_ERROR                                                            Handle        = 0x87AF0001
+	SQLITE_E_INTERNAL                                                         Handle        = 0x87AF0002
+	SQLITE_E_PERM                                                             Handle        = 0x87AF0003
+	SQLITE_E_ABORT                                                            Handle        = 0x87AF0004
+	SQLITE_E_BUSY                                                             Handle        = 0x87AF0005
+	SQLITE_E_LOCKED                                                           Handle        = 0x87AF0006
+	SQLITE_E_NOMEM                                                            Handle        = 0x87AF0007
+	SQLITE_E_READONLY                                                         Handle        = 0x87AF0008
+	SQLITE_E_INTERRUPT                                                        Handle        = 0x87AF0009
+	SQLITE_E_IOERR                                                            Handle        = 0x87AF000A
+	SQLITE_E_CORRUPT                                                          Handle        = 0x87AF000B
+	SQLITE_E_NOTFOUND                                                         Handle        = 0x87AF000C
+	SQLITE_E_FULL                                                             Handle        = 0x87AF000D
+	SQLITE_E_CANTOPEN                                                         Handle        = 0x87AF000E
+	SQLITE_E_PROTOCOL                                                         Handle        = 0x87AF000F
+	SQLITE_E_EMPTY                                                            Handle        = 0x87AF0010
+	SQLITE_E_SCHEMA                                                           Handle        = 0x87AF0011
+	SQLITE_E_TOOBIG                                                           Handle        = 0x87AF0012
+	SQLITE_E_CONSTRAINT                                                       Handle        = 0x87AF0013
+	SQLITE_E_MISMATCH                                                         Handle        = 0x87AF0014
+	SQLITE_E_MISUSE                                                           Handle        = 0x87AF0015
+	SQLITE_E_NOLFS                                                            Handle        = 0x87AF0016
+	SQLITE_E_AUTH                                                             Handle        = 0x87AF0017
+	SQLITE_E_FORMAT                                                           Handle        = 0x87AF0018
+	SQLITE_E_RANGE                                                            Handle        = 0x87AF0019
+	SQLITE_E_NOTADB                                                           Handle        = 0x87AF001A
+	SQLITE_E_NOTICE                                                           Handle        = 0x87AF001B
+	SQLITE_E_WARNING                                                          Handle        = 0x87AF001C
+	SQLITE_E_ROW                                                              Handle        = 0x87AF0064
+	SQLITE_E_DONE                                                             Handle        = 0x87AF0065
+	SQLITE_E_IOERR_READ                                                       Handle        = 0x87AF010A
+	SQLITE_E_IOERR_SHORT_READ                                                 Handle        = 0x87AF020A
+	SQLITE_E_IOERR_WRITE                                                      Handle        = 0x87AF030A
+	SQLITE_E_IOERR_FSYNC                                                      Handle        = 0x87AF040A
+	SQLITE_E_IOERR_DIR_FSYNC                                                  Handle        = 0x87AF050A
+	SQLITE_E_IOERR_TRUNCATE                                                   Handle        = 0x87AF060A
+	SQLITE_E_IOERR_FSTAT                                                      Handle        = 0x87AF070A
+	SQLITE_E_IOERR_UNLOCK                                                     Handle        = 0x87AF080A
+	SQLITE_E_IOERR_RDLOCK                                                     Handle        = 0x87AF090A
+	SQLITE_E_IOERR_DELETE                                                     Handle        = 0x87AF0A0A
+	SQLITE_E_IOERR_BLOCKED                                                    Handle        = 0x87AF0B0A
+	SQLITE_E_IOERR_NOMEM                                                      Handle        = 0x87AF0C0A
+	SQLITE_E_IOERR_ACCESS                                                     Handle        = 0x87AF0D0A
+	SQLITE_E_IOERR_CHECKRESERVEDLOCK                                          Handle        = 0x87AF0E0A
+	SQLITE_E_IOERR_LOCK                                                       Handle        = 0x87AF0F0A
+	SQLITE_E_IOERR_CLOSE                                                      Handle        = 0x87AF100A
+	SQLITE_E_IOERR_DIR_CLOSE                                                  Handle        = 0x87AF110A
+	SQLITE_E_IOERR_SHMOPEN                                                    Handle        = 0x87AF120A
+	SQLITE_E_IOERR_SHMSIZE                                                    Handle        = 0x87AF130A
+	SQLITE_E_IOERR_SHMLOCK                                                    Handle        = 0x87AF140A
+	SQLITE_E_IOERR_SHMMAP                                                     Handle        = 0x87AF150A
+	SQLITE_E_IOERR_SEEK                                                       Handle        = 0x87AF160A
+	SQLITE_E_IOERR_DELETE_NOENT                                               Handle        = 0x87AF170A
+	SQLITE_E_IOERR_MMAP                                                       Handle        = 0x87AF180A
+	SQLITE_E_IOERR_GETTEMPPATH                                                Handle        = 0x87AF190A
+	SQLITE_E_IOERR_CONVPATH                                                   Handle        = 0x87AF1A0A
+	SQLITE_E_IOERR_VNODE                                                      Handle        = 0x87AF1A02
+	SQLITE_E_IOERR_AUTH                                                       Handle        = 0x87AF1A03
+	SQLITE_E_LOCKED_SHAREDCACHE                                               Handle        = 0x87AF0106
+	SQLITE_E_BUSY_RECOVERY                                                    Handle        = 0x87AF0105
+	SQLITE_E_BUSY_SNAPSHOT                                                    Handle        = 0x87AF0205
+	SQLITE_E_CANTOPEN_NOTEMPDIR                                               Handle        = 0x87AF010E
+	SQLITE_E_CANTOPEN_ISDIR                                                   Handle        = 0x87AF020E
+	SQLITE_E_CANTOPEN_FULLPATH                                                Handle        = 0x87AF030E
+	SQLITE_E_CANTOPEN_CONVPATH                                                Handle        = 0x87AF040E
+	SQLITE_E_CORRUPT_VTAB                                                     Handle        = 0x87AF010B
+	SQLITE_E_READONLY_RECOVERY                                                Handle        = 0x87AF0108
+	SQLITE_E_READONLY_CANTLOCK                                                Handle        = 0x87AF0208
+	SQLITE_E_READONLY_ROLLBACK                                                Handle        = 0x87AF0308
+	SQLITE_E_READONLY_DBMOVED                                                 Handle        = 0x87AF0408
+	SQLITE_E_ABORT_ROLLBACK                                                   Handle        = 0x87AF0204
+	SQLITE_E_CONSTRAINT_CHECK                                                 Handle        = 0x87AF0113
+	SQLITE_E_CONSTRAINT_COMMITHOOK                                            Handle        = 0x87AF0213
+	SQLITE_E_CONSTRAINT_FOREIGNKEY                                            Handle        = 0x87AF0313
+	SQLITE_E_CONSTRAINT_FUNCTION                                              Handle        = 0x87AF0413
+	SQLITE_E_CONSTRAINT_NOTNULL                                               Handle        = 0x87AF0513
+	SQLITE_E_CONSTRAINT_PRIMARYKEY                                            Handle        = 0x87AF0613
+	SQLITE_E_CONSTRAINT_TRIGGER                                               Handle        = 0x87AF0713
+	SQLITE_E_CONSTRAINT_UNIQUE                                                Handle        = 0x87AF0813
+	SQLITE_E_CONSTRAINT_VTAB                                                  Handle        = 0x87AF0913
+	SQLITE_E_CONSTRAINT_ROWID                                                 Handle        = 0x87AF0A13
+	SQLITE_E_NOTICE_RECOVER_WAL                                               Handle        = 0x87AF011B
+	SQLITE_E_NOTICE_RECOVER_ROLLBACK                                          Handle        = 0x87AF021B
+	SQLITE_E_WARNING_AUTOINDEX                                                Handle        = 0x87AF011C
+	UTC_E_TOGGLE_TRACE_STARTED                                                Handle        = 0x87C51001
+	UTC_E_ALTERNATIVE_TRACE_CANNOT_PREEMPT                                    Handle        = 0x87C51002
+	UTC_E_AOT_NOT_RUNNING                                                     Handle        = 0x87C51003
+	UTC_E_SCRIPT_TYPE_INVALID                                                 Handle        = 0x87C51004
+	UTC_E_SCENARIODEF_NOT_FOUND                                               Handle        = 0x87C51005
+	UTC_E_TRACEPROFILE_NOT_FOUND                                              Handle        = 0x87C51006
+	UTC_E_FORWARDER_ALREADY_ENABLED                                           Handle        = 0x87C51007
+	UTC_E_FORWARDER_ALREADY_DISABLED                                          Handle        = 0x87C51008
+	UTC_E_EVENTLOG_ENTRY_MALFORMED                                            Handle        = 0x87C51009
+	UTC_E_DIAGRULES_SCHEMAVERSION_MISMATCH                                    Handle        = 0x87C5100A
+	UTC_E_SCRIPT_TERMINATED                                                   Handle        = 0x87C5100B
+	UTC_E_INVALID_CUSTOM_FILTER                                               Handle        = 0x87C5100C
+	UTC_E_TRACE_NOT_RUNNING                                                   Handle        = 0x87C5100D
+	UTC_E_REESCALATED_TOO_QUICKLY                                             Handle        = 0x87C5100E
+	UTC_E_ESCALATION_ALREADY_RUNNING                                          Handle        = 0x87C5100F
+	UTC_E_PERFTRACK_ALREADY_TRACING                                           Handle        = 0x87C51010
+	UTC_E_REACHED_MAX_ESCALATIONS                                             Handle        = 0x87C51011
+	UTC_E_FORWARDER_PRODUCER_MISMATCH                                         Handle        = 0x87C51012
+	UTC_E_INTENTIONAL_SCRIPT_FAILURE                                          Handle        = 0x87C51013
+	UTC_E_SQM_INIT_FAILED                                                     Handle        = 0x87C51014
+	UTC_E_NO_WER_LOGGER_SUPPORTED                                             Handle        = 0x87C51015
+	UTC_E_TRACERS_DONT_EXIST                                                  Handle        = 0x87C51016
+	UTC_E_WINRT_INIT_FAILED                                                   Handle        = 0x87C51017
+	UTC_E_SCENARIODEF_SCHEMAVERSION_MISMATCH                                  Handle        = 0x87C51018
+	UTC_E_INVALID_FILTER                                                      Handle        = 0x87C51019
+	UTC_E_EXE_TERMINATED                                                      Handle        = 0x87C5101A
+	UTC_E_ESCALATION_NOT_AUTHORIZED                                           Handle        = 0x87C5101B
+	UTC_E_SETUP_NOT_AUTHORIZED                                                Handle        = 0x87C5101C
+	UTC_E_CHILD_PROCESS_FAILED                                                Handle        = 0x87C5101D
+	UTC_E_COMMAND_LINE_NOT_AUTHORIZED                                         Handle        = 0x87C5101E
+	UTC_E_CANNOT_LOAD_SCENARIO_EDITOR_XML                                     Handle        = 0x87C5101F
+	UTC_E_ESCALATION_TIMED_OUT                                                Handle        = 0x87C51020
+	UTC_E_SETUP_TIMED_OUT                                                     Handle        = 0x87C51021
+	UTC_E_TRIGGER_MISMATCH                                                    Handle        = 0x87C51022
+	UTC_E_TRIGGER_NOT_FOUND                                                   Handle        = 0x87C51023
+	UTC_E_SIF_NOT_SUPPORTED                                                   Handle        = 0x87C51024
+	UTC_E_DELAY_TERMINATED                                                    Handle        = 0x87C51025
+	UTC_E_DEVICE_TICKET_ERROR                                                 Handle        = 0x87C51026
+	UTC_E_TRACE_BUFFER_LIMIT_EXCEEDED                                         Handle        = 0x87C51027
+	UTC_E_API_RESULT_UNAVAILABLE                                              Handle        = 0x87C51028
+	UTC_E_RPC_TIMEOUT                                                         Handle        = 0x87C51029
+	UTC_E_RPC_WAIT_FAILED                                                     Handle        = 0x87C5102A
+	UTC_E_API_BUSY                                                            Handle        = 0x87C5102B
+	UTC_E_TRACE_MIN_DURATION_REQUIREMENT_NOT_MET                              Handle        = 0x87C5102C
+	UTC_E_EXCLUSIVITY_NOT_AVAILABLE                                           Handle        = 0x87C5102D
+	UTC_E_GETFILE_FILE_PATH_NOT_APPROVED                                      Handle        = 0x87C5102E
+	UTC_E_ESCALATION_DIRECTORY_ALREADY_EXISTS                                 Handle        = 0x87C5102F
+	UTC_E_TIME_TRIGGER_ON_START_INVALID                                       Handle        = 0x87C51030
+	UTC_E_TIME_TRIGGER_ONLY_VALID_ON_SINGLE_TRANSITION                        Handle        = 0x87C51031
+	UTC_E_TIME_TRIGGER_INVALID_TIME_RANGE                                     Handle        = 0x87C51032
+	UTC_E_MULTIPLE_TIME_TRIGGER_ON_SINGLE_STATE                               Handle        = 0x87C51033
+	UTC_E_BINARY_MISSING                                                      Handle        = 0x87C51034
+	UTC_E_NETWORK_CAPTURE_NOT_ALLOWED                                         Handle        = 0x87C51035
+	UTC_E_FAILED_TO_RESOLVE_CONTAINER_ID                                      Handle        = 0x87C51036
+	UTC_E_UNABLE_TO_RESOLVE_SESSION                                           Handle        = 0x87C51037
+	UTC_E_THROTTLED                                                           Handle        = 0x87C51038
+	UTC_E_UNAPPROVED_SCRIPT                                                   Handle        = 0x87C51039
+	UTC_E_SCRIPT_MISSING                                                      Handle        = 0x87C5103A
+	UTC_E_SCENARIO_THROTTLED                                                  Handle        = 0x87C5103B
+	UTC_E_API_NOT_SUPPORTED                                                   Handle        = 0x87C5103C
+	UTC_E_GETFILE_EXTERNAL_PATH_NOT_APPROVED                                  Handle        = 0x87C5103D
+	UTC_E_TRY_GET_SCENARIO_TIMEOUT_EXCEEDED                                   Handle        = 0x87C5103E
+	UTC_E_CERT_REV_FAILED                                                     Handle        = 0x87C5103F
+	UTC_E_FAILED_TO_START_NDISCAP                                             Handle        = 0x87C51040
+	UTC_E_KERNELDUMP_LIMIT_REACHED                                            Handle        = 0x87C51041
+	UTC_E_MISSING_AGGREGATE_EVENT_TAG                                         Handle        = 0x87C51042
+	UTC_E_INVALID_AGGREGATION_STRUCT                                          Handle        = 0x87C51043
+	UTC_E_ACTION_NOT_SUPPORTED_IN_DESTINATION                                 Handle        = 0x87C51044
+	UTC_E_FILTER_MISSING_ATTRIBUTE                                            Handle        = 0x87C51045
+	UTC_E_FILTER_INVALID_TYPE                                                 Handle        = 0x87C51046
+	UTC_E_FILTER_VARIABLE_NOT_FOUND                                           Handle        = 0x87C51047
+	UTC_E_FILTER_FUNCTION_RESTRICTED                                          Handle        = 0x87C51048
+	UTC_E_FILTER_VERSION_MISMATCH                                             Handle        = 0x87C51049
+	UTC_E_FILTER_INVALID_FUNCTION                                             Handle        = 0x87C51050
+	UTC_E_FILTER_INVALID_FUNCTION_PARAMS                                      Handle        = 0x87C51051
+	UTC_E_FILTER_INVALID_COMMAND                                              Handle        = 0x87C51052
+	UTC_E_FILTER_ILLEGAL_EVAL                                                 Handle        = 0x87C51053
+	UTC_E_TTTRACER_RETURNED_ERROR                                             Handle        = 0x87C51054
+	UTC_E_AGENT_DIAGNOSTICS_TOO_LARGE                                         Handle        = 0x87C51055
+	UTC_E_FAILED_TO_RECEIVE_AGENT_DIAGNOSTICS                                 Handle        = 0x87C51056
+	UTC_E_SCENARIO_HAS_NO_ACTIONS                                             Handle        = 0x87C51057
+	UTC_E_TTTRACER_STORAGE_FULL                                               Handle        = 0x87C51058
+	UTC_E_INSUFFICIENT_SPACE_TO_START_TRACE                                   Handle        = 0x87C51059
+	UTC_E_ESCALATION_CANCELLED_AT_SHUTDOWN                                    Handle        = 0x87C5105A
+	UTC_E_GETFILEINFOACTION_FILE_NOT_APPROVED                                 Handle        = 0x87C5105B
+	WINML_ERR_INVALID_DEVICE                                                  Handle        = 0x88900001
+	WINML_ERR_INVALID_BINDING                                                 Handle        = 0x88900002
+	WINML_ERR_VALUE_NOTFOUND                                                  Handle        = 0x88900003
+	WINML_ERR_SIZE_MISMATCH                                                   Handle        = 0x88900004
+)
diff --git a/vendor/golang.org/x/sys/windows/zknownfolderids_windows.go b/vendor/golang.org/x/sys/windows/zknownfolderids_windows.go
new file mode 100644
index 0000000..6048ac6
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/zknownfolderids_windows.go
@@ -0,0 +1,149 @@
+// Code generated by 'mkknownfolderids.bash'; DO NOT EDIT.
+
+package windows
+
+type KNOWNFOLDERID GUID
+
+var (
+	FOLDERID_NetworkFolder          = &KNOWNFOLDERID{0xd20beec4, 0x5ca8, 0x4905, [8]byte{0xae, 0x3b, 0xbf, 0x25, 0x1e, 0xa0, 0x9b, 0x53}}
+	FOLDERID_ComputerFolder         = &KNOWNFOLDERID{0x0ac0837c, 0xbbf8, 0x452a, [8]byte{0x85, 0x0d, 0x79, 0xd0, 0x8e, 0x66, 0x7c, 0xa7}}
+	FOLDERID_InternetFolder         = &KNOWNFOLDERID{0x4d9f7874, 0x4e0c, 0x4904, [8]byte{0x96, 0x7b, 0x40, 0xb0, 0xd2, 0x0c, 0x3e, 0x4b}}
+	FOLDERID_ControlPanelFolder     = &KNOWNFOLDERID{0x82a74aeb, 0xaeb4, 0x465c, [8]byte{0xa0, 0x14, 0xd0, 0x97, 0xee, 0x34, 0x6d, 0x63}}
+	FOLDERID_PrintersFolder         = &KNOWNFOLDERID{0x76fc4e2d, 0xd6ad, 0x4519, [8]byte{0xa6, 0x63, 0x37, 0xbd, 0x56, 0x06, 0x81, 0x85}}
+	FOLDERID_SyncManagerFolder      = &KNOWNFOLDERID{0x43668bf8, 0xc14e, 0x49b2, [8]byte{0x97, 0xc9, 0x74, 0x77, 0x84, 0xd7, 0x84, 0xb7}}
+	FOLDERID_SyncSetupFolder        = &KNOWNFOLDERID{0x0f214138, 0xb1d3, 0x4a90, [8]byte{0xbb, 0xa9, 0x27, 0xcb, 0xc0, 0xc5, 0x38, 0x9a}}
+	FOLDERID_ConflictFolder         = &KNOWNFOLDERID{0x4bfefb45, 0x347d, 0x4006, [8]byte{0xa5, 0xbe, 0xac, 0x0c, 0xb0, 0x56, 0x71, 0x92}}
+	FOLDERID_SyncResultsFolder      = &KNOWNFOLDERID{0x289a9a43, 0xbe44, 0x4057, [8]byte{0xa4, 0x1b, 0x58, 0x7a, 0x76, 0xd7, 0xe7, 0xf9}}
+	FOLDERID_RecycleBinFolder       = &KNOWNFOLDERID{0xb7534046, 0x3ecb, 0x4c18, [8]byte{0xbe, 0x4e, 0x64, 0xcd, 0x4c, 0xb7, 0xd6, 0xac}}
+	FOLDERID_ConnectionsFolder      = &KNOWNFOLDERID{0x6f0cd92b, 0x2e97, 0x45d1, [8]byte{0x88, 0xff, 0xb0, 0xd1, 0x86, 0xb8, 0xde, 0xdd}}
+	FOLDERID_Fonts                  = &KNOWNFOLDERID{0xfd228cb7, 0xae11, 0x4ae3, [8]byte{0x86, 0x4c, 0x16, 0xf3, 0x91, 0x0a, 0xb8, 0xfe}}
+	FOLDERID_Desktop                = &KNOWNFOLDERID{0xb4bfcc3a, 0xdb2c, 0x424c, [8]byte{0xb0, 0x29, 0x7f, 0xe9, 0x9a, 0x87, 0xc6, 0x41}}
+	FOLDERID_Startup                = &KNOWNFOLDERID{0xb97d20bb, 0xf46a, 0x4c97, [8]byte{0xba, 0x10, 0x5e, 0x36, 0x08, 0x43, 0x08, 0x54}}
+	FOLDERID_Programs               = &KNOWNFOLDERID{0xa77f5d77, 0x2e2b, 0x44c3, [8]byte{0xa6, 0xa2, 0xab, 0xa6, 0x01, 0x05, 0x4a, 0x51}}
+	FOLDERID_StartMenu              = &KNOWNFOLDERID{0x625b53c3, 0xab48, 0x4ec1, [8]byte{0xba, 0x1f, 0xa1, 0xef, 0x41, 0x46, 0xfc, 0x19}}
+	FOLDERID_Recent                 = &KNOWNFOLDERID{0xae50c081, 0xebd2, 0x438a, [8]byte{0x86, 0x55, 0x8a, 0x09, 0x2e, 0x34, 0x98, 0x7a}}
+	FOLDERID_SendTo                 = &KNOWNFOLDERID{0x8983036c, 0x27c0, 0x404b, [8]byte{0x8f, 0x08, 0x10, 0x2d, 0x10, 0xdc, 0xfd, 0x74}}
+	FOLDERID_Documents              = &KNOWNFOLDERID{0xfdd39ad0, 0x238f, 0x46af, [8]byte{0xad, 0xb4, 0x6c, 0x85, 0x48, 0x03, 0x69, 0xc7}}
+	FOLDERID_Favorites              = &KNOWNFOLDERID{0x1777f761, 0x68ad, 0x4d8a, [8]byte{0x87, 0xbd, 0x30, 0xb7, 0x59, 0xfa, 0x33, 0xdd}}
+	FOLDERID_NetHood                = &KNOWNFOLDERID{0xc5abbf53, 0xe17f, 0x4121, [8]byte{0x89, 0x00, 0x86, 0x62, 0x6f, 0xc2, 0xc9, 0x73}}
+	FOLDERID_PrintHood              = &KNOWNFOLDERID{0x9274bd8d, 0xcfd1, 0x41c3, [8]byte{0xb3, 0x5e, 0xb1, 0x3f, 0x55, 0xa7, 0x58, 0xf4}}
+	FOLDERID_Templates              = &KNOWNFOLDERID{0xa63293e8, 0x664e, 0x48db, [8]byte{0xa0, 0x79, 0xdf, 0x75, 0x9e, 0x05, 0x09, 0xf7}}
+	FOLDERID_CommonStartup          = &KNOWNFOLDERID{0x82a5ea35, 0xd9cd, 0x47c5, [8]byte{0x96, 0x29, 0xe1, 0x5d, 0x2f, 0x71, 0x4e, 0x6e}}
+	FOLDERID_CommonPrograms         = &KNOWNFOLDERID{0x0139d44e, 0x6afe, 0x49f2, [8]byte{0x86, 0x90, 0x3d, 0xaf, 0xca, 0xe6, 0xff, 0xb8}}
+	FOLDERID_CommonStartMenu        = &KNOWNFOLDERID{0xa4115719, 0xd62e, 0x491d, [8]byte{0xaa, 0x7c, 0xe7, 0x4b, 0x8b, 0xe3, 0xb0, 0x67}}
+	FOLDERID_PublicDesktop          = &KNOWNFOLDERID{0xc4aa340d, 0xf20f, 0x4863, [8]byte{0xaf, 0xef, 0xf8, 0x7e, 0xf2, 0xe6, 0xba, 0x25}}
+	FOLDERID_ProgramData            = &KNOWNFOLDERID{0x62ab5d82, 0xfdc1, 0x4dc3, [8]byte{0xa9, 0xdd, 0x07, 0x0d, 0x1d, 0x49, 0x5d, 0x97}}
+	FOLDERID_CommonTemplates        = &KNOWNFOLDERID{0xb94237e7, 0x57ac, 0x4347, [8]byte{0x91, 0x51, 0xb0, 0x8c, 0x6c, 0x32, 0xd1, 0xf7}}
+	FOLDERID_PublicDocuments        = &KNOWNFOLDERID{0xed4824af, 0xdce4, 0x45a8, [8]byte{0x81, 0xe2, 0xfc, 0x79, 0x65, 0x08, 0x36, 0x34}}
+	FOLDERID_RoamingAppData         = &KNOWNFOLDERID{0x3eb685db, 0x65f9, 0x4cf6, [8]byte{0xa0, 0x3a, 0xe3, 0xef, 0x65, 0x72, 0x9f, 0x3d}}
+	FOLDERID_LocalAppData           = &KNOWNFOLDERID{0xf1b32785, 0x6fba, 0x4fcf, [8]byte{0x9d, 0x55, 0x7b, 0x8e, 0x7f, 0x15, 0x70, 0x91}}
+	FOLDERID_LocalAppDataLow        = &KNOWNFOLDERID{0xa520a1a4, 0x1780, 0x4ff6, [8]byte{0xbd, 0x18, 0x16, 0x73, 0x43, 0xc5, 0xaf, 0x16}}
+	FOLDERID_InternetCache          = &KNOWNFOLDERID{0x352481e8, 0x33be, 0x4251, [8]byte{0xba, 0x85, 0x60, 0x07, 0xca, 0xed, 0xcf, 0x9d}}
+	FOLDERID_Cookies                = &KNOWNFOLDERID{0x2b0f765d, 0xc0e9, 0x4171, [8]byte{0x90, 0x8e, 0x08, 0xa6, 0x11, 0xb8, 0x4f, 0xf6}}
+	FOLDERID_History                = &KNOWNFOLDERID{0xd9dc8a3b, 0xb784, 0x432e, [8]byte{0xa7, 0x81, 0x5a, 0x11, 0x30, 0xa7, 0x59, 0x63}}
+	FOLDERID_System                 = &KNOWNFOLDERID{0x1ac14e77, 0x02e7, 0x4e5d, [8]byte{0xb7, 0x44, 0x2e, 0xb1, 0xae, 0x51, 0x98, 0xb7}}
+	FOLDERID_SystemX86              = &KNOWNFOLDERID{0xd65231b0, 0xb2f1, 0x4857, [8]byte{0xa4, 0xce, 0xa8, 0xe7, 0xc6, 0xea, 0x7d, 0x27}}
+	FOLDERID_Windows                = &KNOWNFOLDERID{0xf38bf404, 0x1d43, 0x42f2, [8]byte{0x93, 0x05, 0x67, 0xde, 0x0b, 0x28, 0xfc, 0x23}}
+	FOLDERID_Profile                = &KNOWNFOLDERID{0x5e6c858f, 0x0e22, 0x4760, [8]byte{0x9a, 0xfe, 0xea, 0x33, 0x17, 0xb6, 0x71, 0x73}}
+	FOLDERID_Pictures               = &KNOWNFOLDERID{0x33e28130, 0x4e1e, 0x4676, [8]byte{0x83, 0x5a, 0x98, 0x39, 0x5c, 0x3b, 0xc3, 0xbb}}
+	FOLDERID_ProgramFilesX86        = &KNOWNFOLDERID{0x7c5a40ef, 0xa0fb, 0x4bfc, [8]byte{0x87, 0x4a, 0xc0, 0xf2, 0xe0, 0xb9, 0xfa, 0x8e}}
+	FOLDERID_ProgramFilesCommonX86  = &KNOWNFOLDERID{0xde974d24, 0xd9c6, 0x4d3e, [8]byte{0xbf, 0x91, 0xf4, 0x45, 0x51, 0x20, 0xb9, 0x17}}
+	FOLDERID_ProgramFilesX64        = &KNOWNFOLDERID{0x6d809377, 0x6af0, 0x444b, [8]byte{0x89, 0x57, 0xa3, 0x77, 0x3f, 0x02, 0x20, 0x0e}}
+	FOLDERID_ProgramFilesCommonX64  = &KNOWNFOLDERID{0x6365d5a7, 0x0f0d, 0x45e5, [8]byte{0x87, 0xf6, 0x0d, 0xa5, 0x6b, 0x6a, 0x4f, 0x7d}}
+	FOLDERID_ProgramFiles           = &KNOWNFOLDERID{0x905e63b6, 0xc1bf, 0x494e, [8]byte{0xb2, 0x9c, 0x65, 0xb7, 0x32, 0xd3, 0xd2, 0x1a}}
+	FOLDERID_ProgramFilesCommon     = &KNOWNFOLDERID{0xf7f1ed05, 0x9f6d, 0x47a2, [8]byte{0xaa, 0xae, 0x29, 0xd3, 0x17, 0xc6, 0xf0, 0x66}}
+	FOLDERID_UserProgramFiles       = &KNOWNFOLDERID{0x5cd7aee2, 0x2219, 0x4a67, [8]byte{0xb8, 0x5d, 0x6c, 0x9c, 0xe1, 0x56, 0x60, 0xcb}}
+	FOLDERID_UserProgramFilesCommon = &KNOWNFOLDERID{0xbcbd3057, 0xca5c, 0x4622, [8]byte{0xb4, 0x2d, 0xbc, 0x56, 0xdb, 0x0a, 0xe5, 0x16}}
+	FOLDERID_AdminTools             = &KNOWNFOLDERID{0x724ef170, 0xa42d, 0x4fef, [8]byte{0x9f, 0x26, 0xb6, 0x0e, 0x84, 0x6f, 0xba, 0x4f}}
+	FOLDERID_CommonAdminTools       = &KNOWNFOLDERID{0xd0384e7d, 0xbac3, 0x4797, [8]byte{0x8f, 0x14, 0xcb, 0xa2, 0x29, 0xb3, 0x92, 0xb5}}
+	FOLDERID_Music                  = &KNOWNFOLDERID{0x4bd8d571, 0x6d19, 0x48d3, [8]byte{0xbe, 0x97, 0x42, 0x22, 0x20, 0x08, 0x0e, 0x43}}
+	FOLDERID_Videos                 = &KNOWNFOLDERID{0x18989b1d, 0x99b5, 0x455b, [8]byte{0x84, 0x1c, 0xab, 0x7c, 0x74, 0xe4, 0xdd, 0xfc}}
+	FOLDERID_Ringtones              = &KNOWNFOLDERID{0xc870044b, 0xf49e, 0x4126, [8]byte{0xa9, 0xc3, 0xb5, 0x2a, 0x1f, 0xf4, 0x11, 0xe8}}
+	FOLDERID_PublicPictures         = &KNOWNFOLDERID{0xb6ebfb86, 0x6907, 0x413c, [8]byte{0x9a, 0xf7, 0x4f, 0xc2, 0xab, 0xf0, 0x7c, 0xc5}}
+	FOLDERID_PublicMusic            = &KNOWNFOLDERID{0x3214fab5, 0x9757, 0x4298, [8]byte{0xbb, 0x61, 0x92, 0xa9, 0xde, 0xaa, 0x44, 0xff}}
+	FOLDERID_PublicVideos           = &KNOWNFOLDERID{0x2400183a, 0x6185, 0x49fb, [8]byte{0xa2, 0xd8, 0x4a, 0x39, 0x2a, 0x60, 0x2b, 0xa3}}
+	FOLDERID_PublicRingtones        = &KNOWNFOLDERID{0xe555ab60, 0x153b, 0x4d17, [8]byte{0x9f, 0x04, 0xa5, 0xfe, 0x99, 0xfc, 0x15, 0xec}}
+	FOLDERID_ResourceDir            = &KNOWNFOLDERID{0x8ad10c31, 0x2adb, 0x4296, [8]byte{0xa8, 0xf7, 0xe4, 0x70, 0x12, 0x32, 0xc9, 0x72}}
+	FOLDERID_LocalizedResourcesDir  = &KNOWNFOLDERID{0x2a00375e, 0x224c, 0x49de, [8]byte{0xb8, 0xd1, 0x44, 0x0d, 0xf7, 0xef, 0x3d, 0xdc}}
+	FOLDERID_CommonOEMLinks         = &KNOWNFOLDERID{0xc1bae2d0, 0x10df, 0x4334, [8]byte{0xbe, 0xdd, 0x7a, 0xa2, 0x0b, 0x22, 0x7a, 0x9d}}
+	FOLDERID_CDBurning              = &KNOWNFOLDERID{0x9e52ab10, 0xf80d, 0x49df, [8]byte{0xac, 0xb8, 0x43, 0x30, 0xf5, 0x68, 0x78, 0x55}}
+	FOLDERID_UserProfiles           = &KNOWNFOLDERID{0x0762d272, 0xc50a, 0x4bb0, [8]byte{0xa3, 0x82, 0x69, 0x7d, 0xcd, 0x72, 0x9b, 0x80}}
+	FOLDERID_Playlists              = &KNOWNFOLDERID{0xde92c1c7, 0x837f, 0x4f69, [8]byte{0xa3, 0xbb, 0x86, 0xe6, 0x31, 0x20, 0x4a, 0x23}}
+	FOLDERID_SamplePlaylists        = &KNOWNFOLDERID{0x15ca69b3, 0x30ee, 0x49c1, [8]byte{0xac, 0xe1, 0x6b, 0x5e, 0xc3, 0x72, 0xaf, 0xb5}}
+	FOLDERID_SampleMusic            = &KNOWNFOLDERID{0xb250c668, 0xf57d, 0x4ee1, [8]byte{0xa6, 0x3c, 0x29, 0x0e, 0xe7, 0xd1, 0xaa, 0x1f}}
+	FOLDERID_SamplePictures         = &KNOWNFOLDERID{0xc4900540, 0x2379, 0x4c75, [8]byte{0x84, 0x4b, 0x64, 0xe6, 0xfa, 0xf8, 0x71, 0x6b}}
+	FOLDERID_SampleVideos           = &KNOWNFOLDERID{0x859ead94, 0x2e85, 0x48ad, [8]byte{0xa7, 0x1a, 0x09, 0x69, 0xcb, 0x56, 0xa6, 0xcd}}
+	FOLDERID_PhotoAlbums            = &KNOWNFOLDERID{0x69d2cf90, 0xfc33, 0x4fb7, [8]byte{0x9a, 0x0c, 0xeb, 0xb0, 0xf0, 0xfc, 0xb4, 0x3c}}
+	FOLDERID_Public                 = &KNOWNFOLDERID{0xdfdf76a2, 0xc82a, 0x4d63, [8]byte{0x90, 0x6a, 0x56, 0x44, 0xac, 0x45, 0x73, 0x85}}
+	FOLDERID_ChangeRemovePrograms   = &KNOWNFOLDERID{0xdf7266ac, 0x9274, 0x4867, [8]byte{0x8d, 0x55, 0x3b, 0xd6, 0x61, 0xde, 0x87, 0x2d}}
+	FOLDERID_AppUpdates             = &KNOWNFOLDERID{0xa305ce99, 0xf527, 0x492b, [8]byte{0x8b, 0x1a, 0x7e, 0x76, 0xfa, 0x98, 0xd6, 0xe4}}
+	FOLDERID_AddNewPrograms         = &KNOWNFOLDERID{0xde61d971, 0x5ebc, 0x4f02, [8]byte{0xa3, 0xa9, 0x6c, 0x82, 0x89, 0x5e, 0x5c, 0x04}}
+	FOLDERID_Downloads              = &KNOWNFOLDERID{0x374de290, 0x123f, 0x4565, [8]byte{0x91, 0x64, 0x39, 0xc4, 0x92, 0x5e, 0x46, 0x7b}}
+	FOLDERID_PublicDownloads        = &KNOWNFOLDERID{0x3d644c9b, 0x1fb8, 0x4f30, [8]byte{0x9b, 0x45, 0xf6, 0x70, 0x23, 0x5f, 0x79, 0xc0}}
+	FOLDERID_SavedSearches          = &KNOWNFOLDERID{0x7d1d3a04, 0xdebb, 0x4115, [8]byte{0x95, 0xcf, 0x2f, 0x29, 0xda, 0x29, 0x20, 0xda}}
+	FOLDERID_QuickLaunch            = &KNOWNFOLDERID{0x52a4f021, 0x7b75, 0x48a9, [8]byte{0x9f, 0x6b, 0x4b, 0x87, 0xa2, 0x10, 0xbc, 0x8f}}
+	FOLDERID_Contacts               = &KNOWNFOLDERID{0x56784854, 0xc6cb, 0x462b, [8]byte{0x81, 0x69, 0x88, 0xe3, 0x50, 0xac, 0xb8, 0x82}}
+	FOLDERID_SidebarParts           = &KNOWNFOLDERID{0xa75d362e, 0x50fc, 0x4fb7, [8]byte{0xac, 0x2c, 0xa8, 0xbe, 0xaa, 0x31, 0x44, 0x93}}
+	FOLDERID_SidebarDefaultParts    = &KNOWNFOLDERID{0x7b396e54, 0x9ec5, 0x4300, [8]byte{0xbe, 0x0a, 0x24, 0x82, 0xeb, 0xae, 0x1a, 0x26}}
+	FOLDERID_PublicGameTasks        = &KNOWNFOLDERID{0xdebf2536, 0xe1a8, 0x4c59, [8]byte{0xb6, 0xa2, 0x41, 0x45, 0x86, 0x47, 0x6a, 0xea}}
+	FOLDERID_GameTasks              = &KNOWNFOLDERID{0x054fae61, 0x4dd8, 0x4787, [8]byte{0x80, 0xb6, 0x09, 0x02, 0x20, 0xc4, 0xb7, 0x00}}
+	FOLDERID_SavedGames             = &KNOWNFOLDERID{0x4c5c32ff, 0xbb9d, 0x43b0, [8]byte{0xb5, 0xb4, 0x2d, 0x72, 0xe5, 0x4e, 0xaa, 0xa4}}
+	FOLDERID_Games                  = &KNOWNFOLDERID{0xcac52c1a, 0xb53d, 0x4edc, [8]byte{0x92, 0xd7, 0x6b, 0x2e, 0x8a, 0xc1, 0x94, 0x34}}
+	FOLDERID_SEARCH_MAPI            = &KNOWNFOLDERID{0x98ec0e18, 0x2098, 0x4d44, [8]byte{0x86, 0x44, 0x66, 0x97, 0x93, 0x15, 0xa2, 0x81}}
+	FOLDERID_SEARCH_CSC             = &KNOWNFOLDERID{0xee32e446, 0x31ca, 0x4aba, [8]byte{0x81, 0x4f, 0xa5, 0xeb, 0xd2, 0xfd, 0x6d, 0x5e}}
+	FOLDERID_Links                  = &KNOWNFOLDERID{0xbfb9d5e0, 0xc6a9, 0x404c, [8]byte{0xb2, 0xb2, 0xae, 0x6d, 0xb6, 0xaf, 0x49, 0x68}}
+	FOLDERID_UsersFiles             = &KNOWNFOLDERID{0xf3ce0f7c, 0x4901, 0x4acc, [8]byte{0x86, 0x48, 0xd5, 0xd4, 0x4b, 0x04, 0xef, 0x8f}}
+	FOLDERID_UsersLibraries         = &KNOWNFOLDERID{0xa302545d, 0xdeff, 0x464b, [8]byte{0xab, 0xe8, 0x61, 0xc8, 0x64, 0x8d, 0x93, 0x9b}}
+	FOLDERID_SearchHome             = &KNOWNFOLDERID{0x190337d1, 0xb8ca, 0x4121, [8]byte{0xa6, 0x39, 0x6d, 0x47, 0x2d, 0x16, 0x97, 0x2a}}
+	FOLDERID_OriginalImages         = &KNOWNFOLDERID{0x2c36c0aa, 0x5812, 0x4b87, [8]byte{0xbf, 0xd0, 0x4c, 0xd0, 0xdf, 0xb1, 0x9b, 0x39}}
+	FOLDERID_DocumentsLibrary       = &KNOWNFOLDERID{0x7b0db17d, 0x9cd2, 0x4a93, [8]byte{0x97, 0x33, 0x46, 0xcc, 0x89, 0x02, 0x2e, 0x7c}}
+	FOLDERID_MusicLibrary           = &KNOWNFOLDERID{0x2112ab0a, 0xc86a, 0x4ffe, [8]byte{0xa3, 0x68, 0x0d, 0xe9, 0x6e, 0x47, 0x01, 0x2e}}
+	FOLDERID_PicturesLibrary        = &KNOWNFOLDERID{0xa990ae9f, 0xa03b, 0x4e80, [8]byte{0x94, 0xbc, 0x99, 0x12, 0xd7, 0x50, 0x41, 0x04}}
+	FOLDERID_VideosLibrary          = &KNOWNFOLDERID{0x491e922f, 0x5643, 0x4af4, [8]byte{0xa7, 0xeb, 0x4e, 0x7a, 0x13, 0x8d, 0x81, 0x74}}
+	FOLDERID_RecordedTVLibrary      = &KNOWNFOLDERID{0x1a6fdba2, 0xf42d, 0x4358, [8]byte{0xa7, 0x98, 0xb7, 0x4d, 0x74, 0x59, 0x26, 0xc5}}
+	FOLDERID_HomeGroup              = &KNOWNFOLDERID{0x52528a6b, 0xb9e3, 0x4add, [8]byte{0xb6, 0x0d, 0x58, 0x8c, 0x2d, 0xba, 0x84, 0x2d}}
+	FOLDERID_HomeGroupCurrentUser   = &KNOWNFOLDERID{0x9b74b6a3, 0x0dfd, 0x4f11, [8]byte{0x9e, 0x78, 0x5f, 0x78, 0x00, 0xf2, 0xe7, 0x72}}
+	FOLDERID_DeviceMetadataStore    = &KNOWNFOLDERID{0x5ce4a5e9, 0xe4eb, 0x479d, [8]byte{0xb8, 0x9f, 0x13, 0x0c, 0x02, 0x88, 0x61, 0x55}}
+	FOLDERID_Libraries              = &KNOWNFOLDERID{0x1b3ea5dc, 0xb587, 0x4786, [8]byte{0xb4, 0xef, 0xbd, 0x1d, 0xc3, 0x32, 0xae, 0xae}}
+	FOLDERID_PublicLibraries        = &KNOWNFOLDERID{0x48daf80b, 0xe6cf, 0x4f4e, [8]byte{0xb8, 0x00, 0x0e, 0x69, 0xd8, 0x4e, 0xe3, 0x84}}
+	FOLDERID_UserPinned             = &KNOWNFOLDERID{0x9e3995ab, 0x1f9c, 0x4f13, [8]byte{0xb8, 0x27, 0x48, 0xb2, 0x4b, 0x6c, 0x71, 0x74}}
+	FOLDERID_ImplicitAppShortcuts   = &KNOWNFOLDERID{0xbcb5256f, 0x79f6, 0x4cee, [8]byte{0xb7, 0x25, 0xdc, 0x34, 0xe4, 0x02, 0xfd, 0x46}}
+	FOLDERID_AccountPictures        = &KNOWNFOLDERID{0x008ca0b1, 0x55b4, 0x4c56, [8]byte{0xb8, 0xa8, 0x4d, 0xe4, 0xb2, 0x99, 0xd3, 0xbe}}
+	FOLDERID_PublicUserTiles        = &KNOWNFOLDERID{0x0482af6c, 0x08f1, 0x4c34, [8]byte{0x8c, 0x90, 0xe1, 0x7e, 0xc9, 0x8b, 0x1e, 0x17}}
+	FOLDERID_AppsFolder             = &KNOWNFOLDERID{0x1e87508d, 0x89c2, 0x42f0, [8]byte{0x8a, 0x7e, 0x64, 0x5a, 0x0f, 0x50, 0xca, 0x58}}
+	FOLDERID_StartMenuAllPrograms   = &KNOWNFOLDERID{0xf26305ef, 0x6948, 0x40b9, [8]byte{0xb2, 0x55, 0x81, 0x45, 0x3d, 0x09, 0xc7, 0x85}}
+	FOLDERID_CommonStartMenuPlaces  = &KNOWNFOLDERID{0xa440879f, 0x87a0, 0x4f7d, [8]byte{0xb7, 0x00, 0x02, 0x07, 0xb9, 0x66, 0x19, 0x4a}}
+	FOLDERID_ApplicationShortcuts   = &KNOWNFOLDERID{0xa3918781, 0xe5f2, 0x4890, [8]byte{0xb3, 0xd9, 0xa7, 0xe5, 0x43, 0x32, 0x32, 0x8c}}
+	FOLDERID_RoamingTiles           = &KNOWNFOLDERID{0x00bcfc5a, 0xed94, 0x4e48, [8]byte{0x96, 0xa1, 0x3f, 0x62, 0x17, 0xf2, 0x19, 0x90}}
+	FOLDERID_RoamedTileImages       = &KNOWNFOLDERID{0xaaa8d5a5, 0xf1d6, 0x4259, [8]byte{0xba, 0xa8, 0x78, 0xe7, 0xef, 0x60, 0x83, 0x5e}}
+	FOLDERID_Screenshots            = &KNOWNFOLDERID{0xb7bede81, 0xdf94, 0x4682, [8]byte{0xa7, 0xd8, 0x57, 0xa5, 0x26, 0x20, 0xb8, 0x6f}}
+	FOLDERID_CameraRoll             = &KNOWNFOLDERID{0xab5fb87b, 0x7ce2, 0x4f83, [8]byte{0x91, 0x5d, 0x55, 0x08, 0x46, 0xc9, 0x53, 0x7b}}
+	FOLDERID_SkyDrive               = &KNOWNFOLDERID{0xa52bba46, 0xe9e1, 0x435f, [8]byte{0xb3, 0xd9, 0x28, 0xda, 0xa6, 0x48, 0xc0, 0xf6}}
+	FOLDERID_OneDrive               = &KNOWNFOLDERID{0xa52bba46, 0xe9e1, 0x435f, [8]byte{0xb3, 0xd9, 0x28, 0xda, 0xa6, 0x48, 0xc0, 0xf6}}
+	FOLDERID_SkyDriveDocuments      = &KNOWNFOLDERID{0x24d89e24, 0x2f19, 0x4534, [8]byte{0x9d, 0xde, 0x6a, 0x66, 0x71, 0xfb, 0xb8, 0xfe}}
+	FOLDERID_SkyDrivePictures       = &KNOWNFOLDERID{0x339719b5, 0x8c47, 0x4894, [8]byte{0x94, 0xc2, 0xd8, 0xf7, 0x7a, 0xdd, 0x44, 0xa6}}
+	FOLDERID_SkyDriveMusic          = &KNOWNFOLDERID{0xc3f2459e, 0x80d6, 0x45dc, [8]byte{0xbf, 0xef, 0x1f, 0x76, 0x9f, 0x2b, 0xe7, 0x30}}
+	FOLDERID_SkyDriveCameraRoll     = &KNOWNFOLDERID{0x767e6811, 0x49cb, 0x4273, [8]byte{0x87, 0xc2, 0x20, 0xf3, 0x55, 0xe1, 0x08, 0x5b}}
+	FOLDERID_SearchHistory          = &KNOWNFOLDERID{0x0d4c3db6, 0x03a3, 0x462f, [8]byte{0xa0, 0xe6, 0x08, 0x92, 0x4c, 0x41, 0xb5, 0xd4}}
+	FOLDERID_SearchTemplates        = &KNOWNFOLDERID{0x7e636bfe, 0xdfa9, 0x4d5e, [8]byte{0xb4, 0x56, 0xd7, 0xb3, 0x98, 0x51, 0xd8, 0xa9}}
+	FOLDERID_CameraRollLibrary      = &KNOWNFOLDERID{0x2b20df75, 0x1eda, 0x4039, [8]byte{0x80, 0x97, 0x38, 0x79, 0x82, 0x27, 0xd5, 0xb7}}
+	FOLDERID_SavedPictures          = &KNOWNFOLDERID{0x3b193882, 0xd3ad, 0x4eab, [8]byte{0x96, 0x5a, 0x69, 0x82, 0x9d, 0x1f, 0xb5, 0x9f}}
+	FOLDERID_SavedPicturesLibrary   = &KNOWNFOLDERID{0xe25b5812, 0xbe88, 0x4bd9, [8]byte{0x94, 0xb0, 0x29, 0x23, 0x34, 0x77, 0xb6, 0xc3}}
+	FOLDERID_RetailDemo             = &KNOWNFOLDERID{0x12d4c69e, 0x24ad, 0x4923, [8]byte{0xbe, 0x19, 0x31, 0x32, 0x1c, 0x43, 0xa7, 0x67}}
+	FOLDERID_Device                 = &KNOWNFOLDERID{0x1c2ac1dc, 0x4358, 0x4b6c, [8]byte{0x97, 0x33, 0xaf, 0x21, 0x15, 0x65, 0x76, 0xf0}}
+	FOLDERID_DevelopmentFiles       = &KNOWNFOLDERID{0xdbe8e08e, 0x3053, 0x4bbc, [8]byte{0xb1, 0x83, 0x2a, 0x7b, 0x2b, 0x19, 0x1e, 0x59}}
+	FOLDERID_Objects3D              = &KNOWNFOLDERID{0x31c0dd25, 0x9439, 0x4f12, [8]byte{0xbf, 0x41, 0x7f, 0xf4, 0xed, 0xa3, 0x87, 0x22}}
+	FOLDERID_AppCaptures            = &KNOWNFOLDERID{0xedc0fe71, 0x98d8, 0x4f4a, [8]byte{0xb9, 0x20, 0xc8, 0xdc, 0x13, 0x3c, 0xb1, 0x65}}
+	FOLDERID_LocalDocuments         = &KNOWNFOLDERID{0xf42ee2d3, 0x909f, 0x4907, [8]byte{0x88, 0x71, 0x4c, 0x22, 0xfc, 0x0b, 0xf7, 0x56}}
+	FOLDERID_LocalPictures          = &KNOWNFOLDERID{0x0ddd015d, 0xb06c, 0x45d5, [8]byte{0x8c, 0x4c, 0xf5, 0x97, 0x13, 0x85, 0x46, 0x39}}
+	FOLDERID_LocalVideos            = &KNOWNFOLDERID{0x35286a68, 0x3c57, 0x41a1, [8]byte{0xbb, 0xb1, 0x0e, 0xae, 0x73, 0xd7, 0x6c, 0x95}}
+	FOLDERID_LocalMusic             = &KNOWNFOLDERID{0xa0c69a99, 0x21c8, 0x4671, [8]byte{0x87, 0x03, 0x79, 0x34, 0x16, 0x2f, 0xcf, 0x1d}}
+	FOLDERID_LocalDownloads         = &KNOWNFOLDERID{0x7d83ee9b, 0x2244, 0x4e70, [8]byte{0xb1, 0xf5, 0x53, 0x93, 0x04, 0x2a, 0xf1, 0xe4}}
+	FOLDERID_RecordedCalls          = &KNOWNFOLDERID{0x2f8b40c2, 0x83ed, 0x48ee, [8]byte{0xb3, 0x83, 0xa1, 0xf1, 0x57, 0xec, 0x6f, 0x9a}}
+	FOLDERID_AllAppMods             = &KNOWNFOLDERID{0x7ad67899, 0x66af, 0x43ba, [8]byte{0x91, 0x56, 0x6a, 0xad, 0x42, 0xe6, 0xc5, 0x96}}
+	FOLDERID_CurrentAppMods         = &KNOWNFOLDERID{0x3db40b20, 0x2a30, 0x4dbe, [8]byte{0x91, 0x7e, 0x77, 0x1d, 0xd2, 0x1d, 0xd0, 0x99}}
+	FOLDERID_AppDataDesktop         = &KNOWNFOLDERID{0xb2c5e279, 0x7add, 0x439f, [8]byte{0xb2, 0x8c, 0xc4, 0x1f, 0xe1, 0xbb, 0xf6, 0x72}}
+	FOLDERID_AppDataDocuments       = &KNOWNFOLDERID{0x7be16610, 0x1f7f, 0x44ac, [8]byte{0xbf, 0xf0, 0x83, 0xe1, 0x5f, 0x2f, 0xfc, 0xa1}}
+	FOLDERID_AppDataFavorites       = &KNOWNFOLDERID{0x7cfbefbc, 0xde1f, 0x45aa, [8]byte{0xb8, 0x43, 0xa5, 0x42, 0xac, 0x53, 0x6c, 0xc9}}
+	FOLDERID_AppDataProgramData     = &KNOWNFOLDERID{0x559d40a3, 0xa036, 0x40fa, [8]byte{0xaf, 0x61, 0x84, 0xcb, 0x43, 0x0a, 0x4d, 0x34}}
+)
diff --git a/vendor/golang.org/x/sys/windows/zsyscall_windows.go b/vendor/golang.org/x/sys/windows/zsyscall_windows.go
new file mode 100644
index 0000000..722be24
--- /dev/null
+++ b/vendor/golang.org/x/sys/windows/zsyscall_windows.go
@@ -0,0 +1,3906 @@
+// Code generated by 'go generate'; DO NOT EDIT.
+
+package windows
+
+import (
+	"syscall"
+	"unsafe"
+)
+
+var _ unsafe.Pointer
+
+// Do the interface allocations only once for common
+// Errno values.
+const (
+	errnoERROR_IO_PENDING = 997
+)
+
+var (
+	errERROR_IO_PENDING error = syscall.Errno(errnoERROR_IO_PENDING)
+)
+
+// errnoErr returns common boxed Errno values, to prevent
+// allocations at runtime.
+func errnoErr(e syscall.Errno) error {
+	switch e {
+	case 0:
+		return nil
+	case errnoERROR_IO_PENDING:
+		return errERROR_IO_PENDING
+	}
+	// TODO: add more here, after collecting data on the common
+	// error values see on Windows. (perhaps when running
+	// all.bat?)
+	return e
+}
+
+var (
+	modadvapi32 = NewLazySystemDLL("advapi32.dll")
+	modkernel32 = NewLazySystemDLL("kernel32.dll")
+	modshell32  = NewLazySystemDLL("shell32.dll")
+	moduserenv  = NewLazySystemDLL("userenv.dll")
+	modmswsock  = NewLazySystemDLL("mswsock.dll")
+	modcrypt32  = NewLazySystemDLL("crypt32.dll")
+	moduser32   = NewLazySystemDLL("user32.dll")
+	modole32    = NewLazySystemDLL("ole32.dll")
+	modntdll    = NewLazySystemDLL("ntdll.dll")
+	modws2_32   = NewLazySystemDLL("ws2_32.dll")
+	moddnsapi   = NewLazySystemDLL("dnsapi.dll")
+	modiphlpapi = NewLazySystemDLL("iphlpapi.dll")
+	modsecur32  = NewLazySystemDLL("secur32.dll")
+	modnetapi32 = NewLazySystemDLL("netapi32.dll")
+	modwtsapi32 = NewLazySystemDLL("wtsapi32.dll")
+
+	procRegisterEventSourceW                                 = modadvapi32.NewProc("RegisterEventSourceW")
+	procDeregisterEventSource                                = modadvapi32.NewProc("DeregisterEventSource")
+	procReportEventW                                         = modadvapi32.NewProc("ReportEventW")
+	procOpenSCManagerW                                       = modadvapi32.NewProc("OpenSCManagerW")
+	procCloseServiceHandle                                   = modadvapi32.NewProc("CloseServiceHandle")
+	procCreateServiceW                                       = modadvapi32.NewProc("CreateServiceW")
+	procOpenServiceW                                         = modadvapi32.NewProc("OpenServiceW")
+	procDeleteService                                        = modadvapi32.NewProc("DeleteService")
+	procStartServiceW                                        = modadvapi32.NewProc("StartServiceW")
+	procQueryServiceStatus                                   = modadvapi32.NewProc("QueryServiceStatus")
+	procQueryServiceLockStatusW                              = modadvapi32.NewProc("QueryServiceLockStatusW")
+	procControlService                                       = modadvapi32.NewProc("ControlService")
+	procStartServiceCtrlDispatcherW                          = modadvapi32.NewProc("StartServiceCtrlDispatcherW")
+	procSetServiceStatus                                     = modadvapi32.NewProc("SetServiceStatus")
+	procChangeServiceConfigW                                 = modadvapi32.NewProc("ChangeServiceConfigW")
+	procQueryServiceConfigW                                  = modadvapi32.NewProc("QueryServiceConfigW")
+	procChangeServiceConfig2W                                = modadvapi32.NewProc("ChangeServiceConfig2W")
+	procQueryServiceConfig2W                                 = modadvapi32.NewProc("QueryServiceConfig2W")
+	procEnumServicesStatusExW                                = modadvapi32.NewProc("EnumServicesStatusExW")
+	procQueryServiceStatusEx                                 = modadvapi32.NewProc("QueryServiceStatusEx")
+	procNotifyServiceStatusChangeW                           = modadvapi32.NewProc("NotifyServiceStatusChangeW")
+	procGetLastError                                         = modkernel32.NewProc("GetLastError")
+	procLoadLibraryW                                         = modkernel32.NewProc("LoadLibraryW")
+	procLoadLibraryExW                                       = modkernel32.NewProc("LoadLibraryExW")
+	procFreeLibrary                                          = modkernel32.NewProc("FreeLibrary")
+	procGetProcAddress                                       = modkernel32.NewProc("GetProcAddress")
+	procGetVersion                                           = modkernel32.NewProc("GetVersion")
+	procFormatMessageW                                       = modkernel32.NewProc("FormatMessageW")
+	procExitProcess                                          = modkernel32.NewProc("ExitProcess")
+	procIsWow64Process                                       = modkernel32.NewProc("IsWow64Process")
+	procCreateFileW                                          = modkernel32.NewProc("CreateFileW")
+	procReadFile                                             = modkernel32.NewProc("ReadFile")
+	procWriteFile                                            = modkernel32.NewProc("WriteFile")
+	procGetOverlappedResult                                  = modkernel32.NewProc("GetOverlappedResult")
+	procSetFilePointer                                       = modkernel32.NewProc("SetFilePointer")
+	procCloseHandle                                          = modkernel32.NewProc("CloseHandle")
+	procGetStdHandle                                         = modkernel32.NewProc("GetStdHandle")
+	procSetStdHandle                                         = modkernel32.NewProc("SetStdHandle")
+	procFindFirstFileW                                       = modkernel32.NewProc("FindFirstFileW")
+	procFindNextFileW                                        = modkernel32.NewProc("FindNextFileW")
+	procFindClose                                            = modkernel32.NewProc("FindClose")
+	procGetFileInformationByHandle                           = modkernel32.NewProc("GetFileInformationByHandle")
+	procGetFileInformationByHandleEx                         = modkernel32.NewProc("GetFileInformationByHandleEx")
+	procGetCurrentDirectoryW                                 = modkernel32.NewProc("GetCurrentDirectoryW")
+	procSetCurrentDirectoryW                                 = modkernel32.NewProc("SetCurrentDirectoryW")
+	procCreateDirectoryW                                     = modkernel32.NewProc("CreateDirectoryW")
+	procRemoveDirectoryW                                     = modkernel32.NewProc("RemoveDirectoryW")
+	procDeleteFileW                                          = modkernel32.NewProc("DeleteFileW")
+	procMoveFileW                                            = modkernel32.NewProc("MoveFileW")
+	procMoveFileExW                                          = modkernel32.NewProc("MoveFileExW")
+	procLockFileEx                                           = modkernel32.NewProc("LockFileEx")
+	procUnlockFileEx                                         = modkernel32.NewProc("UnlockFileEx")
+	procGetComputerNameW                                     = modkernel32.NewProc("GetComputerNameW")
+	procGetComputerNameExW                                   = modkernel32.NewProc("GetComputerNameExW")
+	procSetEndOfFile                                         = modkernel32.NewProc("SetEndOfFile")
+	procGetSystemTimeAsFileTime                              = modkernel32.NewProc("GetSystemTimeAsFileTime")
+	procGetSystemTimePreciseAsFileTime                       = modkernel32.NewProc("GetSystemTimePreciseAsFileTime")
+	procGetTimeZoneInformation                               = modkernel32.NewProc("GetTimeZoneInformation")
+	procCreateIoCompletionPort                               = modkernel32.NewProc("CreateIoCompletionPort")
+	procGetQueuedCompletionStatus                            = modkernel32.NewProc("GetQueuedCompletionStatus")
+	procPostQueuedCompletionStatus                           = modkernel32.NewProc("PostQueuedCompletionStatus")
+	procCancelIo                                             = modkernel32.NewProc("CancelIo")
+	procCancelIoEx                                           = modkernel32.NewProc("CancelIoEx")
+	procCreateProcessW                                       = modkernel32.NewProc("CreateProcessW")
+	procOpenProcess                                          = modkernel32.NewProc("OpenProcess")
+	procShellExecuteW                                        = modshell32.NewProc("ShellExecuteW")
+	procSHGetKnownFolderPath                                 = modshell32.NewProc("SHGetKnownFolderPath")
+	procTerminateProcess                                     = modkernel32.NewProc("TerminateProcess")
+	procGetExitCodeProcess                                   = modkernel32.NewProc("GetExitCodeProcess")
+	procGetStartupInfoW                                      = modkernel32.NewProc("GetStartupInfoW")
+	procGetProcessTimes                                      = modkernel32.NewProc("GetProcessTimes")
+	procDuplicateHandle                                      = modkernel32.NewProc("DuplicateHandle")
+	procWaitForSingleObject                                  = modkernel32.NewProc("WaitForSingleObject")
+	procWaitForMultipleObjects                               = modkernel32.NewProc("WaitForMultipleObjects")
+	procGetTempPathW                                         = modkernel32.NewProc("GetTempPathW")
+	procCreatePipe                                           = modkernel32.NewProc("CreatePipe")
+	procGetFileType                                          = modkernel32.NewProc("GetFileType")
+	procCryptAcquireContextW                                 = modadvapi32.NewProc("CryptAcquireContextW")
+	procCryptReleaseContext                                  = modadvapi32.NewProc("CryptReleaseContext")
+	procCryptGenRandom                                       = modadvapi32.NewProc("CryptGenRandom")
+	procGetEnvironmentStringsW                               = modkernel32.NewProc("GetEnvironmentStringsW")
+	procFreeEnvironmentStringsW                              = modkernel32.NewProc("FreeEnvironmentStringsW")
+	procGetEnvironmentVariableW                              = modkernel32.NewProc("GetEnvironmentVariableW")
+	procSetEnvironmentVariableW                              = modkernel32.NewProc("SetEnvironmentVariableW")
+	procCreateEnvironmentBlock                               = moduserenv.NewProc("CreateEnvironmentBlock")
+	procDestroyEnvironmentBlock                              = moduserenv.NewProc("DestroyEnvironmentBlock")
+	procGetTickCount64                                       = modkernel32.NewProc("GetTickCount64")
+	procSetFileTime                                          = modkernel32.NewProc("SetFileTime")
+	procGetFileAttributesW                                   = modkernel32.NewProc("GetFileAttributesW")
+	procSetFileAttributesW                                   = modkernel32.NewProc("SetFileAttributesW")
+	procGetFileAttributesExW                                 = modkernel32.NewProc("GetFileAttributesExW")
+	procGetCommandLineW                                      = modkernel32.NewProc("GetCommandLineW")
+	procCommandLineToArgvW                                   = modshell32.NewProc("CommandLineToArgvW")
+	procLocalFree                                            = modkernel32.NewProc("LocalFree")
+	procSetHandleInformation                                 = modkernel32.NewProc("SetHandleInformation")
+	procFlushFileBuffers                                     = modkernel32.NewProc("FlushFileBuffers")
+	procGetFullPathNameW                                     = modkernel32.NewProc("GetFullPathNameW")
+	procGetLongPathNameW                                     = modkernel32.NewProc("GetLongPathNameW")
+	procGetShortPathNameW                                    = modkernel32.NewProc("GetShortPathNameW")
+	procCreateFileMappingW                                   = modkernel32.NewProc("CreateFileMappingW")
+	procMapViewOfFile                                        = modkernel32.NewProc("MapViewOfFile")
+	procUnmapViewOfFile                                      = modkernel32.NewProc("UnmapViewOfFile")
+	procFlushViewOfFile                                      = modkernel32.NewProc("FlushViewOfFile")
+	procVirtualLock                                          = modkernel32.NewProc("VirtualLock")
+	procVirtualUnlock                                        = modkernel32.NewProc("VirtualUnlock")
+	procVirtualAlloc                                         = modkernel32.NewProc("VirtualAlloc")
+	procVirtualFree                                          = modkernel32.NewProc("VirtualFree")
+	procVirtualProtect                                       = modkernel32.NewProc("VirtualProtect")
+	procTransmitFile                                         = modmswsock.NewProc("TransmitFile")
+	procReadDirectoryChangesW                                = modkernel32.NewProc("ReadDirectoryChangesW")
+	procCertOpenSystemStoreW                                 = modcrypt32.NewProc("CertOpenSystemStoreW")
+	procCertOpenStore                                        = modcrypt32.NewProc("CertOpenStore")
+	procCertEnumCertificatesInStore                          = modcrypt32.NewProc("CertEnumCertificatesInStore")
+	procCertAddCertificateContextToStore                     = modcrypt32.NewProc("CertAddCertificateContextToStore")
+	procCertCloseStore                                       = modcrypt32.NewProc("CertCloseStore")
+	procCertGetCertificateChain                              = modcrypt32.NewProc("CertGetCertificateChain")
+	procCertFreeCertificateChain                             = modcrypt32.NewProc("CertFreeCertificateChain")
+	procCertCreateCertificateContext                         = modcrypt32.NewProc("CertCreateCertificateContext")
+	procCertFreeCertificateContext                           = modcrypt32.NewProc("CertFreeCertificateContext")
+	procCertVerifyCertificateChainPolicy                     = modcrypt32.NewProc("CertVerifyCertificateChainPolicy")
+	procRegOpenKeyExW                                        = modadvapi32.NewProc("RegOpenKeyExW")
+	procRegCloseKey                                          = modadvapi32.NewProc("RegCloseKey")
+	procRegQueryInfoKeyW                                     = modadvapi32.NewProc("RegQueryInfoKeyW")
+	procRegEnumKeyExW                                        = modadvapi32.NewProc("RegEnumKeyExW")
+	procRegQueryValueExW                                     = modadvapi32.NewProc("RegQueryValueExW")
+	procGetCurrentProcessId                                  = modkernel32.NewProc("GetCurrentProcessId")
+	procGetConsoleMode                                       = modkernel32.NewProc("GetConsoleMode")
+	procSetConsoleMode                                       = modkernel32.NewProc("SetConsoleMode")
+	procGetConsoleScreenBufferInfo                           = modkernel32.NewProc("GetConsoleScreenBufferInfo")
+	procWriteConsoleW                                        = modkernel32.NewProc("WriteConsoleW")
+	procReadConsoleW                                         = modkernel32.NewProc("ReadConsoleW")
+	procCreateToolhelp32Snapshot                             = modkernel32.NewProc("CreateToolhelp32Snapshot")
+	procProcess32FirstW                                      = modkernel32.NewProc("Process32FirstW")
+	procProcess32NextW                                       = modkernel32.NewProc("Process32NextW")
+	procThread32First                                        = modkernel32.NewProc("Thread32First")
+	procThread32Next                                         = modkernel32.NewProc("Thread32Next")
+	procDeviceIoControl                                      = modkernel32.NewProc("DeviceIoControl")
+	procCreateSymbolicLinkW                                  = modkernel32.NewProc("CreateSymbolicLinkW")
+	procCreateHardLinkW                                      = modkernel32.NewProc("CreateHardLinkW")
+	procGetCurrentThreadId                                   = modkernel32.NewProc("GetCurrentThreadId")
+	procCreateEventW                                         = modkernel32.NewProc("CreateEventW")
+	procCreateEventExW                                       = modkernel32.NewProc("CreateEventExW")
+	procOpenEventW                                           = modkernel32.NewProc("OpenEventW")
+	procSetEvent                                             = modkernel32.NewProc("SetEvent")
+	procResetEvent                                           = modkernel32.NewProc("ResetEvent")
+	procPulseEvent                                           = modkernel32.NewProc("PulseEvent")
+	procCreateMutexW                                         = modkernel32.NewProc("CreateMutexW")
+	procCreateMutexExW                                       = modkernel32.NewProc("CreateMutexExW")
+	procOpenMutexW                                           = modkernel32.NewProc("OpenMutexW")
+	procReleaseMutex                                         = modkernel32.NewProc("ReleaseMutex")
+	procSleepEx                                              = modkernel32.NewProc("SleepEx")
+	procCreateJobObjectW                                     = modkernel32.NewProc("CreateJobObjectW")
+	procAssignProcessToJobObject                             = modkernel32.NewProc("AssignProcessToJobObject")
+	procTerminateJobObject                                   = modkernel32.NewProc("TerminateJobObject")
+	procSetErrorMode                                         = modkernel32.NewProc("SetErrorMode")
+	procResumeThread                                         = modkernel32.NewProc("ResumeThread")
+	procSetPriorityClass                                     = modkernel32.NewProc("SetPriorityClass")
+	procGetPriorityClass                                     = modkernel32.NewProc("GetPriorityClass")
+	procSetInformationJobObject                              = modkernel32.NewProc("SetInformationJobObject")
+	procGenerateConsoleCtrlEvent                             = modkernel32.NewProc("GenerateConsoleCtrlEvent")
+	procGetProcessId                                         = modkernel32.NewProc("GetProcessId")
+	procOpenThread                                           = modkernel32.NewProc("OpenThread")
+	procSetProcessPriorityBoost                              = modkernel32.NewProc("SetProcessPriorityBoost")
+	procDefineDosDeviceW                                     = modkernel32.NewProc("DefineDosDeviceW")
+	procDeleteVolumeMountPointW                              = modkernel32.NewProc("DeleteVolumeMountPointW")
+	procFindFirstVolumeW                                     = modkernel32.NewProc("FindFirstVolumeW")
+	procFindFirstVolumeMountPointW                           = modkernel32.NewProc("FindFirstVolumeMountPointW")
+	procFindNextVolumeW                                      = modkernel32.NewProc("FindNextVolumeW")
+	procFindNextVolumeMountPointW                            = modkernel32.NewProc("FindNextVolumeMountPointW")
+	procFindVolumeClose                                      = modkernel32.NewProc("FindVolumeClose")
+	procFindVolumeMountPointClose                            = modkernel32.NewProc("FindVolumeMountPointClose")
+	procGetDriveTypeW                                        = modkernel32.NewProc("GetDriveTypeW")
+	procGetLogicalDrives                                     = modkernel32.NewProc("GetLogicalDrives")
+	procGetLogicalDriveStringsW                              = modkernel32.NewProc("GetLogicalDriveStringsW")
+	procGetVolumeInformationW                                = modkernel32.NewProc("GetVolumeInformationW")
+	procGetVolumeInformationByHandleW                        = modkernel32.NewProc("GetVolumeInformationByHandleW")
+	procGetVolumeNameForVolumeMountPointW                    = modkernel32.NewProc("GetVolumeNameForVolumeMountPointW")
+	procGetVolumePathNameW                                   = modkernel32.NewProc("GetVolumePathNameW")
+	procGetVolumePathNamesForVolumeNameW                     = modkernel32.NewProc("GetVolumePathNamesForVolumeNameW")
+	procQueryDosDeviceW                                      = modkernel32.NewProc("QueryDosDeviceW")
+	procSetVolumeLabelW                                      = modkernel32.NewProc("SetVolumeLabelW")
+	procSetVolumeMountPointW                                 = modkernel32.NewProc("SetVolumeMountPointW")
+	procMessageBoxW                                          = moduser32.NewProc("MessageBoxW")
+	procExitWindowsEx                                        = moduser32.NewProc("ExitWindowsEx")
+	procInitiateSystemShutdownExW                            = modadvapi32.NewProc("InitiateSystemShutdownExW")
+	procSetProcessShutdownParameters                         = modkernel32.NewProc("SetProcessShutdownParameters")
+	procGetProcessShutdownParameters                         = modkernel32.NewProc("GetProcessShutdownParameters")
+	procCLSIDFromString                                      = modole32.NewProc("CLSIDFromString")
+	procStringFromGUID2                                      = modole32.NewProc("StringFromGUID2")
+	procCoCreateGuid                                         = modole32.NewProc("CoCreateGuid")
+	procCoTaskMemFree                                        = modole32.NewProc("CoTaskMemFree")
+	procRtlGetVersion                                        = modntdll.NewProc("RtlGetVersion")
+	procRtlGetNtVersionNumbers                               = modntdll.NewProc("RtlGetNtVersionNumbers")
+	procWSAStartup                                           = modws2_32.NewProc("WSAStartup")
+	procWSACleanup                                           = modws2_32.NewProc("WSACleanup")
+	procWSAIoctl                                             = modws2_32.NewProc("WSAIoctl")
+	procsocket                                               = modws2_32.NewProc("socket")
+	procsetsockopt                                           = modws2_32.NewProc("setsockopt")
+	procgetsockopt                                           = modws2_32.NewProc("getsockopt")
+	procbind                                                 = modws2_32.NewProc("bind")
+	procconnect                                              = modws2_32.NewProc("connect")
+	procgetsockname                                          = modws2_32.NewProc("getsockname")
+	procgetpeername                                          = modws2_32.NewProc("getpeername")
+	proclisten                                               = modws2_32.NewProc("listen")
+	procshutdown                                             = modws2_32.NewProc("shutdown")
+	procclosesocket                                          = modws2_32.NewProc("closesocket")
+	procAcceptEx                                             = modmswsock.NewProc("AcceptEx")
+	procGetAcceptExSockaddrs                                 = modmswsock.NewProc("GetAcceptExSockaddrs")
+	procWSARecv                                              = modws2_32.NewProc("WSARecv")
+	procWSASend                                              = modws2_32.NewProc("WSASend")
+	procWSARecvFrom                                          = modws2_32.NewProc("WSARecvFrom")
+	procWSASendTo                                            = modws2_32.NewProc("WSASendTo")
+	procgethostbyname                                        = modws2_32.NewProc("gethostbyname")
+	procgetservbyname                                        = modws2_32.NewProc("getservbyname")
+	procntohs                                                = modws2_32.NewProc("ntohs")
+	procgetprotobyname                                       = modws2_32.NewProc("getprotobyname")
+	procDnsQuery_W                                           = moddnsapi.NewProc("DnsQuery_W")
+	procDnsRecordListFree                                    = moddnsapi.NewProc("DnsRecordListFree")
+	procDnsNameCompare_W                                     = moddnsapi.NewProc("DnsNameCompare_W")
+	procGetAddrInfoW                                         = modws2_32.NewProc("GetAddrInfoW")
+	procFreeAddrInfoW                                        = modws2_32.NewProc("FreeAddrInfoW")
+	procGetIfEntry                                           = modiphlpapi.NewProc("GetIfEntry")
+	procGetAdaptersInfo                                      = modiphlpapi.NewProc("GetAdaptersInfo")
+	procSetFileCompletionNotificationModes                   = modkernel32.NewProc("SetFileCompletionNotificationModes")
+	procWSAEnumProtocolsW                                    = modws2_32.NewProc("WSAEnumProtocolsW")
+	procGetAdaptersAddresses                                 = modiphlpapi.NewProc("GetAdaptersAddresses")
+	procGetACP                                               = modkernel32.NewProc("GetACP")
+	procMultiByteToWideChar                                  = modkernel32.NewProc("MultiByteToWideChar")
+	procTranslateNameW                                       = modsecur32.NewProc("TranslateNameW")
+	procGetUserNameExW                                       = modsecur32.NewProc("GetUserNameExW")
+	procNetUserGetInfo                                       = modnetapi32.NewProc("NetUserGetInfo")
+	procNetGetJoinInformation                                = modnetapi32.NewProc("NetGetJoinInformation")
+	procNetApiBufferFree                                     = modnetapi32.NewProc("NetApiBufferFree")
+	procLookupAccountSidW                                    = modadvapi32.NewProc("LookupAccountSidW")
+	procLookupAccountNameW                                   = modadvapi32.NewProc("LookupAccountNameW")
+	procConvertSidToStringSidW                               = modadvapi32.NewProc("ConvertSidToStringSidW")
+	procConvertStringSidToSidW                               = modadvapi32.NewProc("ConvertStringSidToSidW")
+	procGetLengthSid                                         = modadvapi32.NewProc("GetLengthSid")
+	procCopySid                                              = modadvapi32.NewProc("CopySid")
+	procAllocateAndInitializeSid                             = modadvapi32.NewProc("AllocateAndInitializeSid")
+	procCreateWellKnownSid                                   = modadvapi32.NewProc("CreateWellKnownSid")
+	procIsWellKnownSid                                       = modadvapi32.NewProc("IsWellKnownSid")
+	procFreeSid                                              = modadvapi32.NewProc("FreeSid")
+	procEqualSid                                             = modadvapi32.NewProc("EqualSid")
+	procGetSidIdentifierAuthority                            = modadvapi32.NewProc("GetSidIdentifierAuthority")
+	procGetSidSubAuthorityCount                              = modadvapi32.NewProc("GetSidSubAuthorityCount")
+	procGetSidSubAuthority                                   = modadvapi32.NewProc("GetSidSubAuthority")
+	procIsValidSid                                           = modadvapi32.NewProc("IsValidSid")
+	procCheckTokenMembership                                 = modadvapi32.NewProc("CheckTokenMembership")
+	procOpenProcessToken                                     = modadvapi32.NewProc("OpenProcessToken")
+	procOpenThreadToken                                      = modadvapi32.NewProc("OpenThreadToken")
+	procImpersonateSelf                                      = modadvapi32.NewProc("ImpersonateSelf")
+	procRevertToSelf                                         = modadvapi32.NewProc("RevertToSelf")
+	procSetThreadToken                                       = modadvapi32.NewProc("SetThreadToken")
+	procLookupPrivilegeValueW                                = modadvapi32.NewProc("LookupPrivilegeValueW")
+	procAdjustTokenPrivileges                                = modadvapi32.NewProc("AdjustTokenPrivileges")
+	procAdjustTokenGroups                                    = modadvapi32.NewProc("AdjustTokenGroups")
+	procGetTokenInformation                                  = modadvapi32.NewProc("GetTokenInformation")
+	procSetTokenInformation                                  = modadvapi32.NewProc("SetTokenInformation")
+	procDuplicateTokenEx                                     = modadvapi32.NewProc("DuplicateTokenEx")
+	procGetUserProfileDirectoryW                             = moduserenv.NewProc("GetUserProfileDirectoryW")
+	procGetSystemDirectoryW                                  = modkernel32.NewProc("GetSystemDirectoryW")
+	procGetWindowsDirectoryW                                 = modkernel32.NewProc("GetWindowsDirectoryW")
+	procGetSystemWindowsDirectoryW                           = modkernel32.NewProc("GetSystemWindowsDirectoryW")
+	procWTSQueryUserToken                                    = modwtsapi32.NewProc("WTSQueryUserToken")
+	procWTSEnumerateSessionsW                                = modwtsapi32.NewProc("WTSEnumerateSessionsW")
+	procWTSFreeMemory                                        = modwtsapi32.NewProc("WTSFreeMemory")
+	procGetSecurityInfo                                      = modadvapi32.NewProc("GetSecurityInfo")
+	procSetSecurityInfo                                      = modadvapi32.NewProc("SetSecurityInfo")
+	procGetNamedSecurityInfoW                                = modadvapi32.NewProc("GetNamedSecurityInfoW")
+	procSetNamedSecurityInfoW                                = modadvapi32.NewProc("SetNamedSecurityInfoW")
+	procBuildSecurityDescriptorW                             = modadvapi32.NewProc("BuildSecurityDescriptorW")
+	procInitializeSecurityDescriptor                         = modadvapi32.NewProc("InitializeSecurityDescriptor")
+	procGetSecurityDescriptorControl                         = modadvapi32.NewProc("GetSecurityDescriptorControl")
+	procGetSecurityDescriptorDacl                            = modadvapi32.NewProc("GetSecurityDescriptorDacl")
+	procGetSecurityDescriptorSacl                            = modadvapi32.NewProc("GetSecurityDescriptorSacl")
+	procGetSecurityDescriptorOwner                           = modadvapi32.NewProc("GetSecurityDescriptorOwner")
+	procGetSecurityDescriptorGroup                           = modadvapi32.NewProc("GetSecurityDescriptorGroup")
+	procGetSecurityDescriptorLength                          = modadvapi32.NewProc("GetSecurityDescriptorLength")
+	procGetSecurityDescriptorRMControl                       = modadvapi32.NewProc("GetSecurityDescriptorRMControl")
+	procIsValidSecurityDescriptor                            = modadvapi32.NewProc("IsValidSecurityDescriptor")
+	procSetSecurityDescriptorControl                         = modadvapi32.NewProc("SetSecurityDescriptorControl")
+	procSetSecurityDescriptorDacl                            = modadvapi32.NewProc("SetSecurityDescriptorDacl")
+	procSetSecurityDescriptorSacl                            = modadvapi32.NewProc("SetSecurityDescriptorSacl")
+	procSetSecurityDescriptorOwner                           = modadvapi32.NewProc("SetSecurityDescriptorOwner")
+	procSetSecurityDescriptorGroup                           = modadvapi32.NewProc("SetSecurityDescriptorGroup")
+	procSetSecurityDescriptorRMControl                       = modadvapi32.NewProc("SetSecurityDescriptorRMControl")
+	procConvertStringSecurityDescriptorToSecurityDescriptorW = modadvapi32.NewProc("ConvertStringSecurityDescriptorToSecurityDescriptorW")
+	procConvertSecurityDescriptorToStringSecurityDescriptorW = modadvapi32.NewProc("ConvertSecurityDescriptorToStringSecurityDescriptorW")
+	procMakeAbsoluteSD                                       = modadvapi32.NewProc("MakeAbsoluteSD")
+	procMakeSelfRelativeSD                                   = modadvapi32.NewProc("MakeSelfRelativeSD")
+	procSetEntriesInAclW                                     = modadvapi32.NewProc("SetEntriesInAclW")
+)
+
+func RegisterEventSource(uncServerName *uint16, sourceName *uint16) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall(procRegisterEventSourceW.Addr(), 2, uintptr(unsafe.Pointer(uncServerName)), uintptr(unsafe.Pointer(sourceName)), 0)
+	handle = Handle(r0)
+	if handle == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func DeregisterEventSource(handle Handle) (err error) {
+	r1, _, e1 := syscall.Syscall(procDeregisterEventSource.Addr(), 1, uintptr(handle), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func ReportEvent(log Handle, etype uint16, category uint16, eventId uint32, usrSId uintptr, numStrings uint16, dataSize uint32, strings **uint16, rawData *byte) (err error) {
+	r1, _, e1 := syscall.Syscall9(procReportEventW.Addr(), 9, uintptr(log), uintptr(etype), uintptr(category), uintptr(eventId), uintptr(usrSId), uintptr(numStrings), uintptr(dataSize), uintptr(unsafe.Pointer(strings)), uintptr(unsafe.Pointer(rawData)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func OpenSCManager(machineName *uint16, databaseName *uint16, access uint32) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall(procOpenSCManagerW.Addr(), 3, uintptr(unsafe.Pointer(machineName)), uintptr(unsafe.Pointer(databaseName)), uintptr(access))
+	handle = Handle(r0)
+	if handle == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CloseServiceHandle(handle Handle) (err error) {
+	r1, _, e1 := syscall.Syscall(procCloseServiceHandle.Addr(), 1, uintptr(handle), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CreateService(mgr Handle, serviceName *uint16, displayName *uint16, access uint32, srvType uint32, startType uint32, errCtl uint32, pathName *uint16, loadOrderGroup *uint16, tagId *uint32, dependencies *uint16, serviceStartName *uint16, password *uint16) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall15(procCreateServiceW.Addr(), 13, uintptr(mgr), uintptr(unsafe.Pointer(serviceName)), uintptr(unsafe.Pointer(displayName)), uintptr(access), uintptr(srvType), uintptr(startType), uintptr(errCtl), uintptr(unsafe.Pointer(pathName)), uintptr(unsafe.Pointer(loadOrderGroup)), uintptr(unsafe.Pointer(tagId)), uintptr(unsafe.Pointer(dependencies)), uintptr(unsafe.Pointer(serviceStartName)), uintptr(unsafe.Pointer(password)), 0, 0)
+	handle = Handle(r0)
+	if handle == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func OpenService(mgr Handle, serviceName *uint16, access uint32) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall(procOpenServiceW.Addr(), 3, uintptr(mgr), uintptr(unsafe.Pointer(serviceName)), uintptr(access))
+	handle = Handle(r0)
+	if handle == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func DeleteService(service Handle) (err error) {
+	r1, _, e1 := syscall.Syscall(procDeleteService.Addr(), 1, uintptr(service), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func StartService(service Handle, numArgs uint32, argVectors **uint16) (err error) {
+	r1, _, e1 := syscall.Syscall(procStartServiceW.Addr(), 3, uintptr(service), uintptr(numArgs), uintptr(unsafe.Pointer(argVectors)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func QueryServiceStatus(service Handle, status *SERVICE_STATUS) (err error) {
+	r1, _, e1 := syscall.Syscall(procQueryServiceStatus.Addr(), 2, uintptr(service), uintptr(unsafe.Pointer(status)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func QueryServiceLockStatus(mgr Handle, lockStatus *QUERY_SERVICE_LOCK_STATUS, bufSize uint32, bytesNeeded *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procQueryServiceLockStatusW.Addr(), 4, uintptr(mgr), uintptr(unsafe.Pointer(lockStatus)), uintptr(bufSize), uintptr(unsafe.Pointer(bytesNeeded)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func ControlService(service Handle, control uint32, status *SERVICE_STATUS) (err error) {
+	r1, _, e1 := syscall.Syscall(procControlService.Addr(), 3, uintptr(service), uintptr(control), uintptr(unsafe.Pointer(status)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func StartServiceCtrlDispatcher(serviceTable *SERVICE_TABLE_ENTRY) (err error) {
+	r1, _, e1 := syscall.Syscall(procStartServiceCtrlDispatcherW.Addr(), 1, uintptr(unsafe.Pointer(serviceTable)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetServiceStatus(service Handle, serviceStatus *SERVICE_STATUS) (err error) {
+	r1, _, e1 := syscall.Syscall(procSetServiceStatus.Addr(), 2, uintptr(service), uintptr(unsafe.Pointer(serviceStatus)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func ChangeServiceConfig(service Handle, serviceType uint32, startType uint32, errorControl uint32, binaryPathName *uint16, loadOrderGroup *uint16, tagId *uint32, dependencies *uint16, serviceStartName *uint16, password *uint16, displayName *uint16) (err error) {
+	r1, _, e1 := syscall.Syscall12(procChangeServiceConfigW.Addr(), 11, uintptr(service), uintptr(serviceType), uintptr(startType), uintptr(errorControl), uintptr(unsafe.Pointer(binaryPathName)), uintptr(unsafe.Pointer(loadOrderGroup)), uintptr(unsafe.Pointer(tagId)), uintptr(unsafe.Pointer(dependencies)), uintptr(unsafe.Pointer(serviceStartName)), uintptr(unsafe.Pointer(password)), uintptr(unsafe.Pointer(displayName)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func QueryServiceConfig(service Handle, serviceConfig *QUERY_SERVICE_CONFIG, bufSize uint32, bytesNeeded *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procQueryServiceConfigW.Addr(), 4, uintptr(service), uintptr(unsafe.Pointer(serviceConfig)), uintptr(bufSize), uintptr(unsafe.Pointer(bytesNeeded)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func ChangeServiceConfig2(service Handle, infoLevel uint32, info *byte) (err error) {
+	r1, _, e1 := syscall.Syscall(procChangeServiceConfig2W.Addr(), 3, uintptr(service), uintptr(infoLevel), uintptr(unsafe.Pointer(info)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func QueryServiceConfig2(service Handle, infoLevel uint32, buff *byte, buffSize uint32, bytesNeeded *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procQueryServiceConfig2W.Addr(), 5, uintptr(service), uintptr(infoLevel), uintptr(unsafe.Pointer(buff)), uintptr(buffSize), uintptr(unsafe.Pointer(bytesNeeded)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func EnumServicesStatusEx(mgr Handle, infoLevel uint32, serviceType uint32, serviceState uint32, services *byte, bufSize uint32, bytesNeeded *uint32, servicesReturned *uint32, resumeHandle *uint32, groupName *uint16) (err error) {
+	r1, _, e1 := syscall.Syscall12(procEnumServicesStatusExW.Addr(), 10, uintptr(mgr), uintptr(infoLevel), uintptr(serviceType), uintptr(serviceState), uintptr(unsafe.Pointer(services)), uintptr(bufSize), uintptr(unsafe.Pointer(bytesNeeded)), uintptr(unsafe.Pointer(servicesReturned)), uintptr(unsafe.Pointer(resumeHandle)), uintptr(unsafe.Pointer(groupName)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func QueryServiceStatusEx(service Handle, infoLevel uint32, buff *byte, buffSize uint32, bytesNeeded *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procQueryServiceStatusEx.Addr(), 5, uintptr(service), uintptr(infoLevel), uintptr(unsafe.Pointer(buff)), uintptr(buffSize), uintptr(unsafe.Pointer(bytesNeeded)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func NotifyServiceStatusChange(service Handle, notifyMask uint32, notifier *SERVICE_NOTIFY) (ret error) {
+	r0, _, _ := syscall.Syscall(procNotifyServiceStatusChangeW.Addr(), 3, uintptr(service), uintptr(notifyMask), uintptr(unsafe.Pointer(notifier)))
+	if r0 != 0 {
+		ret = syscall.Errno(r0)
+	}
+	return
+}
+
+func GetLastError() (lasterr error) {
+	r0, _, _ := syscall.Syscall(procGetLastError.Addr(), 0, 0, 0, 0)
+	if r0 != 0 {
+		lasterr = syscall.Errno(r0)
+	}
+	return
+}
+
+func LoadLibrary(libname string) (handle Handle, err error) {
+	var _p0 *uint16
+	_p0, err = syscall.UTF16PtrFromString(libname)
+	if err != nil {
+		return
+	}
+	return _LoadLibrary(_p0)
+}
+
+func _LoadLibrary(libname *uint16) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall(procLoadLibraryW.Addr(), 1, uintptr(unsafe.Pointer(libname)), 0, 0)
+	handle = Handle(r0)
+	if handle == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func LoadLibraryEx(libname string, zero Handle, flags uintptr) (handle Handle, err error) {
+	var _p0 *uint16
+	_p0, err = syscall.UTF16PtrFromString(libname)
+	if err != nil {
+		return
+	}
+	return _LoadLibraryEx(_p0, zero, flags)
+}
+
+func _LoadLibraryEx(libname *uint16, zero Handle, flags uintptr) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall(procLoadLibraryExW.Addr(), 3, uintptr(unsafe.Pointer(libname)), uintptr(zero), uintptr(flags))
+	handle = Handle(r0)
+	if handle == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func FreeLibrary(handle Handle) (err error) {
+	r1, _, e1 := syscall.Syscall(procFreeLibrary.Addr(), 1, uintptr(handle), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetProcAddress(module Handle, procname string) (proc uintptr, err error) {
+	var _p0 *byte
+	_p0, err = syscall.BytePtrFromString(procname)
+	if err != nil {
+		return
+	}
+	return _GetProcAddress(module, _p0)
+}
+
+func _GetProcAddress(module Handle, procname *byte) (proc uintptr, err error) {
+	r0, _, e1 := syscall.Syscall(procGetProcAddress.Addr(), 2, uintptr(module), uintptr(unsafe.Pointer(procname)), 0)
+	proc = uintptr(r0)
+	if proc == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetVersion() (ver uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procGetVersion.Addr(), 0, 0, 0, 0)
+	ver = uint32(r0)
+	if ver == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func FormatMessage(flags uint32, msgsrc uintptr, msgid uint32, langid uint32, buf []uint16, args *byte) (n uint32, err error) {
+	var _p0 *uint16
+	if len(buf) > 0 {
+		_p0 = &buf[0]
+	}
+	r0, _, e1 := syscall.Syscall9(procFormatMessageW.Addr(), 7, uintptr(flags), uintptr(msgsrc), uintptr(msgid), uintptr(langid), uintptr(unsafe.Pointer(_p0)), uintptr(len(buf)), uintptr(unsafe.Pointer(args)), 0, 0)
+	n = uint32(r0)
+	if n == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func ExitProcess(exitcode uint32) {
+	syscall.Syscall(procExitProcess.Addr(), 1, uintptr(exitcode), 0, 0)
+	return
+}
+
+func IsWow64Process(handle Handle, isWow64 *bool) (err error) {
+	var _p0 uint32
+	if *isWow64 {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r1, _, e1 := syscall.Syscall(procIsWow64Process.Addr(), 2, uintptr(handle), uintptr(unsafe.Pointer(&_p0)), 0)
+	*isWow64 = _p0 != 0
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CreateFile(name *uint16, access uint32, mode uint32, sa *SecurityAttributes, createmode uint32, attrs uint32, templatefile Handle) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall9(procCreateFileW.Addr(), 7, uintptr(unsafe.Pointer(name)), uintptr(access), uintptr(mode), uintptr(unsafe.Pointer(sa)), uintptr(createmode), uintptr(attrs), uintptr(templatefile), 0, 0)
+	handle = Handle(r0)
+	if handle == InvalidHandle {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func ReadFile(handle Handle, buf []byte, done *uint32, overlapped *Overlapped) (err error) {
+	var _p0 *byte
+	if len(buf) > 0 {
+		_p0 = &buf[0]
+	}
+	r1, _, e1 := syscall.Syscall6(procReadFile.Addr(), 5, uintptr(handle), uintptr(unsafe.Pointer(_p0)), uintptr(len(buf)), uintptr(unsafe.Pointer(done)), uintptr(unsafe.Pointer(overlapped)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func WriteFile(handle Handle, buf []byte, done *uint32, overlapped *Overlapped) (err error) {
+	var _p0 *byte
+	if len(buf) > 0 {
+		_p0 = &buf[0]
+	}
+	r1, _, e1 := syscall.Syscall6(procWriteFile.Addr(), 5, uintptr(handle), uintptr(unsafe.Pointer(_p0)), uintptr(len(buf)), uintptr(unsafe.Pointer(done)), uintptr(unsafe.Pointer(overlapped)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetOverlappedResult(handle Handle, overlapped *Overlapped, done *uint32, wait bool) (err error) {
+	var _p0 uint32
+	if wait {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r1, _, e1 := syscall.Syscall6(procGetOverlappedResult.Addr(), 4, uintptr(handle), uintptr(unsafe.Pointer(overlapped)), uintptr(unsafe.Pointer(done)), uintptr(_p0), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetFilePointer(handle Handle, lowoffset int32, highoffsetptr *int32, whence uint32) (newlowoffset uint32, err error) {
+	r0, _, e1 := syscall.Syscall6(procSetFilePointer.Addr(), 4, uintptr(handle), uintptr(lowoffset), uintptr(unsafe.Pointer(highoffsetptr)), uintptr(whence), 0, 0)
+	newlowoffset = uint32(r0)
+	if newlowoffset == 0xffffffff {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CloseHandle(handle Handle) (err error) {
+	r1, _, e1 := syscall.Syscall(procCloseHandle.Addr(), 1, uintptr(handle), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetStdHandle(stdhandle uint32) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall(procGetStdHandle.Addr(), 1, uintptr(stdhandle), 0, 0)
+	handle = Handle(r0)
+	if handle == InvalidHandle {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetStdHandle(stdhandle uint32, handle Handle) (err error) {
+	r1, _, e1 := syscall.Syscall(procSetStdHandle.Addr(), 2, uintptr(stdhandle), uintptr(handle), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func findFirstFile1(name *uint16, data *win32finddata1) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall(procFindFirstFileW.Addr(), 2, uintptr(unsafe.Pointer(name)), uintptr(unsafe.Pointer(data)), 0)
+	handle = Handle(r0)
+	if handle == InvalidHandle {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func findNextFile1(handle Handle, data *win32finddata1) (err error) {
+	r1, _, e1 := syscall.Syscall(procFindNextFileW.Addr(), 2, uintptr(handle), uintptr(unsafe.Pointer(data)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func FindClose(handle Handle) (err error) {
+	r1, _, e1 := syscall.Syscall(procFindClose.Addr(), 1, uintptr(handle), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetFileInformationByHandle(handle Handle, data *ByHandleFileInformation) (err error) {
+	r1, _, e1 := syscall.Syscall(procGetFileInformationByHandle.Addr(), 2, uintptr(handle), uintptr(unsafe.Pointer(data)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetFileInformationByHandleEx(handle Handle, class uint32, outBuffer *byte, outBufferLen uint32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procGetFileInformationByHandleEx.Addr(), 4, uintptr(handle), uintptr(class), uintptr(unsafe.Pointer(outBuffer)), uintptr(outBufferLen), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetCurrentDirectory(buflen uint32, buf *uint16) (n uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procGetCurrentDirectoryW.Addr(), 2, uintptr(buflen), uintptr(unsafe.Pointer(buf)), 0)
+	n = uint32(r0)
+	if n == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetCurrentDirectory(path *uint16) (err error) {
+	r1, _, e1 := syscall.Syscall(procSetCurrentDirectoryW.Addr(), 1, uintptr(unsafe.Pointer(path)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CreateDirectory(path *uint16, sa *SecurityAttributes) (err error) {
+	r1, _, e1 := syscall.Syscall(procCreateDirectoryW.Addr(), 2, uintptr(unsafe.Pointer(path)), uintptr(unsafe.Pointer(sa)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func RemoveDirectory(path *uint16) (err error) {
+	r1, _, e1 := syscall.Syscall(procRemoveDirectoryW.Addr(), 1, uintptr(unsafe.Pointer(path)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func DeleteFile(path *uint16) (err error) {
+	r1, _, e1 := syscall.Syscall(procDeleteFileW.Addr(), 1, uintptr(unsafe.Pointer(path)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func MoveFile(from *uint16, to *uint16) (err error) {
+	r1, _, e1 := syscall.Syscall(procMoveFileW.Addr(), 2, uintptr(unsafe.Pointer(from)), uintptr(unsafe.Pointer(to)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func MoveFileEx(from *uint16, to *uint16, flags uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procMoveFileExW.Addr(), 3, uintptr(unsafe.Pointer(from)), uintptr(unsafe.Pointer(to)), uintptr(flags))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func LockFileEx(file Handle, flags uint32, reserved uint32, bytesLow uint32, bytesHigh uint32, overlapped *Overlapped) (err error) {
+	r1, _, e1 := syscall.Syscall6(procLockFileEx.Addr(), 6, uintptr(file), uintptr(flags), uintptr(reserved), uintptr(bytesLow), uintptr(bytesHigh), uintptr(unsafe.Pointer(overlapped)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func UnlockFileEx(file Handle, reserved uint32, bytesLow uint32, bytesHigh uint32, overlapped *Overlapped) (err error) {
+	r1, _, e1 := syscall.Syscall6(procUnlockFileEx.Addr(), 5, uintptr(file), uintptr(reserved), uintptr(bytesLow), uintptr(bytesHigh), uintptr(unsafe.Pointer(overlapped)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetComputerName(buf *uint16, n *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procGetComputerNameW.Addr(), 2, uintptr(unsafe.Pointer(buf)), uintptr(unsafe.Pointer(n)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetComputerNameEx(nametype uint32, buf *uint16, n *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procGetComputerNameExW.Addr(), 3, uintptr(nametype), uintptr(unsafe.Pointer(buf)), uintptr(unsafe.Pointer(n)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetEndOfFile(handle Handle) (err error) {
+	r1, _, e1 := syscall.Syscall(procSetEndOfFile.Addr(), 1, uintptr(handle), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetSystemTimeAsFileTime(time *Filetime) {
+	syscall.Syscall(procGetSystemTimeAsFileTime.Addr(), 1, uintptr(unsafe.Pointer(time)), 0, 0)
+	return
+}
+
+func GetSystemTimePreciseAsFileTime(time *Filetime) {
+	syscall.Syscall(procGetSystemTimePreciseAsFileTime.Addr(), 1, uintptr(unsafe.Pointer(time)), 0, 0)
+	return
+}
+
+func GetTimeZoneInformation(tzi *Timezoneinformation) (rc uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procGetTimeZoneInformation.Addr(), 1, uintptr(unsafe.Pointer(tzi)), 0, 0)
+	rc = uint32(r0)
+	if rc == 0xffffffff {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CreateIoCompletionPort(filehandle Handle, cphandle Handle, key uint32, threadcnt uint32) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall6(procCreateIoCompletionPort.Addr(), 4, uintptr(filehandle), uintptr(cphandle), uintptr(key), uintptr(threadcnt), 0, 0)
+	handle = Handle(r0)
+	if handle == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetQueuedCompletionStatus(cphandle Handle, qty *uint32, key *uint32, overlapped **Overlapped, timeout uint32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procGetQueuedCompletionStatus.Addr(), 5, uintptr(cphandle), uintptr(unsafe.Pointer(qty)), uintptr(unsafe.Pointer(key)), uintptr(unsafe.Pointer(overlapped)), uintptr(timeout), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func PostQueuedCompletionStatus(cphandle Handle, qty uint32, key uint32, overlapped *Overlapped) (err error) {
+	r1, _, e1 := syscall.Syscall6(procPostQueuedCompletionStatus.Addr(), 4, uintptr(cphandle), uintptr(qty), uintptr(key), uintptr(unsafe.Pointer(overlapped)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CancelIo(s Handle) (err error) {
+	r1, _, e1 := syscall.Syscall(procCancelIo.Addr(), 1, uintptr(s), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CancelIoEx(s Handle, o *Overlapped) (err error) {
+	r1, _, e1 := syscall.Syscall(procCancelIoEx.Addr(), 2, uintptr(s), uintptr(unsafe.Pointer(o)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CreateProcess(appName *uint16, commandLine *uint16, procSecurity *SecurityAttributes, threadSecurity *SecurityAttributes, inheritHandles bool, creationFlags uint32, env *uint16, currentDir *uint16, startupInfo *StartupInfo, outProcInfo *ProcessInformation) (err error) {
+	var _p0 uint32
+	if inheritHandles {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r1, _, e1 := syscall.Syscall12(procCreateProcessW.Addr(), 10, uintptr(unsafe.Pointer(appName)), uintptr(unsafe.Pointer(commandLine)), uintptr(unsafe.Pointer(procSecurity)), uintptr(unsafe.Pointer(threadSecurity)), uintptr(_p0), uintptr(creationFlags), uintptr(unsafe.Pointer(env)), uintptr(unsafe.Pointer(currentDir)), uintptr(unsafe.Pointer(startupInfo)), uintptr(unsafe.Pointer(outProcInfo)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func OpenProcess(desiredAccess uint32, inheritHandle bool, processId uint32) (handle Handle, err error) {
+	var _p0 uint32
+	if inheritHandle {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r0, _, e1 := syscall.Syscall(procOpenProcess.Addr(), 3, uintptr(desiredAccess), uintptr(_p0), uintptr(processId))
+	handle = Handle(r0)
+	if handle == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func ShellExecute(hwnd Handle, verb *uint16, file *uint16, args *uint16, cwd *uint16, showCmd int32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procShellExecuteW.Addr(), 6, uintptr(hwnd), uintptr(unsafe.Pointer(verb)), uintptr(unsafe.Pointer(file)), uintptr(unsafe.Pointer(args)), uintptr(unsafe.Pointer(cwd)), uintptr(showCmd))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func shGetKnownFolderPath(id *KNOWNFOLDERID, flags uint32, token Token, path **uint16) (ret error) {
+	r0, _, _ := syscall.Syscall6(procSHGetKnownFolderPath.Addr(), 4, uintptr(unsafe.Pointer(id)), uintptr(flags), uintptr(token), uintptr(unsafe.Pointer(path)), 0, 0)
+	if r0 != 0 {
+		ret = syscall.Errno(r0)
+	}
+	return
+}
+
+func TerminateProcess(handle Handle, exitcode uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procTerminateProcess.Addr(), 2, uintptr(handle), uintptr(exitcode), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetExitCodeProcess(handle Handle, exitcode *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procGetExitCodeProcess.Addr(), 2, uintptr(handle), uintptr(unsafe.Pointer(exitcode)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetStartupInfo(startupInfo *StartupInfo) (err error) {
+	r1, _, e1 := syscall.Syscall(procGetStartupInfoW.Addr(), 1, uintptr(unsafe.Pointer(startupInfo)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetProcessTimes(handle Handle, creationTime *Filetime, exitTime *Filetime, kernelTime *Filetime, userTime *Filetime) (err error) {
+	r1, _, e1 := syscall.Syscall6(procGetProcessTimes.Addr(), 5, uintptr(handle), uintptr(unsafe.Pointer(creationTime)), uintptr(unsafe.Pointer(exitTime)), uintptr(unsafe.Pointer(kernelTime)), uintptr(unsafe.Pointer(userTime)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func DuplicateHandle(hSourceProcessHandle Handle, hSourceHandle Handle, hTargetProcessHandle Handle, lpTargetHandle *Handle, dwDesiredAccess uint32, bInheritHandle bool, dwOptions uint32) (err error) {
+	var _p0 uint32
+	if bInheritHandle {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r1, _, e1 := syscall.Syscall9(procDuplicateHandle.Addr(), 7, uintptr(hSourceProcessHandle), uintptr(hSourceHandle), uintptr(hTargetProcessHandle), uintptr(unsafe.Pointer(lpTargetHandle)), uintptr(dwDesiredAccess), uintptr(_p0), uintptr(dwOptions), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func WaitForSingleObject(handle Handle, waitMilliseconds uint32) (event uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procWaitForSingleObject.Addr(), 2, uintptr(handle), uintptr(waitMilliseconds), 0)
+	event = uint32(r0)
+	if event == 0xffffffff {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func waitForMultipleObjects(count uint32, handles uintptr, waitAll bool, waitMilliseconds uint32) (event uint32, err error) {
+	var _p0 uint32
+	if waitAll {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r0, _, e1 := syscall.Syscall6(procWaitForMultipleObjects.Addr(), 4, uintptr(count), uintptr(handles), uintptr(_p0), uintptr(waitMilliseconds), 0, 0)
+	event = uint32(r0)
+	if event == 0xffffffff {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetTempPath(buflen uint32, buf *uint16) (n uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procGetTempPathW.Addr(), 2, uintptr(buflen), uintptr(unsafe.Pointer(buf)), 0)
+	n = uint32(r0)
+	if n == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CreatePipe(readhandle *Handle, writehandle *Handle, sa *SecurityAttributes, size uint32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procCreatePipe.Addr(), 4, uintptr(unsafe.Pointer(readhandle)), uintptr(unsafe.Pointer(writehandle)), uintptr(unsafe.Pointer(sa)), uintptr(size), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetFileType(filehandle Handle) (n uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procGetFileType.Addr(), 1, uintptr(filehandle), 0, 0)
+	n = uint32(r0)
+	if n == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CryptAcquireContext(provhandle *Handle, container *uint16, provider *uint16, provtype uint32, flags uint32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procCryptAcquireContextW.Addr(), 5, uintptr(unsafe.Pointer(provhandle)), uintptr(unsafe.Pointer(container)), uintptr(unsafe.Pointer(provider)), uintptr(provtype), uintptr(flags), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CryptReleaseContext(provhandle Handle, flags uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procCryptReleaseContext.Addr(), 2, uintptr(provhandle), uintptr(flags), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CryptGenRandom(provhandle Handle, buflen uint32, buf *byte) (err error) {
+	r1, _, e1 := syscall.Syscall(procCryptGenRandom.Addr(), 3, uintptr(provhandle), uintptr(buflen), uintptr(unsafe.Pointer(buf)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetEnvironmentStrings() (envs *uint16, err error) {
+	r0, _, e1 := syscall.Syscall(procGetEnvironmentStringsW.Addr(), 0, 0, 0, 0)
+	envs = (*uint16)(unsafe.Pointer(r0))
+	if envs == nil {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func FreeEnvironmentStrings(envs *uint16) (err error) {
+	r1, _, e1 := syscall.Syscall(procFreeEnvironmentStringsW.Addr(), 1, uintptr(unsafe.Pointer(envs)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetEnvironmentVariable(name *uint16, buffer *uint16, size uint32) (n uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procGetEnvironmentVariableW.Addr(), 3, uintptr(unsafe.Pointer(name)), uintptr(unsafe.Pointer(buffer)), uintptr(size))
+	n = uint32(r0)
+	if n == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetEnvironmentVariable(name *uint16, value *uint16) (err error) {
+	r1, _, e1 := syscall.Syscall(procSetEnvironmentVariableW.Addr(), 2, uintptr(unsafe.Pointer(name)), uintptr(unsafe.Pointer(value)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CreateEnvironmentBlock(block **uint16, token Token, inheritExisting bool) (err error) {
+	var _p0 uint32
+	if inheritExisting {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r1, _, e1 := syscall.Syscall(procCreateEnvironmentBlock.Addr(), 3, uintptr(unsafe.Pointer(block)), uintptr(token), uintptr(_p0))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func DestroyEnvironmentBlock(block *uint16) (err error) {
+	r1, _, e1 := syscall.Syscall(procDestroyEnvironmentBlock.Addr(), 1, uintptr(unsafe.Pointer(block)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func getTickCount64() (ms uint64) {
+	r0, _, _ := syscall.Syscall(procGetTickCount64.Addr(), 0, 0, 0, 0)
+	ms = uint64(r0)
+	return
+}
+
+func SetFileTime(handle Handle, ctime *Filetime, atime *Filetime, wtime *Filetime) (err error) {
+	r1, _, e1 := syscall.Syscall6(procSetFileTime.Addr(), 4, uintptr(handle), uintptr(unsafe.Pointer(ctime)), uintptr(unsafe.Pointer(atime)), uintptr(unsafe.Pointer(wtime)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetFileAttributes(name *uint16) (attrs uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procGetFileAttributesW.Addr(), 1, uintptr(unsafe.Pointer(name)), 0, 0)
+	attrs = uint32(r0)
+	if attrs == INVALID_FILE_ATTRIBUTES {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetFileAttributes(name *uint16, attrs uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procSetFileAttributesW.Addr(), 2, uintptr(unsafe.Pointer(name)), uintptr(attrs), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetFileAttributesEx(name *uint16, level uint32, info *byte) (err error) {
+	r1, _, e1 := syscall.Syscall(procGetFileAttributesExW.Addr(), 3, uintptr(unsafe.Pointer(name)), uintptr(level), uintptr(unsafe.Pointer(info)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetCommandLine() (cmd *uint16) {
+	r0, _, _ := syscall.Syscall(procGetCommandLineW.Addr(), 0, 0, 0, 0)
+	cmd = (*uint16)(unsafe.Pointer(r0))
+	return
+}
+
+func CommandLineToArgv(cmd *uint16, argc *int32) (argv *[8192]*[8192]uint16, err error) {
+	r0, _, e1 := syscall.Syscall(procCommandLineToArgvW.Addr(), 2, uintptr(unsafe.Pointer(cmd)), uintptr(unsafe.Pointer(argc)), 0)
+	argv = (*[8192]*[8192]uint16)(unsafe.Pointer(r0))
+	if argv == nil {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func LocalFree(hmem Handle) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall(procLocalFree.Addr(), 1, uintptr(hmem), 0, 0)
+	handle = Handle(r0)
+	if handle != 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetHandleInformation(handle Handle, mask uint32, flags uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procSetHandleInformation.Addr(), 3, uintptr(handle), uintptr(mask), uintptr(flags))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func FlushFileBuffers(handle Handle) (err error) {
+	r1, _, e1 := syscall.Syscall(procFlushFileBuffers.Addr(), 1, uintptr(handle), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetFullPathName(path *uint16, buflen uint32, buf *uint16, fname **uint16) (n uint32, err error) {
+	r0, _, e1 := syscall.Syscall6(procGetFullPathNameW.Addr(), 4, uintptr(unsafe.Pointer(path)), uintptr(buflen), uintptr(unsafe.Pointer(buf)), uintptr(unsafe.Pointer(fname)), 0, 0)
+	n = uint32(r0)
+	if n == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetLongPathName(path *uint16, buf *uint16, buflen uint32) (n uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procGetLongPathNameW.Addr(), 3, uintptr(unsafe.Pointer(path)), uintptr(unsafe.Pointer(buf)), uintptr(buflen))
+	n = uint32(r0)
+	if n == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetShortPathName(longpath *uint16, shortpath *uint16, buflen uint32) (n uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procGetShortPathNameW.Addr(), 3, uintptr(unsafe.Pointer(longpath)), uintptr(unsafe.Pointer(shortpath)), uintptr(buflen))
+	n = uint32(r0)
+	if n == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CreateFileMapping(fhandle Handle, sa *SecurityAttributes, prot uint32, maxSizeHigh uint32, maxSizeLow uint32, name *uint16) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall6(procCreateFileMappingW.Addr(), 6, uintptr(fhandle), uintptr(unsafe.Pointer(sa)), uintptr(prot), uintptr(maxSizeHigh), uintptr(maxSizeLow), uintptr(unsafe.Pointer(name)))
+	handle = Handle(r0)
+	if handle == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func MapViewOfFile(handle Handle, access uint32, offsetHigh uint32, offsetLow uint32, length uintptr) (addr uintptr, err error) {
+	r0, _, e1 := syscall.Syscall6(procMapViewOfFile.Addr(), 5, uintptr(handle), uintptr(access), uintptr(offsetHigh), uintptr(offsetLow), uintptr(length), 0)
+	addr = uintptr(r0)
+	if addr == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func UnmapViewOfFile(addr uintptr) (err error) {
+	r1, _, e1 := syscall.Syscall(procUnmapViewOfFile.Addr(), 1, uintptr(addr), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func FlushViewOfFile(addr uintptr, length uintptr) (err error) {
+	r1, _, e1 := syscall.Syscall(procFlushViewOfFile.Addr(), 2, uintptr(addr), uintptr(length), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func VirtualLock(addr uintptr, length uintptr) (err error) {
+	r1, _, e1 := syscall.Syscall(procVirtualLock.Addr(), 2, uintptr(addr), uintptr(length), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func VirtualUnlock(addr uintptr, length uintptr) (err error) {
+	r1, _, e1 := syscall.Syscall(procVirtualUnlock.Addr(), 2, uintptr(addr), uintptr(length), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func VirtualAlloc(address uintptr, size uintptr, alloctype uint32, protect uint32) (value uintptr, err error) {
+	r0, _, e1 := syscall.Syscall6(procVirtualAlloc.Addr(), 4, uintptr(address), uintptr(size), uintptr(alloctype), uintptr(protect), 0, 0)
+	value = uintptr(r0)
+	if value == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func VirtualFree(address uintptr, size uintptr, freetype uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procVirtualFree.Addr(), 3, uintptr(address), uintptr(size), uintptr(freetype))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func VirtualProtect(address uintptr, size uintptr, newprotect uint32, oldprotect *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procVirtualProtect.Addr(), 4, uintptr(address), uintptr(size), uintptr(newprotect), uintptr(unsafe.Pointer(oldprotect)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func TransmitFile(s Handle, handle Handle, bytesToWrite uint32, bytsPerSend uint32, overlapped *Overlapped, transmitFileBuf *TransmitFileBuffers, flags uint32) (err error) {
+	r1, _, e1 := syscall.Syscall9(procTransmitFile.Addr(), 7, uintptr(s), uintptr(handle), uintptr(bytesToWrite), uintptr(bytsPerSend), uintptr(unsafe.Pointer(overlapped)), uintptr(unsafe.Pointer(transmitFileBuf)), uintptr(flags), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func ReadDirectoryChanges(handle Handle, buf *byte, buflen uint32, watchSubTree bool, mask uint32, retlen *uint32, overlapped *Overlapped, completionRoutine uintptr) (err error) {
+	var _p0 uint32
+	if watchSubTree {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r1, _, e1 := syscall.Syscall9(procReadDirectoryChangesW.Addr(), 8, uintptr(handle), uintptr(unsafe.Pointer(buf)), uintptr(buflen), uintptr(_p0), uintptr(mask), uintptr(unsafe.Pointer(retlen)), uintptr(unsafe.Pointer(overlapped)), uintptr(completionRoutine), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CertOpenSystemStore(hprov Handle, name *uint16) (store Handle, err error) {
+	r0, _, e1 := syscall.Syscall(procCertOpenSystemStoreW.Addr(), 2, uintptr(hprov), uintptr(unsafe.Pointer(name)), 0)
+	store = Handle(r0)
+	if store == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CertOpenStore(storeProvider uintptr, msgAndCertEncodingType uint32, cryptProv uintptr, flags uint32, para uintptr) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall6(procCertOpenStore.Addr(), 5, uintptr(storeProvider), uintptr(msgAndCertEncodingType), uintptr(cryptProv), uintptr(flags), uintptr(para), 0)
+	handle = Handle(r0)
+	if handle == InvalidHandle {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CertEnumCertificatesInStore(store Handle, prevContext *CertContext) (context *CertContext, err error) {
+	r0, _, e1 := syscall.Syscall(procCertEnumCertificatesInStore.Addr(), 2, uintptr(store), uintptr(unsafe.Pointer(prevContext)), 0)
+	context = (*CertContext)(unsafe.Pointer(r0))
+	if context == nil {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CertAddCertificateContextToStore(store Handle, certContext *CertContext, addDisposition uint32, storeContext **CertContext) (err error) {
+	r1, _, e1 := syscall.Syscall6(procCertAddCertificateContextToStore.Addr(), 4, uintptr(store), uintptr(unsafe.Pointer(certContext)), uintptr(addDisposition), uintptr(unsafe.Pointer(storeContext)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CertCloseStore(store Handle, flags uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procCertCloseStore.Addr(), 2, uintptr(store), uintptr(flags), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CertGetCertificateChain(engine Handle, leaf *CertContext, time *Filetime, additionalStore Handle, para *CertChainPara, flags uint32, reserved uintptr, chainCtx **CertChainContext) (err error) {
+	r1, _, e1 := syscall.Syscall9(procCertGetCertificateChain.Addr(), 8, uintptr(engine), uintptr(unsafe.Pointer(leaf)), uintptr(unsafe.Pointer(time)), uintptr(additionalStore), uintptr(unsafe.Pointer(para)), uintptr(flags), uintptr(reserved), uintptr(unsafe.Pointer(chainCtx)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CertFreeCertificateChain(ctx *CertChainContext) {
+	syscall.Syscall(procCertFreeCertificateChain.Addr(), 1, uintptr(unsafe.Pointer(ctx)), 0, 0)
+	return
+}
+
+func CertCreateCertificateContext(certEncodingType uint32, certEncoded *byte, encodedLen uint32) (context *CertContext, err error) {
+	r0, _, e1 := syscall.Syscall(procCertCreateCertificateContext.Addr(), 3, uintptr(certEncodingType), uintptr(unsafe.Pointer(certEncoded)), uintptr(encodedLen))
+	context = (*CertContext)(unsafe.Pointer(r0))
+	if context == nil {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CertFreeCertificateContext(ctx *CertContext) (err error) {
+	r1, _, e1 := syscall.Syscall(procCertFreeCertificateContext.Addr(), 1, uintptr(unsafe.Pointer(ctx)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CertVerifyCertificateChainPolicy(policyOID uintptr, chain *CertChainContext, para *CertChainPolicyPara, status *CertChainPolicyStatus) (err error) {
+	r1, _, e1 := syscall.Syscall6(procCertVerifyCertificateChainPolicy.Addr(), 4, uintptr(policyOID), uintptr(unsafe.Pointer(chain)), uintptr(unsafe.Pointer(para)), uintptr(unsafe.Pointer(status)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func RegOpenKeyEx(key Handle, subkey *uint16, options uint32, desiredAccess uint32, result *Handle) (regerrno error) {
+	r0, _, _ := syscall.Syscall6(procRegOpenKeyExW.Addr(), 5, uintptr(key), uintptr(unsafe.Pointer(subkey)), uintptr(options), uintptr(desiredAccess), uintptr(unsafe.Pointer(result)), 0)
+	if r0 != 0 {
+		regerrno = syscall.Errno(r0)
+	}
+	return
+}
+
+func RegCloseKey(key Handle) (regerrno error) {
+	r0, _, _ := syscall.Syscall(procRegCloseKey.Addr(), 1, uintptr(key), 0, 0)
+	if r0 != 0 {
+		regerrno = syscall.Errno(r0)
+	}
+	return
+}
+
+func RegQueryInfoKey(key Handle, class *uint16, classLen *uint32, reserved *uint32, subkeysLen *uint32, maxSubkeyLen *uint32, maxClassLen *uint32, valuesLen *uint32, maxValueNameLen *uint32, maxValueLen *uint32, saLen *uint32, lastWriteTime *Filetime) (regerrno error) {
+	r0, _, _ := syscall.Syscall12(procRegQueryInfoKeyW.Addr(), 12, uintptr(key), uintptr(unsafe.Pointer(class)), uintptr(unsafe.Pointer(classLen)), uintptr(unsafe.Pointer(reserved)), uintptr(unsafe.Pointer(subkeysLen)), uintptr(unsafe.Pointer(maxSubkeyLen)), uintptr(unsafe.Pointer(maxClassLen)), uintptr(unsafe.Pointer(valuesLen)), uintptr(unsafe.Pointer(maxValueNameLen)), uintptr(unsafe.Pointer(maxValueLen)), uintptr(unsafe.Pointer(saLen)), uintptr(unsafe.Pointer(lastWriteTime)))
+	if r0 != 0 {
+		regerrno = syscall.Errno(r0)
+	}
+	return
+}
+
+func RegEnumKeyEx(key Handle, index uint32, name *uint16, nameLen *uint32, reserved *uint32, class *uint16, classLen *uint32, lastWriteTime *Filetime) (regerrno error) {
+	r0, _, _ := syscall.Syscall9(procRegEnumKeyExW.Addr(), 8, uintptr(key), uintptr(index), uintptr(unsafe.Pointer(name)), uintptr(unsafe.Pointer(nameLen)), uintptr(unsafe.Pointer(reserved)), uintptr(unsafe.Pointer(class)), uintptr(unsafe.Pointer(classLen)), uintptr(unsafe.Pointer(lastWriteTime)), 0)
+	if r0 != 0 {
+		regerrno = syscall.Errno(r0)
+	}
+	return
+}
+
+func RegQueryValueEx(key Handle, name *uint16, reserved *uint32, valtype *uint32, buf *byte, buflen *uint32) (regerrno error) {
+	r0, _, _ := syscall.Syscall6(procRegQueryValueExW.Addr(), 6, uintptr(key), uintptr(unsafe.Pointer(name)), uintptr(unsafe.Pointer(reserved)), uintptr(unsafe.Pointer(valtype)), uintptr(unsafe.Pointer(buf)), uintptr(unsafe.Pointer(buflen)))
+	if r0 != 0 {
+		regerrno = syscall.Errno(r0)
+	}
+	return
+}
+
+func GetCurrentProcessId() (pid uint32) {
+	r0, _, _ := syscall.Syscall(procGetCurrentProcessId.Addr(), 0, 0, 0, 0)
+	pid = uint32(r0)
+	return
+}
+
+func GetConsoleMode(console Handle, mode *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procGetConsoleMode.Addr(), 2, uintptr(console), uintptr(unsafe.Pointer(mode)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetConsoleMode(console Handle, mode uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procSetConsoleMode.Addr(), 2, uintptr(console), uintptr(mode), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetConsoleScreenBufferInfo(console Handle, info *ConsoleScreenBufferInfo) (err error) {
+	r1, _, e1 := syscall.Syscall(procGetConsoleScreenBufferInfo.Addr(), 2, uintptr(console), uintptr(unsafe.Pointer(info)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func WriteConsole(console Handle, buf *uint16, towrite uint32, written *uint32, reserved *byte) (err error) {
+	r1, _, e1 := syscall.Syscall6(procWriteConsoleW.Addr(), 5, uintptr(console), uintptr(unsafe.Pointer(buf)), uintptr(towrite), uintptr(unsafe.Pointer(written)), uintptr(unsafe.Pointer(reserved)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func ReadConsole(console Handle, buf *uint16, toread uint32, read *uint32, inputControl *byte) (err error) {
+	r1, _, e1 := syscall.Syscall6(procReadConsoleW.Addr(), 5, uintptr(console), uintptr(unsafe.Pointer(buf)), uintptr(toread), uintptr(unsafe.Pointer(read)), uintptr(unsafe.Pointer(inputControl)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CreateToolhelp32Snapshot(flags uint32, processId uint32) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall(procCreateToolhelp32Snapshot.Addr(), 2, uintptr(flags), uintptr(processId), 0)
+	handle = Handle(r0)
+	if handle == InvalidHandle {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func Process32First(snapshot Handle, procEntry *ProcessEntry32) (err error) {
+	r1, _, e1 := syscall.Syscall(procProcess32FirstW.Addr(), 2, uintptr(snapshot), uintptr(unsafe.Pointer(procEntry)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func Process32Next(snapshot Handle, procEntry *ProcessEntry32) (err error) {
+	r1, _, e1 := syscall.Syscall(procProcess32NextW.Addr(), 2, uintptr(snapshot), uintptr(unsafe.Pointer(procEntry)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func Thread32First(snapshot Handle, threadEntry *ThreadEntry32) (err error) {
+	r1, _, e1 := syscall.Syscall(procThread32First.Addr(), 2, uintptr(snapshot), uintptr(unsafe.Pointer(threadEntry)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func Thread32Next(snapshot Handle, threadEntry *ThreadEntry32) (err error) {
+	r1, _, e1 := syscall.Syscall(procThread32Next.Addr(), 2, uintptr(snapshot), uintptr(unsafe.Pointer(threadEntry)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func DeviceIoControl(handle Handle, ioControlCode uint32, inBuffer *byte, inBufferSize uint32, outBuffer *byte, outBufferSize uint32, bytesReturned *uint32, overlapped *Overlapped) (err error) {
+	r1, _, e1 := syscall.Syscall9(procDeviceIoControl.Addr(), 8, uintptr(handle), uintptr(ioControlCode), uintptr(unsafe.Pointer(inBuffer)), uintptr(inBufferSize), uintptr(unsafe.Pointer(outBuffer)), uintptr(outBufferSize), uintptr(unsafe.Pointer(bytesReturned)), uintptr(unsafe.Pointer(overlapped)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CreateSymbolicLink(symlinkfilename *uint16, targetfilename *uint16, flags uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procCreateSymbolicLinkW.Addr(), 3, uintptr(unsafe.Pointer(symlinkfilename)), uintptr(unsafe.Pointer(targetfilename)), uintptr(flags))
+	if r1&0xff == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CreateHardLink(filename *uint16, existingfilename *uint16, reserved uintptr) (err error) {
+	r1, _, e1 := syscall.Syscall(procCreateHardLinkW.Addr(), 3, uintptr(unsafe.Pointer(filename)), uintptr(unsafe.Pointer(existingfilename)), uintptr(reserved))
+	if r1&0xff == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetCurrentThreadId() (id uint32) {
+	r0, _, _ := syscall.Syscall(procGetCurrentThreadId.Addr(), 0, 0, 0, 0)
+	id = uint32(r0)
+	return
+}
+
+func CreateEvent(eventAttrs *SecurityAttributes, manualReset uint32, initialState uint32, name *uint16) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall6(procCreateEventW.Addr(), 4, uintptr(unsafe.Pointer(eventAttrs)), uintptr(manualReset), uintptr(initialState), uintptr(unsafe.Pointer(name)), 0, 0)
+	handle = Handle(r0)
+	if handle == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CreateEventEx(eventAttrs *SecurityAttributes, name *uint16, flags uint32, desiredAccess uint32) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall6(procCreateEventExW.Addr(), 4, uintptr(unsafe.Pointer(eventAttrs)), uintptr(unsafe.Pointer(name)), uintptr(flags), uintptr(desiredAccess), 0, 0)
+	handle = Handle(r0)
+	if handle == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func OpenEvent(desiredAccess uint32, inheritHandle bool, name *uint16) (handle Handle, err error) {
+	var _p0 uint32
+	if inheritHandle {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r0, _, e1 := syscall.Syscall(procOpenEventW.Addr(), 3, uintptr(desiredAccess), uintptr(_p0), uintptr(unsafe.Pointer(name)))
+	handle = Handle(r0)
+	if handle == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetEvent(event Handle) (err error) {
+	r1, _, e1 := syscall.Syscall(procSetEvent.Addr(), 1, uintptr(event), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func ResetEvent(event Handle) (err error) {
+	r1, _, e1 := syscall.Syscall(procResetEvent.Addr(), 1, uintptr(event), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func PulseEvent(event Handle) (err error) {
+	r1, _, e1 := syscall.Syscall(procPulseEvent.Addr(), 1, uintptr(event), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CreateMutex(mutexAttrs *SecurityAttributes, initialOwner bool, name *uint16) (handle Handle, err error) {
+	var _p0 uint32
+	if initialOwner {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r0, _, e1 := syscall.Syscall(procCreateMutexW.Addr(), 3, uintptr(unsafe.Pointer(mutexAttrs)), uintptr(_p0), uintptr(unsafe.Pointer(name)))
+	handle = Handle(r0)
+	if handle == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func CreateMutexEx(mutexAttrs *SecurityAttributes, name *uint16, flags uint32, desiredAccess uint32) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall6(procCreateMutexExW.Addr(), 4, uintptr(unsafe.Pointer(mutexAttrs)), uintptr(unsafe.Pointer(name)), uintptr(flags), uintptr(desiredAccess), 0, 0)
+	handle = Handle(r0)
+	if handle == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func OpenMutex(desiredAccess uint32, inheritHandle bool, name *uint16) (handle Handle, err error) {
+	var _p0 uint32
+	if inheritHandle {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r0, _, e1 := syscall.Syscall(procOpenMutexW.Addr(), 3, uintptr(desiredAccess), uintptr(_p0), uintptr(unsafe.Pointer(name)))
+	handle = Handle(r0)
+	if handle == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func ReleaseMutex(mutex Handle) (err error) {
+	r1, _, e1 := syscall.Syscall(procReleaseMutex.Addr(), 1, uintptr(mutex), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SleepEx(milliseconds uint32, alertable bool) (ret uint32) {
+	var _p0 uint32
+	if alertable {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r0, _, _ := syscall.Syscall(procSleepEx.Addr(), 2, uintptr(milliseconds), uintptr(_p0), 0)
+	ret = uint32(r0)
+	return
+}
+
+func CreateJobObject(jobAttr *SecurityAttributes, name *uint16) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall(procCreateJobObjectW.Addr(), 2, uintptr(unsafe.Pointer(jobAttr)), uintptr(unsafe.Pointer(name)), 0)
+	handle = Handle(r0)
+	if handle == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func AssignProcessToJobObject(job Handle, process Handle) (err error) {
+	r1, _, e1 := syscall.Syscall(procAssignProcessToJobObject.Addr(), 2, uintptr(job), uintptr(process), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func TerminateJobObject(job Handle, exitCode uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procTerminateJobObject.Addr(), 2, uintptr(job), uintptr(exitCode), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetErrorMode(mode uint32) (ret uint32) {
+	r0, _, _ := syscall.Syscall(procSetErrorMode.Addr(), 1, uintptr(mode), 0, 0)
+	ret = uint32(r0)
+	return
+}
+
+func ResumeThread(thread Handle) (ret uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procResumeThread.Addr(), 1, uintptr(thread), 0, 0)
+	ret = uint32(r0)
+	if ret == 0xffffffff {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetPriorityClass(process Handle, priorityClass uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procSetPriorityClass.Addr(), 2, uintptr(process), uintptr(priorityClass), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetPriorityClass(process Handle) (ret uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procGetPriorityClass.Addr(), 1, uintptr(process), 0, 0)
+	ret = uint32(r0)
+	if ret == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetInformationJobObject(job Handle, JobObjectInformationClass uint32, JobObjectInformation uintptr, JobObjectInformationLength uint32) (ret int, err error) {
+	r0, _, e1 := syscall.Syscall6(procSetInformationJobObject.Addr(), 4, uintptr(job), uintptr(JobObjectInformationClass), uintptr(JobObjectInformation), uintptr(JobObjectInformationLength), 0, 0)
+	ret = int(r0)
+	if ret == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GenerateConsoleCtrlEvent(ctrlEvent uint32, processGroupID uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procGenerateConsoleCtrlEvent.Addr(), 2, uintptr(ctrlEvent), uintptr(processGroupID), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetProcessId(process Handle) (id uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procGetProcessId.Addr(), 1, uintptr(process), 0, 0)
+	id = uint32(r0)
+	if id == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func OpenThread(desiredAccess uint32, inheritHandle bool, threadId uint32) (handle Handle, err error) {
+	var _p0 uint32
+	if inheritHandle {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r0, _, e1 := syscall.Syscall(procOpenThread.Addr(), 3, uintptr(desiredAccess), uintptr(_p0), uintptr(threadId))
+	handle = Handle(r0)
+	if handle == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetProcessPriorityBoost(process Handle, disable bool) (err error) {
+	var _p0 uint32
+	if disable {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r1, _, e1 := syscall.Syscall(procSetProcessPriorityBoost.Addr(), 2, uintptr(process), uintptr(_p0), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func DefineDosDevice(flags uint32, deviceName *uint16, targetPath *uint16) (err error) {
+	r1, _, e1 := syscall.Syscall(procDefineDosDeviceW.Addr(), 3, uintptr(flags), uintptr(unsafe.Pointer(deviceName)), uintptr(unsafe.Pointer(targetPath)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func DeleteVolumeMountPoint(volumeMountPoint *uint16) (err error) {
+	r1, _, e1 := syscall.Syscall(procDeleteVolumeMountPointW.Addr(), 1, uintptr(unsafe.Pointer(volumeMountPoint)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func FindFirstVolume(volumeName *uint16, bufferLength uint32) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall(procFindFirstVolumeW.Addr(), 2, uintptr(unsafe.Pointer(volumeName)), uintptr(bufferLength), 0)
+	handle = Handle(r0)
+	if handle == InvalidHandle {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func FindFirstVolumeMountPoint(rootPathName *uint16, volumeMountPoint *uint16, bufferLength uint32) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall(procFindFirstVolumeMountPointW.Addr(), 3, uintptr(unsafe.Pointer(rootPathName)), uintptr(unsafe.Pointer(volumeMountPoint)), uintptr(bufferLength))
+	handle = Handle(r0)
+	if handle == InvalidHandle {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func FindNextVolume(findVolume Handle, volumeName *uint16, bufferLength uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procFindNextVolumeW.Addr(), 3, uintptr(findVolume), uintptr(unsafe.Pointer(volumeName)), uintptr(bufferLength))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func FindNextVolumeMountPoint(findVolumeMountPoint Handle, volumeMountPoint *uint16, bufferLength uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procFindNextVolumeMountPointW.Addr(), 3, uintptr(findVolumeMountPoint), uintptr(unsafe.Pointer(volumeMountPoint)), uintptr(bufferLength))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func FindVolumeClose(findVolume Handle) (err error) {
+	r1, _, e1 := syscall.Syscall(procFindVolumeClose.Addr(), 1, uintptr(findVolume), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func FindVolumeMountPointClose(findVolumeMountPoint Handle) (err error) {
+	r1, _, e1 := syscall.Syscall(procFindVolumeMountPointClose.Addr(), 1, uintptr(findVolumeMountPoint), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetDriveType(rootPathName *uint16) (driveType uint32) {
+	r0, _, _ := syscall.Syscall(procGetDriveTypeW.Addr(), 1, uintptr(unsafe.Pointer(rootPathName)), 0, 0)
+	driveType = uint32(r0)
+	return
+}
+
+func GetLogicalDrives() (drivesBitMask uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procGetLogicalDrives.Addr(), 0, 0, 0, 0)
+	drivesBitMask = uint32(r0)
+	if drivesBitMask == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetLogicalDriveStrings(bufferLength uint32, buffer *uint16) (n uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procGetLogicalDriveStringsW.Addr(), 2, uintptr(bufferLength), uintptr(unsafe.Pointer(buffer)), 0)
+	n = uint32(r0)
+	if n == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetVolumeInformation(rootPathName *uint16, volumeNameBuffer *uint16, volumeNameSize uint32, volumeNameSerialNumber *uint32, maximumComponentLength *uint32, fileSystemFlags *uint32, fileSystemNameBuffer *uint16, fileSystemNameSize uint32) (err error) {
+	r1, _, e1 := syscall.Syscall9(procGetVolumeInformationW.Addr(), 8, uintptr(unsafe.Pointer(rootPathName)), uintptr(unsafe.Pointer(volumeNameBuffer)), uintptr(volumeNameSize), uintptr(unsafe.Pointer(volumeNameSerialNumber)), uintptr(unsafe.Pointer(maximumComponentLength)), uintptr(unsafe.Pointer(fileSystemFlags)), uintptr(unsafe.Pointer(fileSystemNameBuffer)), uintptr(fileSystemNameSize), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetVolumeInformationByHandle(file Handle, volumeNameBuffer *uint16, volumeNameSize uint32, volumeNameSerialNumber *uint32, maximumComponentLength *uint32, fileSystemFlags *uint32, fileSystemNameBuffer *uint16, fileSystemNameSize uint32) (err error) {
+	r1, _, e1 := syscall.Syscall9(procGetVolumeInformationByHandleW.Addr(), 8, uintptr(file), uintptr(unsafe.Pointer(volumeNameBuffer)), uintptr(volumeNameSize), uintptr(unsafe.Pointer(volumeNameSerialNumber)), uintptr(unsafe.Pointer(maximumComponentLength)), uintptr(unsafe.Pointer(fileSystemFlags)), uintptr(unsafe.Pointer(fileSystemNameBuffer)), uintptr(fileSystemNameSize), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetVolumeNameForVolumeMountPoint(volumeMountPoint *uint16, volumeName *uint16, bufferlength uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procGetVolumeNameForVolumeMountPointW.Addr(), 3, uintptr(unsafe.Pointer(volumeMountPoint)), uintptr(unsafe.Pointer(volumeName)), uintptr(bufferlength))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetVolumePathName(fileName *uint16, volumePathName *uint16, bufferLength uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procGetVolumePathNameW.Addr(), 3, uintptr(unsafe.Pointer(fileName)), uintptr(unsafe.Pointer(volumePathName)), uintptr(bufferLength))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetVolumePathNamesForVolumeName(volumeName *uint16, volumePathNames *uint16, bufferLength uint32, returnLength *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procGetVolumePathNamesForVolumeNameW.Addr(), 4, uintptr(unsafe.Pointer(volumeName)), uintptr(unsafe.Pointer(volumePathNames)), uintptr(bufferLength), uintptr(unsafe.Pointer(returnLength)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func QueryDosDevice(deviceName *uint16, targetPath *uint16, max uint32) (n uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procQueryDosDeviceW.Addr(), 3, uintptr(unsafe.Pointer(deviceName)), uintptr(unsafe.Pointer(targetPath)), uintptr(max))
+	n = uint32(r0)
+	if n == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetVolumeLabel(rootPathName *uint16, volumeName *uint16) (err error) {
+	r1, _, e1 := syscall.Syscall(procSetVolumeLabelW.Addr(), 2, uintptr(unsafe.Pointer(rootPathName)), uintptr(unsafe.Pointer(volumeName)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetVolumeMountPoint(volumeMountPoint *uint16, volumeName *uint16) (err error) {
+	r1, _, e1 := syscall.Syscall(procSetVolumeMountPointW.Addr(), 2, uintptr(unsafe.Pointer(volumeMountPoint)), uintptr(unsafe.Pointer(volumeName)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func MessageBox(hwnd Handle, text *uint16, caption *uint16, boxtype uint32) (ret int32, err error) {
+	r0, _, e1 := syscall.Syscall6(procMessageBoxW.Addr(), 4, uintptr(hwnd), uintptr(unsafe.Pointer(text)), uintptr(unsafe.Pointer(caption)), uintptr(boxtype), 0, 0)
+	ret = int32(r0)
+	if ret == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func ExitWindowsEx(flags uint32, reason uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procExitWindowsEx.Addr(), 2, uintptr(flags), uintptr(reason), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func InitiateSystemShutdownEx(machineName *uint16, message *uint16, timeout uint32, forceAppsClosed bool, rebootAfterShutdown bool, reason uint32) (err error) {
+	var _p0 uint32
+	if forceAppsClosed {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	var _p1 uint32
+	if rebootAfterShutdown {
+		_p1 = 1
+	} else {
+		_p1 = 0
+	}
+	r1, _, e1 := syscall.Syscall6(procInitiateSystemShutdownExW.Addr(), 6, uintptr(unsafe.Pointer(machineName)), uintptr(unsafe.Pointer(message)), uintptr(timeout), uintptr(_p0), uintptr(_p1), uintptr(reason))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetProcessShutdownParameters(level uint32, flags uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procSetProcessShutdownParameters.Addr(), 2, uintptr(level), uintptr(flags), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetProcessShutdownParameters(level *uint32, flags *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procGetProcessShutdownParameters.Addr(), 2, uintptr(unsafe.Pointer(level)), uintptr(unsafe.Pointer(flags)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func clsidFromString(lpsz *uint16, pclsid *GUID) (ret error) {
+	r0, _, _ := syscall.Syscall(procCLSIDFromString.Addr(), 2, uintptr(unsafe.Pointer(lpsz)), uintptr(unsafe.Pointer(pclsid)), 0)
+	if r0 != 0 {
+		ret = syscall.Errno(r0)
+	}
+	return
+}
+
+func stringFromGUID2(rguid *GUID, lpsz *uint16, cchMax int32) (chars int32) {
+	r0, _, _ := syscall.Syscall(procStringFromGUID2.Addr(), 3, uintptr(unsafe.Pointer(rguid)), uintptr(unsafe.Pointer(lpsz)), uintptr(cchMax))
+	chars = int32(r0)
+	return
+}
+
+func coCreateGuid(pguid *GUID) (ret error) {
+	r0, _, _ := syscall.Syscall(procCoCreateGuid.Addr(), 1, uintptr(unsafe.Pointer(pguid)), 0, 0)
+	if r0 != 0 {
+		ret = syscall.Errno(r0)
+	}
+	return
+}
+
+func CoTaskMemFree(address unsafe.Pointer) {
+	syscall.Syscall(procCoTaskMemFree.Addr(), 1, uintptr(address), 0, 0)
+	return
+}
+
+func rtlGetVersion(info *OsVersionInfoEx) (ret error) {
+	r0, _, _ := syscall.Syscall(procRtlGetVersion.Addr(), 1, uintptr(unsafe.Pointer(info)), 0, 0)
+	if r0 != 0 {
+		ret = syscall.Errno(r0)
+	}
+	return
+}
+
+func rtlGetNtVersionNumbers(majorVersion *uint32, minorVersion *uint32, buildNumber *uint32) {
+	syscall.Syscall(procRtlGetNtVersionNumbers.Addr(), 3, uintptr(unsafe.Pointer(majorVersion)), uintptr(unsafe.Pointer(minorVersion)), uintptr(unsafe.Pointer(buildNumber)))
+	return
+}
+
+func WSAStartup(verreq uint32, data *WSAData) (sockerr error) {
+	r0, _, _ := syscall.Syscall(procWSAStartup.Addr(), 2, uintptr(verreq), uintptr(unsafe.Pointer(data)), 0)
+	if r0 != 0 {
+		sockerr = syscall.Errno(r0)
+	}
+	return
+}
+
+func WSACleanup() (err error) {
+	r1, _, e1 := syscall.Syscall(procWSACleanup.Addr(), 0, 0, 0, 0)
+	if r1 == socket_error {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func WSAIoctl(s Handle, iocc uint32, inbuf *byte, cbif uint32, outbuf *byte, cbob uint32, cbbr *uint32, overlapped *Overlapped, completionRoutine uintptr) (err error) {
+	r1, _, e1 := syscall.Syscall9(procWSAIoctl.Addr(), 9, uintptr(s), uintptr(iocc), uintptr(unsafe.Pointer(inbuf)), uintptr(cbif), uintptr(unsafe.Pointer(outbuf)), uintptr(cbob), uintptr(unsafe.Pointer(cbbr)), uintptr(unsafe.Pointer(overlapped)), uintptr(completionRoutine))
+	if r1 == socket_error {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func socket(af int32, typ int32, protocol int32) (handle Handle, err error) {
+	r0, _, e1 := syscall.Syscall(procsocket.Addr(), 3, uintptr(af), uintptr(typ), uintptr(protocol))
+	handle = Handle(r0)
+	if handle == InvalidHandle {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func Setsockopt(s Handle, level int32, optname int32, optval *byte, optlen int32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procsetsockopt.Addr(), 5, uintptr(s), uintptr(level), uintptr(optname), uintptr(unsafe.Pointer(optval)), uintptr(optlen), 0)
+	if r1 == socket_error {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func Getsockopt(s Handle, level int32, optname int32, optval *byte, optlen *int32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procgetsockopt.Addr(), 5, uintptr(s), uintptr(level), uintptr(optname), uintptr(unsafe.Pointer(optval)), uintptr(unsafe.Pointer(optlen)), 0)
+	if r1 == socket_error {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func bind(s Handle, name unsafe.Pointer, namelen int32) (err error) {
+	r1, _, e1 := syscall.Syscall(procbind.Addr(), 3, uintptr(s), uintptr(name), uintptr(namelen))
+	if r1 == socket_error {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func connect(s Handle, name unsafe.Pointer, namelen int32) (err error) {
+	r1, _, e1 := syscall.Syscall(procconnect.Addr(), 3, uintptr(s), uintptr(name), uintptr(namelen))
+	if r1 == socket_error {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func getsockname(s Handle, rsa *RawSockaddrAny, addrlen *int32) (err error) {
+	r1, _, e1 := syscall.Syscall(procgetsockname.Addr(), 3, uintptr(s), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
+	if r1 == socket_error {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func getpeername(s Handle, rsa *RawSockaddrAny, addrlen *int32) (err error) {
+	r1, _, e1 := syscall.Syscall(procgetpeername.Addr(), 3, uintptr(s), uintptr(unsafe.Pointer(rsa)), uintptr(unsafe.Pointer(addrlen)))
+	if r1 == socket_error {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func listen(s Handle, backlog int32) (err error) {
+	r1, _, e1 := syscall.Syscall(proclisten.Addr(), 2, uintptr(s), uintptr(backlog), 0)
+	if r1 == socket_error {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func shutdown(s Handle, how int32) (err error) {
+	r1, _, e1 := syscall.Syscall(procshutdown.Addr(), 2, uintptr(s), uintptr(how), 0)
+	if r1 == socket_error {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func Closesocket(s Handle) (err error) {
+	r1, _, e1 := syscall.Syscall(procclosesocket.Addr(), 1, uintptr(s), 0, 0)
+	if r1 == socket_error {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func AcceptEx(ls Handle, as Handle, buf *byte, rxdatalen uint32, laddrlen uint32, raddrlen uint32, recvd *uint32, overlapped *Overlapped) (err error) {
+	r1, _, e1 := syscall.Syscall9(procAcceptEx.Addr(), 8, uintptr(ls), uintptr(as), uintptr(unsafe.Pointer(buf)), uintptr(rxdatalen), uintptr(laddrlen), uintptr(raddrlen), uintptr(unsafe.Pointer(recvd)), uintptr(unsafe.Pointer(overlapped)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetAcceptExSockaddrs(buf *byte, rxdatalen uint32, laddrlen uint32, raddrlen uint32, lrsa **RawSockaddrAny, lrsalen *int32, rrsa **RawSockaddrAny, rrsalen *int32) {
+	syscall.Syscall9(procGetAcceptExSockaddrs.Addr(), 8, uintptr(unsafe.Pointer(buf)), uintptr(rxdatalen), uintptr(laddrlen), uintptr(raddrlen), uintptr(unsafe.Pointer(lrsa)), uintptr(unsafe.Pointer(lrsalen)), uintptr(unsafe.Pointer(rrsa)), uintptr(unsafe.Pointer(rrsalen)), 0)
+	return
+}
+
+func WSARecv(s Handle, bufs *WSABuf, bufcnt uint32, recvd *uint32, flags *uint32, overlapped *Overlapped, croutine *byte) (err error) {
+	r1, _, e1 := syscall.Syscall9(procWSARecv.Addr(), 7, uintptr(s), uintptr(unsafe.Pointer(bufs)), uintptr(bufcnt), uintptr(unsafe.Pointer(recvd)), uintptr(unsafe.Pointer(flags)), uintptr(unsafe.Pointer(overlapped)), uintptr(unsafe.Pointer(croutine)), 0, 0)
+	if r1 == socket_error {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func WSASend(s Handle, bufs *WSABuf, bufcnt uint32, sent *uint32, flags uint32, overlapped *Overlapped, croutine *byte) (err error) {
+	r1, _, e1 := syscall.Syscall9(procWSASend.Addr(), 7, uintptr(s), uintptr(unsafe.Pointer(bufs)), uintptr(bufcnt), uintptr(unsafe.Pointer(sent)), uintptr(flags), uintptr(unsafe.Pointer(overlapped)), uintptr(unsafe.Pointer(croutine)), 0, 0)
+	if r1 == socket_error {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func WSARecvFrom(s Handle, bufs *WSABuf, bufcnt uint32, recvd *uint32, flags *uint32, from *RawSockaddrAny, fromlen *int32, overlapped *Overlapped, croutine *byte) (err error) {
+	r1, _, e1 := syscall.Syscall9(procWSARecvFrom.Addr(), 9, uintptr(s), uintptr(unsafe.Pointer(bufs)), uintptr(bufcnt), uintptr(unsafe.Pointer(recvd)), uintptr(unsafe.Pointer(flags)), uintptr(unsafe.Pointer(from)), uintptr(unsafe.Pointer(fromlen)), uintptr(unsafe.Pointer(overlapped)), uintptr(unsafe.Pointer(croutine)))
+	if r1 == socket_error {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func WSASendTo(s Handle, bufs *WSABuf, bufcnt uint32, sent *uint32, flags uint32, to *RawSockaddrAny, tolen int32, overlapped *Overlapped, croutine *byte) (err error) {
+	r1, _, e1 := syscall.Syscall9(procWSASendTo.Addr(), 9, uintptr(s), uintptr(unsafe.Pointer(bufs)), uintptr(bufcnt), uintptr(unsafe.Pointer(sent)), uintptr(flags), uintptr(unsafe.Pointer(to)), uintptr(tolen), uintptr(unsafe.Pointer(overlapped)), uintptr(unsafe.Pointer(croutine)))
+	if r1 == socket_error {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetHostByName(name string) (h *Hostent, err error) {
+	var _p0 *byte
+	_p0, err = syscall.BytePtrFromString(name)
+	if err != nil {
+		return
+	}
+	return _GetHostByName(_p0)
+}
+
+func _GetHostByName(name *byte) (h *Hostent, err error) {
+	r0, _, e1 := syscall.Syscall(procgethostbyname.Addr(), 1, uintptr(unsafe.Pointer(name)), 0, 0)
+	h = (*Hostent)(unsafe.Pointer(r0))
+	if h == nil {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetServByName(name string, proto string) (s *Servent, err error) {
+	var _p0 *byte
+	_p0, err = syscall.BytePtrFromString(name)
+	if err != nil {
+		return
+	}
+	var _p1 *byte
+	_p1, err = syscall.BytePtrFromString(proto)
+	if err != nil {
+		return
+	}
+	return _GetServByName(_p0, _p1)
+}
+
+func _GetServByName(name *byte, proto *byte) (s *Servent, err error) {
+	r0, _, e1 := syscall.Syscall(procgetservbyname.Addr(), 2, uintptr(unsafe.Pointer(name)), uintptr(unsafe.Pointer(proto)), 0)
+	s = (*Servent)(unsafe.Pointer(r0))
+	if s == nil {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func Ntohs(netshort uint16) (u uint16) {
+	r0, _, _ := syscall.Syscall(procntohs.Addr(), 1, uintptr(netshort), 0, 0)
+	u = uint16(r0)
+	return
+}
+
+func GetProtoByName(name string) (p *Protoent, err error) {
+	var _p0 *byte
+	_p0, err = syscall.BytePtrFromString(name)
+	if err != nil {
+		return
+	}
+	return _GetProtoByName(_p0)
+}
+
+func _GetProtoByName(name *byte) (p *Protoent, err error) {
+	r0, _, e1 := syscall.Syscall(procgetprotobyname.Addr(), 1, uintptr(unsafe.Pointer(name)), 0, 0)
+	p = (*Protoent)(unsafe.Pointer(r0))
+	if p == nil {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func DnsQuery(name string, qtype uint16, options uint32, extra *byte, qrs **DNSRecord, pr *byte) (status error) {
+	var _p0 *uint16
+	_p0, status = syscall.UTF16PtrFromString(name)
+	if status != nil {
+		return
+	}
+	return _DnsQuery(_p0, qtype, options, extra, qrs, pr)
+}
+
+func _DnsQuery(name *uint16, qtype uint16, options uint32, extra *byte, qrs **DNSRecord, pr *byte) (status error) {
+	r0, _, _ := syscall.Syscall6(procDnsQuery_W.Addr(), 6, uintptr(unsafe.Pointer(name)), uintptr(qtype), uintptr(options), uintptr(unsafe.Pointer(extra)), uintptr(unsafe.Pointer(qrs)), uintptr(unsafe.Pointer(pr)))
+	if r0 != 0 {
+		status = syscall.Errno(r0)
+	}
+	return
+}
+
+func DnsRecordListFree(rl *DNSRecord, freetype uint32) {
+	syscall.Syscall(procDnsRecordListFree.Addr(), 2, uintptr(unsafe.Pointer(rl)), uintptr(freetype), 0)
+	return
+}
+
+func DnsNameCompare(name1 *uint16, name2 *uint16) (same bool) {
+	r0, _, _ := syscall.Syscall(procDnsNameCompare_W.Addr(), 2, uintptr(unsafe.Pointer(name1)), uintptr(unsafe.Pointer(name2)), 0)
+	same = r0 != 0
+	return
+}
+
+func GetAddrInfoW(nodename *uint16, servicename *uint16, hints *AddrinfoW, result **AddrinfoW) (sockerr error) {
+	r0, _, _ := syscall.Syscall6(procGetAddrInfoW.Addr(), 4, uintptr(unsafe.Pointer(nodename)), uintptr(unsafe.Pointer(servicename)), uintptr(unsafe.Pointer(hints)), uintptr(unsafe.Pointer(result)), 0, 0)
+	if r0 != 0 {
+		sockerr = syscall.Errno(r0)
+	}
+	return
+}
+
+func FreeAddrInfoW(addrinfo *AddrinfoW) {
+	syscall.Syscall(procFreeAddrInfoW.Addr(), 1, uintptr(unsafe.Pointer(addrinfo)), 0, 0)
+	return
+}
+
+func GetIfEntry(pIfRow *MibIfRow) (errcode error) {
+	r0, _, _ := syscall.Syscall(procGetIfEntry.Addr(), 1, uintptr(unsafe.Pointer(pIfRow)), 0, 0)
+	if r0 != 0 {
+		errcode = syscall.Errno(r0)
+	}
+	return
+}
+
+func GetAdaptersInfo(ai *IpAdapterInfo, ol *uint32) (errcode error) {
+	r0, _, _ := syscall.Syscall(procGetAdaptersInfo.Addr(), 2, uintptr(unsafe.Pointer(ai)), uintptr(unsafe.Pointer(ol)), 0)
+	if r0 != 0 {
+		errcode = syscall.Errno(r0)
+	}
+	return
+}
+
+func SetFileCompletionNotificationModes(handle Handle, flags uint8) (err error) {
+	r1, _, e1 := syscall.Syscall(procSetFileCompletionNotificationModes.Addr(), 2, uintptr(handle), uintptr(flags), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func WSAEnumProtocols(protocols *int32, protocolBuffer *WSAProtocolInfo, bufferLength *uint32) (n int32, err error) {
+	r0, _, e1 := syscall.Syscall(procWSAEnumProtocolsW.Addr(), 3, uintptr(unsafe.Pointer(protocols)), uintptr(unsafe.Pointer(protocolBuffer)), uintptr(unsafe.Pointer(bufferLength)))
+	n = int32(r0)
+	if n == -1 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetAdaptersAddresses(family uint32, flags uint32, reserved uintptr, adapterAddresses *IpAdapterAddresses, sizePointer *uint32) (errcode error) {
+	r0, _, _ := syscall.Syscall6(procGetAdaptersAddresses.Addr(), 5, uintptr(family), uintptr(flags), uintptr(reserved), uintptr(unsafe.Pointer(adapterAddresses)), uintptr(unsafe.Pointer(sizePointer)), 0)
+	if r0 != 0 {
+		errcode = syscall.Errno(r0)
+	}
+	return
+}
+
+func GetACP() (acp uint32) {
+	r0, _, _ := syscall.Syscall(procGetACP.Addr(), 0, 0, 0, 0)
+	acp = uint32(r0)
+	return
+}
+
+func MultiByteToWideChar(codePage uint32, dwFlags uint32, str *byte, nstr int32, wchar *uint16, nwchar int32) (nwrite int32, err error) {
+	r0, _, e1 := syscall.Syscall6(procMultiByteToWideChar.Addr(), 6, uintptr(codePage), uintptr(dwFlags), uintptr(unsafe.Pointer(str)), uintptr(nstr), uintptr(unsafe.Pointer(wchar)), uintptr(nwchar))
+	nwrite = int32(r0)
+	if nwrite == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func TranslateName(accName *uint16, accNameFormat uint32, desiredNameFormat uint32, translatedName *uint16, nSize *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procTranslateNameW.Addr(), 5, uintptr(unsafe.Pointer(accName)), uintptr(accNameFormat), uintptr(desiredNameFormat), uintptr(unsafe.Pointer(translatedName)), uintptr(unsafe.Pointer(nSize)), 0)
+	if r1&0xff == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetUserNameEx(nameFormat uint32, nameBuffre *uint16, nSize *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procGetUserNameExW.Addr(), 3, uintptr(nameFormat), uintptr(unsafe.Pointer(nameBuffre)), uintptr(unsafe.Pointer(nSize)))
+	if r1&0xff == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func NetUserGetInfo(serverName *uint16, userName *uint16, level uint32, buf **byte) (neterr error) {
+	r0, _, _ := syscall.Syscall6(procNetUserGetInfo.Addr(), 4, uintptr(unsafe.Pointer(serverName)), uintptr(unsafe.Pointer(userName)), uintptr(level), uintptr(unsafe.Pointer(buf)), 0, 0)
+	if r0 != 0 {
+		neterr = syscall.Errno(r0)
+	}
+	return
+}
+
+func NetGetJoinInformation(server *uint16, name **uint16, bufType *uint32) (neterr error) {
+	r0, _, _ := syscall.Syscall(procNetGetJoinInformation.Addr(), 3, uintptr(unsafe.Pointer(server)), uintptr(unsafe.Pointer(name)), uintptr(unsafe.Pointer(bufType)))
+	if r0 != 0 {
+		neterr = syscall.Errno(r0)
+	}
+	return
+}
+
+func NetApiBufferFree(buf *byte) (neterr error) {
+	r0, _, _ := syscall.Syscall(procNetApiBufferFree.Addr(), 1, uintptr(unsafe.Pointer(buf)), 0, 0)
+	if r0 != 0 {
+		neterr = syscall.Errno(r0)
+	}
+	return
+}
+
+func LookupAccountSid(systemName *uint16, sid *SID, name *uint16, nameLen *uint32, refdDomainName *uint16, refdDomainNameLen *uint32, use *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall9(procLookupAccountSidW.Addr(), 7, uintptr(unsafe.Pointer(systemName)), uintptr(unsafe.Pointer(sid)), uintptr(unsafe.Pointer(name)), uintptr(unsafe.Pointer(nameLen)), uintptr(unsafe.Pointer(refdDomainName)), uintptr(unsafe.Pointer(refdDomainNameLen)), uintptr(unsafe.Pointer(use)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func LookupAccountName(systemName *uint16, accountName *uint16, sid *SID, sidLen *uint32, refdDomainName *uint16, refdDomainNameLen *uint32, use *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall9(procLookupAccountNameW.Addr(), 7, uintptr(unsafe.Pointer(systemName)), uintptr(unsafe.Pointer(accountName)), uintptr(unsafe.Pointer(sid)), uintptr(unsafe.Pointer(sidLen)), uintptr(unsafe.Pointer(refdDomainName)), uintptr(unsafe.Pointer(refdDomainNameLen)), uintptr(unsafe.Pointer(use)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func ConvertSidToStringSid(sid *SID, stringSid **uint16) (err error) {
+	r1, _, e1 := syscall.Syscall(procConvertSidToStringSidW.Addr(), 2, uintptr(unsafe.Pointer(sid)), uintptr(unsafe.Pointer(stringSid)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func ConvertStringSidToSid(stringSid *uint16, sid **SID) (err error) {
+	r1, _, e1 := syscall.Syscall(procConvertStringSidToSidW.Addr(), 2, uintptr(unsafe.Pointer(stringSid)), uintptr(unsafe.Pointer(sid)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetLengthSid(sid *SID) (len uint32) {
+	r0, _, _ := syscall.Syscall(procGetLengthSid.Addr(), 1, uintptr(unsafe.Pointer(sid)), 0, 0)
+	len = uint32(r0)
+	return
+}
+
+func CopySid(destSidLen uint32, destSid *SID, srcSid *SID) (err error) {
+	r1, _, e1 := syscall.Syscall(procCopySid.Addr(), 3, uintptr(destSidLen), uintptr(unsafe.Pointer(destSid)), uintptr(unsafe.Pointer(srcSid)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func AllocateAndInitializeSid(identAuth *SidIdentifierAuthority, subAuth byte, subAuth0 uint32, subAuth1 uint32, subAuth2 uint32, subAuth3 uint32, subAuth4 uint32, subAuth5 uint32, subAuth6 uint32, subAuth7 uint32, sid **SID) (err error) {
+	r1, _, e1 := syscall.Syscall12(procAllocateAndInitializeSid.Addr(), 11, uintptr(unsafe.Pointer(identAuth)), uintptr(subAuth), uintptr(subAuth0), uintptr(subAuth1), uintptr(subAuth2), uintptr(subAuth3), uintptr(subAuth4), uintptr(subAuth5), uintptr(subAuth6), uintptr(subAuth7), uintptr(unsafe.Pointer(sid)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func createWellKnownSid(sidType WELL_KNOWN_SID_TYPE, domainSid *SID, sid *SID, sizeSid *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procCreateWellKnownSid.Addr(), 4, uintptr(sidType), uintptr(unsafe.Pointer(domainSid)), uintptr(unsafe.Pointer(sid)), uintptr(unsafe.Pointer(sizeSid)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func isWellKnownSid(sid *SID, sidType WELL_KNOWN_SID_TYPE) (isWellKnown bool) {
+	r0, _, _ := syscall.Syscall(procIsWellKnownSid.Addr(), 2, uintptr(unsafe.Pointer(sid)), uintptr(sidType), 0)
+	isWellKnown = r0 != 0
+	return
+}
+
+func FreeSid(sid *SID) (err error) {
+	r1, _, e1 := syscall.Syscall(procFreeSid.Addr(), 1, uintptr(unsafe.Pointer(sid)), 0, 0)
+	if r1 != 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func EqualSid(sid1 *SID, sid2 *SID) (isEqual bool) {
+	r0, _, _ := syscall.Syscall(procEqualSid.Addr(), 2, uintptr(unsafe.Pointer(sid1)), uintptr(unsafe.Pointer(sid2)), 0)
+	isEqual = r0 != 0
+	return
+}
+
+func getSidIdentifierAuthority(sid *SID) (authority *SidIdentifierAuthority) {
+	r0, _, _ := syscall.Syscall(procGetSidIdentifierAuthority.Addr(), 1, uintptr(unsafe.Pointer(sid)), 0, 0)
+	authority = (*SidIdentifierAuthority)(unsafe.Pointer(r0))
+	return
+}
+
+func getSidSubAuthorityCount(sid *SID) (count *uint8) {
+	r0, _, _ := syscall.Syscall(procGetSidSubAuthorityCount.Addr(), 1, uintptr(unsafe.Pointer(sid)), 0, 0)
+	count = (*uint8)(unsafe.Pointer(r0))
+	return
+}
+
+func getSidSubAuthority(sid *SID, index uint32) (subAuthority *uint32) {
+	r0, _, _ := syscall.Syscall(procGetSidSubAuthority.Addr(), 2, uintptr(unsafe.Pointer(sid)), uintptr(index), 0)
+	subAuthority = (*uint32)(unsafe.Pointer(r0))
+	return
+}
+
+func isValidSid(sid *SID) (isValid bool) {
+	r0, _, _ := syscall.Syscall(procIsValidSid.Addr(), 1, uintptr(unsafe.Pointer(sid)), 0, 0)
+	isValid = r0 != 0
+	return
+}
+
+func checkTokenMembership(tokenHandle Token, sidToCheck *SID, isMember *int32) (err error) {
+	r1, _, e1 := syscall.Syscall(procCheckTokenMembership.Addr(), 3, uintptr(tokenHandle), uintptr(unsafe.Pointer(sidToCheck)), uintptr(unsafe.Pointer(isMember)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func OpenProcessToken(process Handle, access uint32, token *Token) (err error) {
+	r1, _, e1 := syscall.Syscall(procOpenProcessToken.Addr(), 3, uintptr(process), uintptr(access), uintptr(unsafe.Pointer(token)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func OpenThreadToken(thread Handle, access uint32, openAsSelf bool, token *Token) (err error) {
+	var _p0 uint32
+	if openAsSelf {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r1, _, e1 := syscall.Syscall6(procOpenThreadToken.Addr(), 4, uintptr(thread), uintptr(access), uintptr(_p0), uintptr(unsafe.Pointer(token)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func ImpersonateSelf(impersonationlevel uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procImpersonateSelf.Addr(), 1, uintptr(impersonationlevel), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func RevertToSelf() (err error) {
+	r1, _, e1 := syscall.Syscall(procRevertToSelf.Addr(), 0, 0, 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetThreadToken(thread *Handle, token Token) (err error) {
+	r1, _, e1 := syscall.Syscall(procSetThreadToken.Addr(), 2, uintptr(unsafe.Pointer(thread)), uintptr(token), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func LookupPrivilegeValue(systemname *uint16, name *uint16, luid *LUID) (err error) {
+	r1, _, e1 := syscall.Syscall(procLookupPrivilegeValueW.Addr(), 3, uintptr(unsafe.Pointer(systemname)), uintptr(unsafe.Pointer(name)), uintptr(unsafe.Pointer(luid)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func AdjustTokenPrivileges(token Token, disableAllPrivileges bool, newstate *Tokenprivileges, buflen uint32, prevstate *Tokenprivileges, returnlen *uint32) (err error) {
+	var _p0 uint32
+	if disableAllPrivileges {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r1, _, e1 := syscall.Syscall6(procAdjustTokenPrivileges.Addr(), 6, uintptr(token), uintptr(_p0), uintptr(unsafe.Pointer(newstate)), uintptr(buflen), uintptr(unsafe.Pointer(prevstate)), uintptr(unsafe.Pointer(returnlen)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func AdjustTokenGroups(token Token, resetToDefault bool, newstate *Tokengroups, buflen uint32, prevstate *Tokengroups, returnlen *uint32) (err error) {
+	var _p0 uint32
+	if resetToDefault {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r1, _, e1 := syscall.Syscall6(procAdjustTokenGroups.Addr(), 6, uintptr(token), uintptr(_p0), uintptr(unsafe.Pointer(newstate)), uintptr(buflen), uintptr(unsafe.Pointer(prevstate)), uintptr(unsafe.Pointer(returnlen)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetTokenInformation(token Token, infoClass uint32, info *byte, infoLen uint32, returnedLen *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procGetTokenInformation.Addr(), 5, uintptr(token), uintptr(infoClass), uintptr(unsafe.Pointer(info)), uintptr(infoLen), uintptr(unsafe.Pointer(returnedLen)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func SetTokenInformation(token Token, infoClass uint32, info *byte, infoLen uint32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procSetTokenInformation.Addr(), 4, uintptr(token), uintptr(infoClass), uintptr(unsafe.Pointer(info)), uintptr(infoLen), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func DuplicateTokenEx(existingToken Token, desiredAccess uint32, tokenAttributes *SecurityAttributes, impersonationLevel uint32, tokenType uint32, newToken *Token) (err error) {
+	r1, _, e1 := syscall.Syscall6(procDuplicateTokenEx.Addr(), 6, uintptr(existingToken), uintptr(desiredAccess), uintptr(unsafe.Pointer(tokenAttributes)), uintptr(impersonationLevel), uintptr(tokenType), uintptr(unsafe.Pointer(newToken)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func GetUserProfileDirectory(t Token, dir *uint16, dirLen *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procGetUserProfileDirectoryW.Addr(), 3, uintptr(t), uintptr(unsafe.Pointer(dir)), uintptr(unsafe.Pointer(dirLen)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func getSystemDirectory(dir *uint16, dirLen uint32) (len uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procGetSystemDirectoryW.Addr(), 2, uintptr(unsafe.Pointer(dir)), uintptr(dirLen), 0)
+	len = uint32(r0)
+	if len == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func getWindowsDirectory(dir *uint16, dirLen uint32) (len uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procGetWindowsDirectoryW.Addr(), 2, uintptr(unsafe.Pointer(dir)), uintptr(dirLen), 0)
+	len = uint32(r0)
+	if len == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func getSystemWindowsDirectory(dir *uint16, dirLen uint32) (len uint32, err error) {
+	r0, _, e1 := syscall.Syscall(procGetSystemWindowsDirectoryW.Addr(), 2, uintptr(unsafe.Pointer(dir)), uintptr(dirLen), 0)
+	len = uint32(r0)
+	if len == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func WTSQueryUserToken(session uint32, token *Token) (err error) {
+	r1, _, e1 := syscall.Syscall(procWTSQueryUserToken.Addr(), 2, uintptr(session), uintptr(unsafe.Pointer(token)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func WTSEnumerateSessions(handle Handle, reserved uint32, version uint32, sessions **WTS_SESSION_INFO, count *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procWTSEnumerateSessionsW.Addr(), 5, uintptr(handle), uintptr(reserved), uintptr(version), uintptr(unsafe.Pointer(sessions)), uintptr(unsafe.Pointer(count)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func WTSFreeMemory(ptr uintptr) {
+	syscall.Syscall(procWTSFreeMemory.Addr(), 1, uintptr(ptr), 0, 0)
+	return
+}
+
+func getSecurityInfo(handle Handle, objectType SE_OBJECT_TYPE, securityInformation SECURITY_INFORMATION, owner **SID, group **SID, dacl **ACL, sacl **ACL, sd **SECURITY_DESCRIPTOR) (ret error) {
+	r0, _, _ := syscall.Syscall9(procGetSecurityInfo.Addr(), 8, uintptr(handle), uintptr(objectType), uintptr(securityInformation), uintptr(unsafe.Pointer(owner)), uintptr(unsafe.Pointer(group)), uintptr(unsafe.Pointer(dacl)), uintptr(unsafe.Pointer(sacl)), uintptr(unsafe.Pointer(sd)), 0)
+	if r0 != 0 {
+		ret = syscall.Errno(r0)
+	}
+	return
+}
+
+func SetSecurityInfo(handle Handle, objectType SE_OBJECT_TYPE, securityInformation SECURITY_INFORMATION, owner *SID, group *SID, dacl *ACL, sacl *ACL) {
+	syscall.Syscall9(procSetSecurityInfo.Addr(), 7, uintptr(handle), uintptr(objectType), uintptr(securityInformation), uintptr(unsafe.Pointer(owner)), uintptr(unsafe.Pointer(group)), uintptr(unsafe.Pointer(dacl)), uintptr(unsafe.Pointer(sacl)), 0, 0)
+	return
+}
+
+func getNamedSecurityInfo(objectName string, objectType SE_OBJECT_TYPE, securityInformation SECURITY_INFORMATION, owner **SID, group **SID, dacl **ACL, sacl **ACL, sd **SECURITY_DESCRIPTOR) (ret error) {
+	var _p0 *uint16
+	_p0, ret = syscall.UTF16PtrFromString(objectName)
+	if ret != nil {
+		return
+	}
+	return _getNamedSecurityInfo(_p0, objectType, securityInformation, owner, group, dacl, sacl, sd)
+}
+
+func _getNamedSecurityInfo(objectName *uint16, objectType SE_OBJECT_TYPE, securityInformation SECURITY_INFORMATION, owner **SID, group **SID, dacl **ACL, sacl **ACL, sd **SECURITY_DESCRIPTOR) (ret error) {
+	r0, _, _ := syscall.Syscall9(procGetNamedSecurityInfoW.Addr(), 8, uintptr(unsafe.Pointer(objectName)), uintptr(objectType), uintptr(securityInformation), uintptr(unsafe.Pointer(owner)), uintptr(unsafe.Pointer(group)), uintptr(unsafe.Pointer(dacl)), uintptr(unsafe.Pointer(sacl)), uintptr(unsafe.Pointer(sd)), 0)
+	if r0 != 0 {
+		ret = syscall.Errno(r0)
+	}
+	return
+}
+
+func SetNamedSecurityInfo(objectName string, objectType SE_OBJECT_TYPE, securityInformation SECURITY_INFORMATION, owner *SID, group *SID, dacl *ACL, sacl *ACL) (ret error) {
+	var _p0 *uint16
+	_p0, ret = syscall.UTF16PtrFromString(objectName)
+	if ret != nil {
+		return
+	}
+	return _SetNamedSecurityInfo(_p0, objectType, securityInformation, owner, group, dacl, sacl)
+}
+
+func _SetNamedSecurityInfo(objectName *uint16, objectType SE_OBJECT_TYPE, securityInformation SECURITY_INFORMATION, owner *SID, group *SID, dacl *ACL, sacl *ACL) (ret error) {
+	r0, _, _ := syscall.Syscall9(procSetNamedSecurityInfoW.Addr(), 7, uintptr(unsafe.Pointer(objectName)), uintptr(objectType), uintptr(securityInformation), uintptr(unsafe.Pointer(owner)), uintptr(unsafe.Pointer(group)), uintptr(unsafe.Pointer(dacl)), uintptr(unsafe.Pointer(sacl)), 0, 0)
+	if r0 != 0 {
+		ret = syscall.Errno(r0)
+	}
+	return
+}
+
+func buildSecurityDescriptor(owner *TRUSTEE, group *TRUSTEE, countAccessEntries uint32, accessEntries *EXPLICIT_ACCESS, countAuditEntries uint32, auditEntries *EXPLICIT_ACCESS, oldSecurityDescriptor *SECURITY_DESCRIPTOR, sizeNewSecurityDescriptor *uint32, newSecurityDescriptor **SECURITY_DESCRIPTOR) (ret error) {
+	r0, _, _ := syscall.Syscall9(procBuildSecurityDescriptorW.Addr(), 9, uintptr(unsafe.Pointer(owner)), uintptr(unsafe.Pointer(group)), uintptr(countAccessEntries), uintptr(unsafe.Pointer(accessEntries)), uintptr(countAuditEntries), uintptr(unsafe.Pointer(auditEntries)), uintptr(unsafe.Pointer(oldSecurityDescriptor)), uintptr(unsafe.Pointer(sizeNewSecurityDescriptor)), uintptr(unsafe.Pointer(newSecurityDescriptor)))
+	if r0 != 0 {
+		ret = syscall.Errno(r0)
+	}
+	return
+}
+
+func initializeSecurityDescriptor(absoluteSD *SECURITY_DESCRIPTOR, revision uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procInitializeSecurityDescriptor.Addr(), 2, uintptr(unsafe.Pointer(absoluteSD)), uintptr(revision), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func getSecurityDescriptorControl(sd *SECURITY_DESCRIPTOR, control *SECURITY_DESCRIPTOR_CONTROL, revision *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procGetSecurityDescriptorControl.Addr(), 3, uintptr(unsafe.Pointer(sd)), uintptr(unsafe.Pointer(control)), uintptr(unsafe.Pointer(revision)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func getSecurityDescriptorDacl(sd *SECURITY_DESCRIPTOR, daclPresent *bool, dacl **ACL, daclDefaulted *bool) (err error) {
+	var _p0 uint32
+	if *daclPresent {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	var _p1 uint32
+	if *daclDefaulted {
+		_p1 = 1
+	} else {
+		_p1 = 0
+	}
+	r1, _, e1 := syscall.Syscall6(procGetSecurityDescriptorDacl.Addr(), 4, uintptr(unsafe.Pointer(sd)), uintptr(unsafe.Pointer(&_p0)), uintptr(unsafe.Pointer(dacl)), uintptr(unsafe.Pointer(&_p1)), 0, 0)
+	*daclPresent = _p0 != 0
+	*daclDefaulted = _p1 != 0
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func getSecurityDescriptorSacl(sd *SECURITY_DESCRIPTOR, saclPresent *bool, sacl **ACL, saclDefaulted *bool) (err error) {
+	var _p0 uint32
+	if *saclPresent {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	var _p1 uint32
+	if *saclDefaulted {
+		_p1 = 1
+	} else {
+		_p1 = 0
+	}
+	r1, _, e1 := syscall.Syscall6(procGetSecurityDescriptorSacl.Addr(), 4, uintptr(unsafe.Pointer(sd)), uintptr(unsafe.Pointer(&_p0)), uintptr(unsafe.Pointer(sacl)), uintptr(unsafe.Pointer(&_p1)), 0, 0)
+	*saclPresent = _p0 != 0
+	*saclDefaulted = _p1 != 0
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func getSecurityDescriptorOwner(sd *SECURITY_DESCRIPTOR, owner **SID, ownerDefaulted *bool) (err error) {
+	var _p0 uint32
+	if *ownerDefaulted {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r1, _, e1 := syscall.Syscall(procGetSecurityDescriptorOwner.Addr(), 3, uintptr(unsafe.Pointer(sd)), uintptr(unsafe.Pointer(owner)), uintptr(unsafe.Pointer(&_p0)))
+	*ownerDefaulted = _p0 != 0
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func getSecurityDescriptorGroup(sd *SECURITY_DESCRIPTOR, group **SID, groupDefaulted *bool) (err error) {
+	var _p0 uint32
+	if *groupDefaulted {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r1, _, e1 := syscall.Syscall(procGetSecurityDescriptorGroup.Addr(), 3, uintptr(unsafe.Pointer(sd)), uintptr(unsafe.Pointer(group)), uintptr(unsafe.Pointer(&_p0)))
+	*groupDefaulted = _p0 != 0
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func getSecurityDescriptorLength(sd *SECURITY_DESCRIPTOR) (len uint32) {
+	r0, _, _ := syscall.Syscall(procGetSecurityDescriptorLength.Addr(), 1, uintptr(unsafe.Pointer(sd)), 0, 0)
+	len = uint32(r0)
+	return
+}
+
+func getSecurityDescriptorRMControl(sd *SECURITY_DESCRIPTOR, rmControl *uint8) (ret error) {
+	r0, _, _ := syscall.Syscall(procGetSecurityDescriptorRMControl.Addr(), 2, uintptr(unsafe.Pointer(sd)), uintptr(unsafe.Pointer(rmControl)), 0)
+	if r0 != 0 {
+		ret = syscall.Errno(r0)
+	}
+	return
+}
+
+func isValidSecurityDescriptor(sd *SECURITY_DESCRIPTOR) (isValid bool) {
+	r0, _, _ := syscall.Syscall(procIsValidSecurityDescriptor.Addr(), 1, uintptr(unsafe.Pointer(sd)), 0, 0)
+	isValid = r0 != 0
+	return
+}
+
+func setSecurityDescriptorControl(sd *SECURITY_DESCRIPTOR, controlBitsOfInterest SECURITY_DESCRIPTOR_CONTROL, controlBitsToSet SECURITY_DESCRIPTOR_CONTROL) (err error) {
+	r1, _, e1 := syscall.Syscall(procSetSecurityDescriptorControl.Addr(), 3, uintptr(unsafe.Pointer(sd)), uintptr(controlBitsOfInterest), uintptr(controlBitsToSet))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func setSecurityDescriptorDacl(sd *SECURITY_DESCRIPTOR, daclPresent bool, dacl *ACL, daclDefaulted bool) (err error) {
+	var _p0 uint32
+	if daclPresent {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	var _p1 uint32
+	if daclDefaulted {
+		_p1 = 1
+	} else {
+		_p1 = 0
+	}
+	r1, _, e1 := syscall.Syscall6(procSetSecurityDescriptorDacl.Addr(), 4, uintptr(unsafe.Pointer(sd)), uintptr(_p0), uintptr(unsafe.Pointer(dacl)), uintptr(_p1), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func setSecurityDescriptorSacl(sd *SECURITY_DESCRIPTOR, saclPresent bool, sacl *ACL, saclDefaulted bool) (err error) {
+	var _p0 uint32
+	if saclPresent {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	var _p1 uint32
+	if saclDefaulted {
+		_p1 = 1
+	} else {
+		_p1 = 0
+	}
+	r1, _, e1 := syscall.Syscall6(procSetSecurityDescriptorSacl.Addr(), 4, uintptr(unsafe.Pointer(sd)), uintptr(_p0), uintptr(unsafe.Pointer(sacl)), uintptr(_p1), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func setSecurityDescriptorOwner(sd *SECURITY_DESCRIPTOR, owner *SID, ownerDefaulted bool) (err error) {
+	var _p0 uint32
+	if ownerDefaulted {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r1, _, e1 := syscall.Syscall(procSetSecurityDescriptorOwner.Addr(), 3, uintptr(unsafe.Pointer(sd)), uintptr(unsafe.Pointer(owner)), uintptr(_p0))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func setSecurityDescriptorGroup(sd *SECURITY_DESCRIPTOR, group *SID, groupDefaulted bool) (err error) {
+	var _p0 uint32
+	if groupDefaulted {
+		_p0 = 1
+	} else {
+		_p0 = 0
+	}
+	r1, _, e1 := syscall.Syscall(procSetSecurityDescriptorGroup.Addr(), 3, uintptr(unsafe.Pointer(sd)), uintptr(unsafe.Pointer(group)), uintptr(_p0))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func setSecurityDescriptorRMControl(sd *SECURITY_DESCRIPTOR, rmControl *uint8) {
+	syscall.Syscall(procSetSecurityDescriptorRMControl.Addr(), 2, uintptr(unsafe.Pointer(sd)), uintptr(unsafe.Pointer(rmControl)), 0)
+	return
+}
+
+func convertStringSecurityDescriptorToSecurityDescriptor(str string, revision uint32, sd **SECURITY_DESCRIPTOR, size *uint32) (err error) {
+	var _p0 *uint16
+	_p0, err = syscall.UTF16PtrFromString(str)
+	if err != nil {
+		return
+	}
+	return _convertStringSecurityDescriptorToSecurityDescriptor(_p0, revision, sd, size)
+}
+
+func _convertStringSecurityDescriptorToSecurityDescriptor(str *uint16, revision uint32, sd **SECURITY_DESCRIPTOR, size *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procConvertStringSecurityDescriptorToSecurityDescriptorW.Addr(), 4, uintptr(unsafe.Pointer(str)), uintptr(revision), uintptr(unsafe.Pointer(sd)), uintptr(unsafe.Pointer(size)), 0, 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func convertSecurityDescriptorToStringSecurityDescriptor(sd *SECURITY_DESCRIPTOR, revision uint32, securityInformation SECURITY_INFORMATION, str **uint16, strLen *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall6(procConvertSecurityDescriptorToStringSecurityDescriptorW.Addr(), 5, uintptr(unsafe.Pointer(sd)), uintptr(revision), uintptr(securityInformation), uintptr(unsafe.Pointer(str)), uintptr(unsafe.Pointer(strLen)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func makeAbsoluteSD(selfRelativeSD *SECURITY_DESCRIPTOR, absoluteSD *SECURITY_DESCRIPTOR, absoluteSDSize *uint32, dacl *ACL, daclSize *uint32, sacl *ACL, saclSize *uint32, owner *SID, ownerSize *uint32, group *SID, groupSize *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall12(procMakeAbsoluteSD.Addr(), 11, uintptr(unsafe.Pointer(selfRelativeSD)), uintptr(unsafe.Pointer(absoluteSD)), uintptr(unsafe.Pointer(absoluteSDSize)), uintptr(unsafe.Pointer(dacl)), uintptr(unsafe.Pointer(daclSize)), uintptr(unsafe.Pointer(sacl)), uintptr(unsafe.Pointer(saclSize)), uintptr(unsafe.Pointer(owner)), uintptr(unsafe.Pointer(ownerSize)), uintptr(unsafe.Pointer(group)), uintptr(unsafe.Pointer(groupSize)), 0)
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func makeSelfRelativeSD(absoluteSD *SECURITY_DESCRIPTOR, selfRelativeSD *SECURITY_DESCRIPTOR, selfRelativeSDSize *uint32) (err error) {
+	r1, _, e1 := syscall.Syscall(procMakeSelfRelativeSD.Addr(), 3, uintptr(unsafe.Pointer(absoluteSD)), uintptr(unsafe.Pointer(selfRelativeSD)), uintptr(unsafe.Pointer(selfRelativeSDSize)))
+	if r1 == 0 {
+		if e1 != 0 {
+			err = errnoErr(e1)
+		} else {
+			err = syscall.EINVAL
+		}
+	}
+	return
+}
+
+func setEntriesInAcl(countExplicitEntries uint32, explicitEntries *EXPLICIT_ACCESS, oldACL *ACL, newACL **ACL) (ret error) {
+	r0, _, _ := syscall.Syscall6(procSetEntriesInAclW.Addr(), 4, uintptr(countExplicitEntries), uintptr(unsafe.Pointer(explicitEntries)), uintptr(unsafe.Pointer(oldACL)), uintptr(unsafe.Pointer(newACL)), 0, 0)
+	if r0 != 0 {
+		ret = syscall.Errno(r0)
+	}
+	return
+}
diff --git a/vendor/golang.org/x/time/AUTHORS b/vendor/golang.org/x/time/AUTHORS
new file mode 100644
index 0000000..15167cd
--- /dev/null
+++ b/vendor/golang.org/x/time/AUTHORS
@@ -0,0 +1,3 @@
+# This source code refers to The Go Authors for copyright purposes.
+# The master list of authors is in the main Go distribution,
+# visible at http://tip.golang.org/AUTHORS.
diff --git a/vendor/golang.org/x/time/CONTRIBUTORS b/vendor/golang.org/x/time/CONTRIBUTORS
new file mode 100644
index 0000000..1c4577e
--- /dev/null
+++ b/vendor/golang.org/x/time/CONTRIBUTORS
@@ -0,0 +1,3 @@
+# This source code was written by the Go contributors.
+# The master list of contributors is in the main Go distribution,
+# visible at http://tip.golang.org/CONTRIBUTORS.
diff --git a/vendor/golang.org/x/time/LICENSE b/vendor/golang.org/x/time/LICENSE
new file mode 100644
index 0000000..6a66aea
--- /dev/null
+++ b/vendor/golang.org/x/time/LICENSE
@@ -0,0 +1,27 @@
+Copyright (c) 2009 The Go Authors. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+   * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+   * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+   * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/vendor/golang.org/x/time/PATENTS b/vendor/golang.org/x/time/PATENTS
new file mode 100644
index 0000000..7330990
--- /dev/null
+++ b/vendor/golang.org/x/time/PATENTS
@@ -0,0 +1,22 @@
+Additional IP Rights Grant (Patents)
+
+"This implementation" means the copyrightable works distributed by
+Google as part of the Go project.
+
+Google hereby grants to You a perpetual, worldwide, non-exclusive,
+no-charge, royalty-free, irrevocable (except as stated in this section)
+patent license to make, have made, use, offer to sell, sell, import,
+transfer and otherwise run, modify and propagate the contents of this
+implementation of Go, where such license applies only to those patent
+claims, both currently owned or controlled by Google and acquired in
+the future, licensable by Google that are necessarily infringed by this
+implementation of Go.  This grant does not include claims that would be
+infringed only as a consequence of further modification of this
+implementation.  If you or your agent or exclusive licensee institute or
+order or agree to the institution of patent litigation against any
+entity (including a cross-claim or counterclaim in a lawsuit) alleging
+that this implementation of Go or any code incorporated within this
+implementation of Go constitutes direct or contributory patent
+infringement, or inducement of patent infringement, then any patent
+rights granted to you under this License for this implementation of Go
+shall terminate as of the date such litigation is filed.
diff --git a/vendor/golang.org/x/time/rate/rate.go b/vendor/golang.org/x/time/rate/rate.go
new file mode 100644
index 0000000..7228d97
--- /dev/null
+++ b/vendor/golang.org/x/time/rate/rate.go
@@ -0,0 +1,384 @@
+// Copyright 2015 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// Package rate provides a rate limiter.
+package rate
+
+import (
+	"fmt"
+	"math"
+	"sync"
+	"time"
+)
+
+// Limit defines the maximum frequency of some events.
+// Limit is represented as number of events per second.
+// A zero Limit allows no events.
+type Limit float64
+
+// Inf is the infinite rate limit; it allows all events (even if burst is zero).
+const Inf = Limit(math.MaxFloat64)
+
+// Every converts a minimum time interval between events to a Limit.
+func Every(interval time.Duration) Limit {
+	if interval <= 0 {
+		return Inf
+	}
+	return 1 / Limit(interval.Seconds())
+}
+
+// A Limiter controls how frequently events are allowed to happen.
+// It implements a "token bucket" of size b, initially full and refilled
+// at rate r tokens per second.
+// Informally, in any large enough time interval, the Limiter limits the
+// rate to r tokens per second, with a maximum burst size of b events.
+// As a special case, if r == Inf (the infinite rate), b is ignored.
+// See https://en.wikipedia.org/wiki/Token_bucket for more about token buckets.
+//
+// The zero value is a valid Limiter, but it will reject all events.
+// Use NewLimiter to create non-zero Limiters.
+//
+// Limiter has three main methods, Allow, Reserve, and Wait.
+// Most callers should use Wait.
+//
+// Each of the three methods consumes a single token.
+// They differ in their behavior when no token is available.
+// If no token is available, Allow returns false.
+// If no token is available, Reserve returns a reservation for a future token
+// and the amount of time the caller must wait before using it.
+// If no token is available, Wait blocks until one can be obtained
+// or its associated context.Context is canceled.
+//
+// The methods AllowN, ReserveN, and WaitN consume n tokens.
+type Limiter struct {
+	limit Limit
+	burst int
+
+	mu     sync.Mutex
+	tokens float64
+	// last is the last time the limiter's tokens field was updated
+	last time.Time
+	// lastEvent is the latest time of a rate-limited event (past or future)
+	lastEvent time.Time
+}
+
+// Limit returns the maximum overall event rate.
+func (lim *Limiter) Limit() Limit {
+	lim.mu.Lock()
+	defer lim.mu.Unlock()
+	return lim.limit
+}
+
+// Burst returns the maximum burst size. Burst is the maximum number of tokens
+// that can be consumed in a single call to Allow, Reserve, or Wait, so higher
+// Burst values allow more events to happen at once.
+// A zero Burst allows no events, unless limit == Inf.
+func (lim *Limiter) Burst() int {
+	return lim.burst
+}
+
+// NewLimiter returns a new Limiter that allows events up to rate r and permits
+// bursts of at most b tokens.
+func NewLimiter(r Limit, b int) *Limiter {
+	return &Limiter{
+		limit: r,
+		burst: b,
+	}
+}
+
+// Allow is shorthand for AllowN(time.Now(), 1).
+func (lim *Limiter) Allow() bool {
+	return lim.AllowN(time.Now(), 1)
+}
+
+// AllowN reports whether n events may happen at time now.
+// Use this method if you intend to drop / skip events that exceed the rate limit.
+// Otherwise use Reserve or Wait.
+func (lim *Limiter) AllowN(now time.Time, n int) bool {
+	return lim.reserveN(now, n, 0).ok
+}
+
+// A Reservation holds information about events that are permitted by a Limiter to happen after a delay.
+// A Reservation may be canceled, which may enable the Limiter to permit additional events.
+type Reservation struct {
+	ok        bool
+	lim       *Limiter
+	tokens    int
+	timeToAct time.Time
+	// This is the Limit at reservation time, it can change later.
+	limit Limit
+}
+
+// OK returns whether the limiter can provide the requested number of tokens
+// within the maximum wait time.  If OK is false, Delay returns InfDuration, and
+// Cancel does nothing.
+func (r *Reservation) OK() bool {
+	return r.ok
+}
+
+// Delay is shorthand for DelayFrom(time.Now()).
+func (r *Reservation) Delay() time.Duration {
+	return r.DelayFrom(time.Now())
+}
+
+// InfDuration is the duration returned by Delay when a Reservation is not OK.
+const InfDuration = time.Duration(1<<63 - 1)
+
+// DelayFrom returns the duration for which the reservation holder must wait
+// before taking the reserved action.  Zero duration means act immediately.
+// InfDuration means the limiter cannot grant the tokens requested in this
+// Reservation within the maximum wait time.
+func (r *Reservation) DelayFrom(now time.Time) time.Duration {
+	if !r.ok {
+		return InfDuration
+	}
+	delay := r.timeToAct.Sub(now)
+	if delay < 0 {
+		return 0
+	}
+	return delay
+}
+
+// Cancel is shorthand for CancelAt(time.Now()).
+func (r *Reservation) Cancel() {
+	r.CancelAt(time.Now())
+	return
+}
+
+// CancelAt indicates that the reservation holder will not perform the reserved action
+// and reverses the effects of this Reservation on the rate limit as much as possible,
+// considering that other reservations may have already been made.
+func (r *Reservation) CancelAt(now time.Time) {
+	if !r.ok {
+		return
+	}
+
+	r.lim.mu.Lock()
+	defer r.lim.mu.Unlock()
+
+	if r.lim.limit == Inf || r.tokens == 0 || r.timeToAct.Before(now) {
+		return
+	}
+
+	// calculate tokens to restore
+	// The duration between lim.lastEvent and r.timeToAct tells us how many tokens were reserved
+	// after r was obtained. These tokens should not be restored.
+	restoreTokens := float64(r.tokens) - r.limit.tokensFromDuration(r.lim.lastEvent.Sub(r.timeToAct))
+	if restoreTokens <= 0 {
+		return
+	}
+	// advance time to now
+	now, _, tokens := r.lim.advance(now)
+	// calculate new number of tokens
+	tokens += restoreTokens
+	if burst := float64(r.lim.burst); tokens > burst {
+		tokens = burst
+	}
+	// update state
+	r.lim.last = now
+	r.lim.tokens = tokens
+	if r.timeToAct == r.lim.lastEvent {
+		prevEvent := r.timeToAct.Add(r.limit.durationFromTokens(float64(-r.tokens)))
+		if !prevEvent.Before(now) {
+			r.lim.lastEvent = prevEvent
+		}
+	}
+
+	return
+}
+
+// Reserve is shorthand for ReserveN(time.Now(), 1).
+func (lim *Limiter) Reserve() *Reservation {
+	return lim.ReserveN(time.Now(), 1)
+}
+
+// ReserveN returns a Reservation that indicates how long the caller must wait before n events happen.
+// The Limiter takes this Reservation into account when allowing future events.
+// ReserveN returns false if n exceeds the Limiter's burst size.
+// Usage example:
+//   r := lim.ReserveN(time.Now(), 1)
+//   if !r.OK() {
+//     // Not allowed to act! Did you remember to set lim.burst to be > 0 ?
+//     return
+//   }
+//   time.Sleep(r.Delay())
+//   Act()
+// Use this method if you wish to wait and slow down in accordance with the rate limit without dropping events.
+// If you need to respect a deadline or cancel the delay, use Wait instead.
+// To drop or skip events exceeding rate limit, use Allow instead.
+func (lim *Limiter) ReserveN(now time.Time, n int) *Reservation {
+	r := lim.reserveN(now, n, InfDuration)
+	return &r
+}
+
+// contextContext is a temporary(?) copy of the context.Context type
+// to support both Go 1.6 using golang.org/x/net/context and Go 1.7+
+// with the built-in context package. If people ever stop using Go 1.6
+// we can remove this.
+type contextContext interface {
+	Deadline() (deadline time.Time, ok bool)
+	Done() <-chan struct{}
+	Err() error
+	Value(key interface{}) interface{}
+}
+
+// Wait is shorthand for WaitN(ctx, 1).
+func (lim *Limiter) wait(ctx contextContext) (err error) {
+	return lim.WaitN(ctx, 1)
+}
+
+// WaitN blocks until lim permits n events to happen.
+// It returns an error if n exceeds the Limiter's burst size, the Context is
+// canceled, or the expected wait time exceeds the Context's Deadline.
+// The burst limit is ignored if the rate limit is Inf.
+func (lim *Limiter) waitN(ctx contextContext, n int) (err error) {
+	if n > lim.burst && lim.limit != Inf {
+		return fmt.Errorf("rate: Wait(n=%d) exceeds limiter's burst %d", n, lim.burst)
+	}
+	// Check if ctx is already cancelled
+	select {
+	case <-ctx.Done():
+		return ctx.Err()
+	default:
+	}
+	// Determine wait limit
+	now := time.Now()
+	waitLimit := InfDuration
+	if deadline, ok := ctx.Deadline(); ok {
+		waitLimit = deadline.Sub(now)
+	}
+	// Reserve
+	r := lim.reserveN(now, n, waitLimit)
+	if !r.ok {
+		return fmt.Errorf("rate: Wait(n=%d) would exceed context deadline", n)
+	}
+	// Wait if necessary
+	delay := r.DelayFrom(now)
+	if delay == 0 {
+		return nil
+	}
+	t := time.NewTimer(delay)
+	defer t.Stop()
+	select {
+	case <-t.C:
+		// We can proceed.
+		return nil
+	case <-ctx.Done():
+		// Context was canceled before we could proceed.  Cancel the
+		// reservation, which may permit other events to proceed sooner.
+		r.Cancel()
+		return ctx.Err()
+	}
+}
+
+// SetLimit is shorthand for SetLimitAt(time.Now(), newLimit).
+func (lim *Limiter) SetLimit(newLimit Limit) {
+	lim.SetLimitAt(time.Now(), newLimit)
+}
+
+// SetLimitAt sets a new Limit for the limiter. The new Limit, and Burst, may be violated
+// or underutilized by those which reserved (using Reserve or Wait) but did not yet act
+// before SetLimitAt was called.
+func (lim *Limiter) SetLimitAt(now time.Time, newLimit Limit) {
+	lim.mu.Lock()
+	defer lim.mu.Unlock()
+
+	now, _, tokens := lim.advance(now)
+
+	lim.last = now
+	lim.tokens = tokens
+	lim.limit = newLimit
+}
+
+// reserveN is a helper method for AllowN, ReserveN, and WaitN.
+// maxFutureReserve specifies the maximum reservation wait duration allowed.
+// reserveN returns Reservation, not *Reservation, to avoid allocation in AllowN and WaitN.
+func (lim *Limiter) reserveN(now time.Time, n int, maxFutureReserve time.Duration) Reservation {
+	lim.mu.Lock()
+
+	if lim.limit == Inf {
+		lim.mu.Unlock()
+		return Reservation{
+			ok:        true,
+			lim:       lim,
+			tokens:    n,
+			timeToAct: now,
+		}
+	}
+
+	now, last, tokens := lim.advance(now)
+
+	// Calculate the remaining number of tokens resulting from the request.
+	tokens -= float64(n)
+
+	// Calculate the wait duration
+	var waitDuration time.Duration
+	if tokens < 0 {
+		waitDuration = lim.limit.durationFromTokens(-tokens)
+	}
+
+	// Decide result
+	ok := n <= lim.burst && waitDuration <= maxFutureReserve
+
+	// Prepare reservation
+	r := Reservation{
+		ok:    ok,
+		lim:   lim,
+		limit: lim.limit,
+	}
+	if ok {
+		r.tokens = n
+		r.timeToAct = now.Add(waitDuration)
+	}
+
+	// Update state
+	if ok {
+		lim.last = now
+		lim.tokens = tokens
+		lim.lastEvent = r.timeToAct
+	} else {
+		lim.last = last
+	}
+
+	lim.mu.Unlock()
+	return r
+}
+
+// advance calculates and returns an updated state for lim resulting from the passage of time.
+// lim is not changed.
+func (lim *Limiter) advance(now time.Time) (newNow time.Time, newLast time.Time, newTokens float64) {
+	last := lim.last
+	if now.Before(last) {
+		last = now
+	}
+
+	// Avoid making delta overflow below when last is very old.
+	maxElapsed := lim.limit.durationFromTokens(float64(lim.burst) - lim.tokens)
+	elapsed := now.Sub(last)
+	if elapsed > maxElapsed {
+		elapsed = maxElapsed
+	}
+
+	// Calculate the new number of tokens, due to time that passed.
+	delta := lim.limit.tokensFromDuration(elapsed)
+	tokens := lim.tokens + delta
+	if burst := float64(lim.burst); tokens > burst {
+		tokens = burst
+	}
+
+	return now, last, tokens
+}
+
+// durationFromTokens is a unit conversion function from the number of tokens to the duration
+// of time it takes to accumulate them at a rate of limit tokens per second.
+func (limit Limit) durationFromTokens(tokens float64) time.Duration {
+	seconds := tokens / float64(limit)
+	return time.Nanosecond * time.Duration(1e9*seconds)
+}
+
+// tokensFromDuration is a unit conversion function from a time duration to the number of tokens
+// which could be accumulated during that duration at a rate of limit tokens per second.
+func (limit Limit) tokensFromDuration(d time.Duration) float64 {
+	return d.Seconds() * float64(limit)
+}
diff --git a/vendor/golang.org/x/time/rate/rate_go16.go b/vendor/golang.org/x/time/rate/rate_go16.go
new file mode 100644
index 0000000..6bab185
--- /dev/null
+++ b/vendor/golang.org/x/time/rate/rate_go16.go
@@ -0,0 +1,21 @@
+// Copyright 2017 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build !go1.7
+
+package rate
+
+import "golang.org/x/net/context"
+
+// Wait is shorthand for WaitN(ctx, 1).
+func (lim *Limiter) Wait(ctx context.Context) (err error) {
+	return lim.waitN(ctx, 1)
+}
+
+// WaitN blocks until lim permits n events to happen.
+// It returns an error if n exceeds the Limiter's burst size, the Context is
+// canceled, or the expected wait time exceeds the Context's Deadline.
+func (lim *Limiter) WaitN(ctx context.Context, n int) (err error) {
+	return lim.waitN(ctx, n)
+}
diff --git a/vendor/golang.org/x/time/rate/rate_go17.go b/vendor/golang.org/x/time/rate/rate_go17.go
new file mode 100644
index 0000000..f90d85f
--- /dev/null
+++ b/vendor/golang.org/x/time/rate/rate_go17.go
@@ -0,0 +1,21 @@
+// Copyright 2017 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// +build go1.7
+
+package rate
+
+import "context"
+
+// Wait is shorthand for WaitN(ctx, 1).
+func (lim *Limiter) Wait(ctx context.Context) (err error) {
+	return lim.waitN(ctx, 1)
+}
+
+// WaitN blocks until lim permits n events to happen.
+// It returns an error if n exceeds the Limiter's burst size, the Context is
+// canceled, or the expected wait time exceeds the Context's Deadline.
+func (lim *Limiter) WaitN(ctx context.Context, n int) (err error) {
+	return lim.waitN(ctx, n)
+}
diff --git a/vendor/google.golang.org/genproto/googleapis/api/httpbody/httpbody.pb.go b/vendor/google.golang.org/genproto/googleapis/api/httpbody/httpbody.pb.go
new file mode 100644
index 0000000..5b6c587
--- /dev/null
+++ b/vendor/google.golang.org/genproto/googleapis/api/httpbody/httpbody.pb.go
@@ -0,0 +1,146 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: google/api/httpbody.proto
+
+package httpbody
+
+import (
+	fmt "fmt"
+	math "math"
+
+	proto "github.com/golang/protobuf/proto"
+	any "github.com/golang/protobuf/ptypes/any"
+)
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
+
+// Message that represents an arbitrary HTTP body. It should only be used for
+// payload formats that can't be represented as JSON, such as raw binary or
+// an HTML page.
+//
+//
+// This message can be used both in streaming and non-streaming API methods in
+// the request as well as the response.
+//
+// It can be used as a top-level request field, which is convenient if one
+// wants to extract parameters from either the URL or HTTP template into the
+// request fields and also want access to the raw HTTP body.
+//
+// Example:
+//
+//     message GetResourceRequest {
+//       // A unique request id.
+//       string request_id = 1;
+//
+//       // The raw HTTP body is bound to this field.
+//       google.api.HttpBody http_body = 2;
+//     }
+//
+//     service ResourceService {
+//       rpc GetResource(GetResourceRequest) returns (google.api.HttpBody);
+//       rpc UpdateResource(google.api.HttpBody) returns
+//       (google.protobuf.Empty);
+//     }
+//
+// Example with streaming methods:
+//
+//     service CaldavService {
+//       rpc GetCalendar(stream google.api.HttpBody)
+//         returns (stream google.api.HttpBody);
+//       rpc UpdateCalendar(stream google.api.HttpBody)
+//         returns (stream google.api.HttpBody);
+//     }
+//
+// Use of this type only changes how the request and response bodies are
+// handled, all other features will continue to work unchanged.
+type HttpBody struct {
+	// The HTTP Content-Type header value specifying the content type of the body.
+	ContentType string `protobuf:"bytes,1,opt,name=content_type,json=contentType,proto3" json:"content_type,omitempty"`
+	// The HTTP request/response body as raw binary.
+	Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"`
+	// Application specific response metadata. Must be set in the first response
+	// for streaming APIs.
+	Extensions           []*any.Any `protobuf:"bytes,3,rep,name=extensions,proto3" json:"extensions,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
+	XXX_unrecognized     []byte     `json:"-"`
+	XXX_sizecache        int32      `json:"-"`
+}
+
+func (m *HttpBody) Reset()         { *m = HttpBody{} }
+func (m *HttpBody) String() string { return proto.CompactTextString(m) }
+func (*HttpBody) ProtoMessage()    {}
+func (*HttpBody) Descriptor() ([]byte, []int) {
+	return fileDescriptor_09ea2ecaa32a0070, []int{0}
+}
+
+func (m *HttpBody) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_HttpBody.Unmarshal(m, b)
+}
+func (m *HttpBody) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_HttpBody.Marshal(b, m, deterministic)
+}
+func (m *HttpBody) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_HttpBody.Merge(m, src)
+}
+func (m *HttpBody) XXX_Size() int {
+	return xxx_messageInfo_HttpBody.Size(m)
+}
+func (m *HttpBody) XXX_DiscardUnknown() {
+	xxx_messageInfo_HttpBody.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_HttpBody proto.InternalMessageInfo
+
+func (m *HttpBody) GetContentType() string {
+	if m != nil {
+		return m.ContentType
+	}
+	return ""
+}
+
+func (m *HttpBody) GetData() []byte {
+	if m != nil {
+		return m.Data
+	}
+	return nil
+}
+
+func (m *HttpBody) GetExtensions() []*any.Any {
+	if m != nil {
+		return m.Extensions
+	}
+	return nil
+}
+
+func init() {
+	proto.RegisterType((*HttpBody)(nil), "google.api.HttpBody")
+}
+
+func init() { proto.RegisterFile("google/api/httpbody.proto", fileDescriptor_09ea2ecaa32a0070) }
+
+var fileDescriptor_09ea2ecaa32a0070 = []byte{
+	// 229 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x54, 0x8f, 0x31, 0x4f, 0xc3, 0x30,
+	0x10, 0x85, 0xe5, 0xb6, 0x42, 0x70, 0x2d, 0x0c, 0x16, 0x43, 0x60, 0x0a, 0x4c, 0x99, 0x6c, 0x09,
+	0xd8, 0x3a, 0x35, 0x0b, 0xb0, 0x45, 0x11, 0x13, 0x0b, 0x72, 0x1a, 0xe3, 0x46, 0x2a, 0x77, 0xa7,
+	0xe6, 0x10, 0xf8, 0xef, 0xf0, 0x2b, 0x19, 0x11, 0x69, 0x2c, 0xe8, 0xf6, 0xe4, 0xef, 0x3d, 0xbf,
+	0x77, 0x70, 0x11, 0x88, 0xc2, 0xd6, 0x5b, 0xc7, 0x9d, 0xdd, 0x88, 0x70, 0x43, 0x6d, 0x34, 0xbc,
+	0x23, 0x21, 0x0d, 0x7b, 0x64, 0x1c, 0x77, 0x97, 0xc9, 0x36, 0x90, 0xe6, 0xfd, 0xd5, 0x3a, 0x1c,
+	0x6d, 0xd7, 0x1f, 0x70, 0xfc, 0x20, 0xc2, 0x25, 0xb5, 0x51, 0x5f, 0xc1, 0x62, 0x4d, 0x28, 0x1e,
+	0xe5, 0x45, 0x22, 0xfb, 0x4c, 0xe5, 0xaa, 0x38, 0xa9, 0xe7, 0xe3, 0xdb, 0x53, 0x64, 0xaf, 0x35,
+	0xcc, 0x5a, 0x27, 0x2e, 0x9b, 0xe4, 0xaa, 0x58, 0xd4, 0x83, 0xd6, 0x77, 0x00, 0xfe, 0x53, 0x3c,
+	0xf6, 0x1d, 0x61, 0x9f, 0x4d, 0xf3, 0x69, 0x31, 0xbf, 0x39, 0x37, 0x63, 0x7d, 0xaa, 0x34, 0x2b,
+	0x8c, 0xf5, 0x3f, 0x5f, 0xb9, 0x81, 0xb3, 0x35, 0xbd, 0x99, 0xbf, 0x95, 0xe5, 0x69, 0x1a, 0x52,
+	0xfd, 0x66, 0x2a, 0xf5, 0xbc, 0x1c, 0x61, 0xa0, 0xad, 0xc3, 0x60, 0x68, 0x17, 0x6c, 0xf0, 0x38,
+	0xfc, 0x68, 0xf7, 0xc8, 0x71, 0xd7, 0x1f, 0x1c, 0xbf, 0x4c, 0xe2, 0x5b, 0xa9, 0xaf, 0xc9, 0xec,
+	0x7e, 0x55, 0x3d, 0x36, 0x47, 0x43, 0xe2, 0xf6, 0x27, 0x00, 0x00, 0xff, 0xff, 0x78, 0xb9, 0x16,
+	0x2b, 0x2d, 0x01, 0x00, 0x00,
+}
diff --git a/vendor/google.golang.org/genproto/protobuf/field_mask/field_mask.pb.go b/vendor/google.golang.org/genproto/protobuf/field_mask/field_mask.pb.go
new file mode 100644
index 0000000..a0889f0
--- /dev/null
+++ b/vendor/google.golang.org/genproto/protobuf/field_mask/field_mask.pb.go
@@ -0,0 +1,282 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: google/protobuf/field_mask.proto
+
+package field_mask
+
+import (
+	fmt "fmt"
+	math "math"
+
+	proto "github.com/golang/protobuf/proto"
+)
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
+
+// `FieldMask` represents a set of symbolic field paths, for example:
+//
+//     paths: "f.a"
+//     paths: "f.b.d"
+//
+// Here `f` represents a field in some root message, `a` and `b`
+// fields in the message found in `f`, and `d` a field found in the
+// message in `f.b`.
+//
+// Field masks are used to specify a subset of fields that should be
+// returned by a get operation or modified by an update operation.
+// Field masks also have a custom JSON encoding (see below).
+//
+// # Field Masks in Projections
+//
+// When used in the context of a projection, a response message or
+// sub-message is filtered by the API to only contain those fields as
+// specified in the mask. For example, if the mask in the previous
+// example is applied to a response message as follows:
+//
+//     f {
+//       a : 22
+//       b {
+//         d : 1
+//         x : 2
+//       }
+//       y : 13
+//     }
+//     z: 8
+//
+// The result will not contain specific values for fields x,y and z
+// (their value will be set to the default, and omitted in proto text
+// output):
+//
+//
+//     f {
+//       a : 22
+//       b {
+//         d : 1
+//       }
+//     }
+//
+// A repeated field is not allowed except at the last position of a
+// paths string.
+//
+// If a FieldMask object is not present in a get operation, the
+// operation applies to all fields (as if a FieldMask of all fields
+// had been specified).
+//
+// Note that a field mask does not necessarily apply to the
+// top-level response message. In case of a REST get operation, the
+// field mask applies directly to the response, but in case of a REST
+// list operation, the mask instead applies to each individual message
+// in the returned resource list. In case of a REST custom method,
+// other definitions may be used. Where the mask applies will be
+// clearly documented together with its declaration in the API.  In
+// any case, the effect on the returned resource/resources is required
+// behavior for APIs.
+//
+// # Field Masks in Update Operations
+//
+// A field mask in update operations specifies which fields of the
+// targeted resource are going to be updated. The API is required
+// to only change the values of the fields as specified in the mask
+// and leave the others untouched. If a resource is passed in to
+// describe the updated values, the API ignores the values of all
+// fields not covered by the mask.
+//
+// If a repeated field is specified for an update operation, new values will
+// be appended to the existing repeated field in the target resource. Note that
+// a repeated field is only allowed in the last position of a `paths` string.
+//
+// If a sub-message is specified in the last position of the field mask for an
+// update operation, then new value will be merged into the existing sub-message
+// in the target resource.
+//
+// For example, given the target message:
+//
+//     f {
+//       b {
+//         d: 1
+//         x: 2
+//       }
+//       c: [1]
+//     }
+//
+// And an update message:
+//
+//     f {
+//       b {
+//         d: 10
+//       }
+//       c: [2]
+//     }
+//
+// then if the field mask is:
+//
+//  paths: ["f.b", "f.c"]
+//
+// then the result will be:
+//
+//     f {
+//       b {
+//         d: 10
+//         x: 2
+//       }
+//       c: [1, 2]
+//     }
+//
+// An implementation may provide options to override this default behavior for
+// repeated and message fields.
+//
+// In order to reset a field's value to the default, the field must
+// be in the mask and set to the default value in the provided resource.
+// Hence, in order to reset all fields of a resource, provide a default
+// instance of the resource and set all fields in the mask, or do
+// not provide a mask as described below.
+//
+// If a field mask is not present on update, the operation applies to
+// all fields (as if a field mask of all fields has been specified).
+// Note that in the presence of schema evolution, this may mean that
+// fields the client does not know and has therefore not filled into
+// the request will be reset to their default. If this is unwanted
+// behavior, a specific service may require a client to always specify
+// a field mask, producing an error if not.
+//
+// As with get operations, the location of the resource which
+// describes the updated values in the request message depends on the
+// operation kind. In any case, the effect of the field mask is
+// required to be honored by the API.
+//
+// ## Considerations for HTTP REST
+//
+// The HTTP kind of an update operation which uses a field mask must
+// be set to PATCH instead of PUT in order to satisfy HTTP semantics
+// (PUT must only be used for full updates).
+//
+// # JSON Encoding of Field Masks
+//
+// In JSON, a field mask is encoded as a single string where paths are
+// separated by a comma. Fields name in each path are converted
+// to/from lower-camel naming conventions.
+//
+// As an example, consider the following message declarations:
+//
+//     message Profile {
+//       User user = 1;
+//       Photo photo = 2;
+//     }
+//     message User {
+//       string display_name = 1;
+//       string address = 2;
+//     }
+//
+// In proto a field mask for `Profile` may look as such:
+//
+//     mask {
+//       paths: "user.display_name"
+//       paths: "photo"
+//     }
+//
+// In JSON, the same mask is represented as below:
+//
+//     {
+//       mask: "user.displayName,photo"
+//     }
+//
+// # Field Masks and Oneof Fields
+//
+// Field masks treat fields in oneofs just as regular fields. Consider the
+// following message:
+//
+//     message SampleMessage {
+//       oneof test_oneof {
+//         string name = 4;
+//         SubMessage sub_message = 9;
+//       }
+//     }
+//
+// The field mask can be:
+//
+//     mask {
+//       paths: "name"
+//     }
+//
+// Or:
+//
+//     mask {
+//       paths: "sub_message"
+//     }
+//
+// Note that oneof type names ("test_oneof" in this case) cannot be used in
+// paths.
+//
+// ## Field Mask Verification
+//
+// The implementation of any API method which has a FieldMask type field in the
+// request should verify the included field paths, and return an
+// `INVALID_ARGUMENT` error if any path is duplicated or unmappable.
+type FieldMask struct {
+	// The set of field mask paths.
+	Paths                []string `protobuf:"bytes,1,rep,name=paths,proto3" json:"paths,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *FieldMask) Reset()         { *m = FieldMask{} }
+func (m *FieldMask) String() string { return proto.CompactTextString(m) }
+func (*FieldMask) ProtoMessage()    {}
+func (*FieldMask) Descriptor() ([]byte, []int) {
+	return fileDescriptor_5158202634f0da48, []int{0}
+}
+
+func (m *FieldMask) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_FieldMask.Unmarshal(m, b)
+}
+func (m *FieldMask) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_FieldMask.Marshal(b, m, deterministic)
+}
+func (m *FieldMask) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_FieldMask.Merge(m, src)
+}
+func (m *FieldMask) XXX_Size() int {
+	return xxx_messageInfo_FieldMask.Size(m)
+}
+func (m *FieldMask) XXX_DiscardUnknown() {
+	xxx_messageInfo_FieldMask.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_FieldMask proto.InternalMessageInfo
+
+func (m *FieldMask) GetPaths() []string {
+	if m != nil {
+		return m.Paths
+	}
+	return nil
+}
+
+func init() {
+	proto.RegisterType((*FieldMask)(nil), "google.protobuf.FieldMask")
+}
+
+func init() { proto.RegisterFile("google/protobuf/field_mask.proto", fileDescriptor_5158202634f0da48) }
+
+var fileDescriptor_5158202634f0da48 = []byte{
+	// 175 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x52, 0x48, 0xcf, 0xcf, 0x4f,
+	0xcf, 0x49, 0xd5, 0x2f, 0x28, 0xca, 0x2f, 0xc9, 0x4f, 0x2a, 0x4d, 0xd3, 0x4f, 0xcb, 0x4c, 0xcd,
+	0x49, 0x89, 0xcf, 0x4d, 0x2c, 0xce, 0xd6, 0x03, 0x8b, 0x09, 0xf1, 0x43, 0x54, 0xe8, 0xc1, 0x54,
+	0x28, 0x29, 0x72, 0x71, 0xba, 0x81, 0x14, 0xf9, 0x26, 0x16, 0x67, 0x0b, 0x89, 0x70, 0xb1, 0x16,
+	0x24, 0x96, 0x64, 0x14, 0x4b, 0x30, 0x2a, 0x30, 0x6b, 0x70, 0x06, 0x41, 0x38, 0x4e, 0x3d, 0x8c,
+	0x5c, 0xc2, 0xc9, 0xf9, 0xb9, 0x7a, 0x68, 0x5a, 0x9d, 0xf8, 0xe0, 0x1a, 0x03, 0x40, 0x42, 0x01,
+	0x8c, 0x51, 0x96, 0x50, 0x25, 0xe9, 0xf9, 0x39, 0x89, 0x79, 0xe9, 0x7a, 0xf9, 0x45, 0xe9, 0xfa,
+	0xe9, 0xa9, 0x79, 0x60, 0x0d, 0xd8, 0xdc, 0x64, 0x8d, 0x60, 0xfe, 0x60, 0x64, 0x5c, 0xc4, 0xc4,
+	0xec, 0x1e, 0xe0, 0xb4, 0x8a, 0x49, 0xce, 0x1d, 0x62, 0x48, 0x00, 0x54, 0x83, 0x5e, 0x78, 0x6a,
+	0x4e, 0x8e, 0x77, 0x5e, 0x7e, 0x79, 0x5e, 0x48, 0x65, 0x41, 0x6a, 0x71, 0x12, 0x1b, 0xd8, 0x24,
+	0x63, 0x40, 0x00, 0x00, 0x00, 0xff, 0xff, 0xfd, 0xda, 0xb7, 0xa8, 0xed, 0x00, 0x00, 0x00,
+}
diff --git a/vendor/google.golang.org/grpc/health/client.go b/vendor/google.golang.org/grpc/health/client.go
new file mode 100644
index 0000000..b43746e
--- /dev/null
+++ b/vendor/google.golang.org/grpc/health/client.go
@@ -0,0 +1,117 @@
+/*
+ *
+ * Copyright 2018 gRPC 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 health
+
+import (
+	"context"
+	"fmt"
+	"io"
+	"time"
+
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/connectivity"
+	healthpb "google.golang.org/grpc/health/grpc_health_v1"
+	"google.golang.org/grpc/internal"
+	"google.golang.org/grpc/internal/backoff"
+	"google.golang.org/grpc/status"
+)
+
+const maxDelay = 120 * time.Second
+
+var backoffStrategy = backoff.Exponential{MaxDelay: maxDelay}
+var backoffFunc = func(ctx context.Context, retries int) bool {
+	d := backoffStrategy.Backoff(retries)
+	timer := time.NewTimer(d)
+	select {
+	case <-timer.C:
+		return true
+	case <-ctx.Done():
+		timer.Stop()
+		return false
+	}
+}
+
+func init() {
+	internal.HealthCheckFunc = clientHealthCheck
+}
+
+const healthCheckMethod = "/grpc.health.v1.Health/Watch"
+
+// This function implements the protocol defined at:
+// https://github.com/grpc/grpc/blob/master/doc/health-checking.md
+func clientHealthCheck(ctx context.Context, newStream func(string) (interface{}, error), setConnectivityState func(connectivity.State), service string) error {
+	tryCnt := 0
+
+retryConnection:
+	for {
+		// Backs off if the connection has failed in some way without receiving a message in the previous retry.
+		if tryCnt > 0 && !backoffFunc(ctx, tryCnt-1) {
+			return nil
+		}
+		tryCnt++
+
+		if ctx.Err() != nil {
+			return nil
+		}
+		setConnectivityState(connectivity.Connecting)
+		rawS, err := newStream(healthCheckMethod)
+		if err != nil {
+			continue retryConnection
+		}
+
+		s, ok := rawS.(grpc.ClientStream)
+		// Ideally, this should never happen. But if it happens, the server is marked as healthy for LBing purposes.
+		if !ok {
+			setConnectivityState(connectivity.Ready)
+			return fmt.Errorf("newStream returned %v (type %T); want grpc.ClientStream", rawS, rawS)
+		}
+
+		if err = s.SendMsg(&healthpb.HealthCheckRequest{Service: service}); err != nil && err != io.EOF {
+			// Stream should have been closed, so we can safely continue to create a new stream.
+			continue retryConnection
+		}
+		s.CloseSend()
+
+		resp := new(healthpb.HealthCheckResponse)
+		for {
+			err = s.RecvMsg(resp)
+
+			// Reports healthy for the LBing purposes if health check is not implemented in the server.
+			if status.Code(err) == codes.Unimplemented {
+				setConnectivityState(connectivity.Ready)
+				return err
+			}
+
+			// Reports unhealthy if server's Watch method gives an error other than UNIMPLEMENTED.
+			if err != nil {
+				setConnectivityState(connectivity.TransientFailure)
+				continue retryConnection
+			}
+
+			// As a message has been received, removes the need for backoff for the next retry by reseting the try count.
+			tryCnt = 0
+			if resp.Status == healthpb.HealthCheckResponse_SERVING {
+				setConnectivityState(connectivity.Ready)
+			} else {
+				setConnectivityState(connectivity.TransientFailure)
+			}
+		}
+	}
+}
diff --git a/vendor/google.golang.org/grpc/health/grpc_health_v1/health.pb.go b/vendor/google.golang.org/grpc/health/grpc_health_v1/health.pb.go
new file mode 100644
index 0000000..c99e27a
--- /dev/null
+++ b/vendor/google.golang.org/grpc/health/grpc_health_v1/health.pb.go
@@ -0,0 +1,343 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// source: grpc/health/v1/health.proto
+
+package grpc_health_v1
+
+import (
+	context "context"
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	grpc "google.golang.org/grpc"
+	codes "google.golang.org/grpc/codes"
+	status "google.golang.org/grpc/status"
+	math "math"
+)
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the proto package it is being compiled against.
+// A compilation error at this line likely means your copy of the
+// proto package needs to be updated.
+const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
+
+type HealthCheckResponse_ServingStatus int32
+
+const (
+	HealthCheckResponse_UNKNOWN         HealthCheckResponse_ServingStatus = 0
+	HealthCheckResponse_SERVING         HealthCheckResponse_ServingStatus = 1
+	HealthCheckResponse_NOT_SERVING     HealthCheckResponse_ServingStatus = 2
+	HealthCheckResponse_SERVICE_UNKNOWN HealthCheckResponse_ServingStatus = 3
+)
+
+var HealthCheckResponse_ServingStatus_name = map[int32]string{
+	0: "UNKNOWN",
+	1: "SERVING",
+	2: "NOT_SERVING",
+	3: "SERVICE_UNKNOWN",
+}
+
+var HealthCheckResponse_ServingStatus_value = map[string]int32{
+	"UNKNOWN":         0,
+	"SERVING":         1,
+	"NOT_SERVING":     2,
+	"SERVICE_UNKNOWN": 3,
+}
+
+func (x HealthCheckResponse_ServingStatus) String() string {
+	return proto.EnumName(HealthCheckResponse_ServingStatus_name, int32(x))
+}
+
+func (HealthCheckResponse_ServingStatus) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_e265fd9d4e077217, []int{1, 0}
+}
+
+type HealthCheckRequest struct {
+	Service              string   `protobuf:"bytes,1,opt,name=service,proto3" json:"service,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *HealthCheckRequest) Reset()         { *m = HealthCheckRequest{} }
+func (m *HealthCheckRequest) String() string { return proto.CompactTextString(m) }
+func (*HealthCheckRequest) ProtoMessage()    {}
+func (*HealthCheckRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e265fd9d4e077217, []int{0}
+}
+
+func (m *HealthCheckRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_HealthCheckRequest.Unmarshal(m, b)
+}
+func (m *HealthCheckRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_HealthCheckRequest.Marshal(b, m, deterministic)
+}
+func (m *HealthCheckRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_HealthCheckRequest.Merge(m, src)
+}
+func (m *HealthCheckRequest) XXX_Size() int {
+	return xxx_messageInfo_HealthCheckRequest.Size(m)
+}
+func (m *HealthCheckRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_HealthCheckRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_HealthCheckRequest proto.InternalMessageInfo
+
+func (m *HealthCheckRequest) GetService() string {
+	if m != nil {
+		return m.Service
+	}
+	return ""
+}
+
+type HealthCheckResponse struct {
+	Status               HealthCheckResponse_ServingStatus `protobuf:"varint,1,opt,name=status,proto3,enum=grpc.health.v1.HealthCheckResponse_ServingStatus" json:"status,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                          `json:"-"`
+	XXX_unrecognized     []byte                            `json:"-"`
+	XXX_sizecache        int32                             `json:"-"`
+}
+
+func (m *HealthCheckResponse) Reset()         { *m = HealthCheckResponse{} }
+func (m *HealthCheckResponse) String() string { return proto.CompactTextString(m) }
+func (*HealthCheckResponse) ProtoMessage()    {}
+func (*HealthCheckResponse) Descriptor() ([]byte, []int) {
+	return fileDescriptor_e265fd9d4e077217, []int{1}
+}
+
+func (m *HealthCheckResponse) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_HealthCheckResponse.Unmarshal(m, b)
+}
+func (m *HealthCheckResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_HealthCheckResponse.Marshal(b, m, deterministic)
+}
+func (m *HealthCheckResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_HealthCheckResponse.Merge(m, src)
+}
+func (m *HealthCheckResponse) XXX_Size() int {
+	return xxx_messageInfo_HealthCheckResponse.Size(m)
+}
+func (m *HealthCheckResponse) XXX_DiscardUnknown() {
+	xxx_messageInfo_HealthCheckResponse.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_HealthCheckResponse proto.InternalMessageInfo
+
+func (m *HealthCheckResponse) GetStatus() HealthCheckResponse_ServingStatus {
+	if m != nil {
+		return m.Status
+	}
+	return HealthCheckResponse_UNKNOWN
+}
+
+func init() {
+	proto.RegisterEnum("grpc.health.v1.HealthCheckResponse_ServingStatus", HealthCheckResponse_ServingStatus_name, HealthCheckResponse_ServingStatus_value)
+	proto.RegisterType((*HealthCheckRequest)(nil), "grpc.health.v1.HealthCheckRequest")
+	proto.RegisterType((*HealthCheckResponse)(nil), "grpc.health.v1.HealthCheckResponse")
+}
+
+func init() { proto.RegisterFile("grpc/health/v1/health.proto", fileDescriptor_e265fd9d4e077217) }
+
+var fileDescriptor_e265fd9d4e077217 = []byte{
+	// 297 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x4e, 0x2f, 0x2a, 0x48,
+	0xd6, 0xcf, 0x48, 0x4d, 0xcc, 0x29, 0xc9, 0xd0, 0x2f, 0x33, 0x84, 0xb2, 0xf4, 0x0a, 0x8a, 0xf2,
+	0x4b, 0xf2, 0x85, 0xf8, 0x40, 0x92, 0x7a, 0x50, 0xa1, 0x32, 0x43, 0x25, 0x3d, 0x2e, 0x21, 0x0f,
+	0x30, 0xc7, 0x39, 0x23, 0x35, 0x39, 0x3b, 0x28, 0xb5, 0xb0, 0x34, 0xb5, 0xb8, 0x44, 0x48, 0x82,
+	0x8b, 0xbd, 0x38, 0xb5, 0xa8, 0x2c, 0x33, 0x39, 0x55, 0x82, 0x51, 0x81, 0x51, 0x83, 0x33, 0x08,
+	0xc6, 0x55, 0xda, 0xc8, 0xc8, 0x25, 0x8c, 0xa2, 0xa1, 0xb8, 0x20, 0x3f, 0xaf, 0x38, 0x55, 0xc8,
+	0x93, 0x8b, 0xad, 0xb8, 0x24, 0xb1, 0xa4, 0xb4, 0x18, 0xac, 0x81, 0xcf, 0xc8, 0x50, 0x0f, 0xd5,
+	0x22, 0x3d, 0x2c, 0x9a, 0xf4, 0x82, 0x41, 0x86, 0xe6, 0xa5, 0x07, 0x83, 0x35, 0x06, 0x41, 0x0d,
+	0x50, 0xf2, 0xe7, 0xe2, 0x45, 0x91, 0x10, 0xe2, 0xe6, 0x62, 0x0f, 0xf5, 0xf3, 0xf6, 0xf3, 0x0f,
+	0xf7, 0x13, 0x60, 0x00, 0x71, 0x82, 0x5d, 0x83, 0xc2, 0x3c, 0xfd, 0xdc, 0x05, 0x18, 0x85, 0xf8,
+	0xb9, 0xb8, 0xfd, 0xfc, 0x43, 0xe2, 0x61, 0x02, 0x4c, 0x42, 0xc2, 0x5c, 0xfc, 0x60, 0x8e, 0xb3,
+	0x6b, 0x3c, 0x4c, 0x0b, 0xb3, 0xd1, 0x3a, 0x46, 0x2e, 0x36, 0x88, 0xf5, 0x42, 0x01, 0x5c, 0xac,
+	0x60, 0x27, 0x08, 0x29, 0xe1, 0x75, 0x1f, 0x38, 0x14, 0xa4, 0x94, 0x89, 0xf0, 0x83, 0x50, 0x10,
+	0x17, 0x6b, 0x78, 0x62, 0x49, 0x72, 0x06, 0xd5, 0x4c, 0x34, 0x60, 0x74, 0x4a, 0xe4, 0x12, 0xcc,
+	0xcc, 0x47, 0x53, 0xea, 0xc4, 0x0d, 0x51, 0x1b, 0x00, 0x8a, 0xc6, 0x00, 0xc6, 0x28, 0x9d, 0xf4,
+	0xfc, 0xfc, 0xf4, 0x9c, 0x54, 0xbd, 0xf4, 0xfc, 0x9c, 0xc4, 0xbc, 0x74, 0xbd, 0xfc, 0xa2, 0x74,
+	0x7d, 0xe4, 0x78, 0x07, 0xb1, 0xe3, 0x21, 0xec, 0xf8, 0x32, 0xc3, 0x55, 0x4c, 0x7c, 0xee, 0x20,
+	0xd3, 0x20, 0x46, 0xe8, 0x85, 0x19, 0x26, 0xb1, 0x81, 0x93, 0x83, 0x31, 0x20, 0x00, 0x00, 0xff,
+	0xff, 0x12, 0x7d, 0x96, 0xcb, 0x2d, 0x02, 0x00, 0x00,
+}
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ context.Context
+var _ grpc.ClientConn
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the grpc package it is being compiled against.
+const _ = grpc.SupportPackageIsVersion4
+
+// HealthClient is the client API for Health service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
+type HealthClient interface {
+	// If the requested service is unknown, the call will fail with status
+	// NOT_FOUND.
+	Check(ctx context.Context, in *HealthCheckRequest, opts ...grpc.CallOption) (*HealthCheckResponse, error)
+	// Performs a watch for the serving status of the requested service.
+	// The server will immediately send back a message indicating the current
+	// serving status.  It will then subsequently send a new message whenever
+	// the service's serving status changes.
+	//
+	// If the requested service is unknown when the call is received, the
+	// server will send a message setting the serving status to
+	// SERVICE_UNKNOWN but will *not* terminate the call.  If at some
+	// future point, the serving status of the service becomes known, the
+	// server will send a new message with the service's serving status.
+	//
+	// If the call terminates with status UNIMPLEMENTED, then clients
+	// should assume this method is not supported and should not retry the
+	// call.  If the call terminates with any other status (including OK),
+	// clients should retry the call with appropriate exponential backoff.
+	Watch(ctx context.Context, in *HealthCheckRequest, opts ...grpc.CallOption) (Health_WatchClient, error)
+}
+
+type healthClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewHealthClient(cc *grpc.ClientConn) HealthClient {
+	return &healthClient{cc}
+}
+
+func (c *healthClient) Check(ctx context.Context, in *HealthCheckRequest, opts ...grpc.CallOption) (*HealthCheckResponse, error) {
+	out := new(HealthCheckResponse)
+	err := c.cc.Invoke(ctx, "/grpc.health.v1.Health/Check", in, out, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
+func (c *healthClient) Watch(ctx context.Context, in *HealthCheckRequest, opts ...grpc.CallOption) (Health_WatchClient, error) {
+	stream, err := c.cc.NewStream(ctx, &_Health_serviceDesc.Streams[0], "/grpc.health.v1.Health/Watch", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &healthWatchClient{stream}
+	if err := x.ClientStream.SendMsg(in); err != nil {
+		return nil, err
+	}
+	if err := x.ClientStream.CloseSend(); err != nil {
+		return nil, err
+	}
+	return x, nil
+}
+
+type Health_WatchClient interface {
+	Recv() (*HealthCheckResponse, error)
+	grpc.ClientStream
+}
+
+type healthWatchClient struct {
+	grpc.ClientStream
+}
+
+func (x *healthWatchClient) Recv() (*HealthCheckResponse, error) {
+	m := new(HealthCheckResponse)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+// HealthServer is the server API for Health service.
+type HealthServer interface {
+	// If the requested service is unknown, the call will fail with status
+	// NOT_FOUND.
+	Check(context.Context, *HealthCheckRequest) (*HealthCheckResponse, error)
+	// Performs a watch for the serving status of the requested service.
+	// The server will immediately send back a message indicating the current
+	// serving status.  It will then subsequently send a new message whenever
+	// the service's serving status changes.
+	//
+	// If the requested service is unknown when the call is received, the
+	// server will send a message setting the serving status to
+	// SERVICE_UNKNOWN but will *not* terminate the call.  If at some
+	// future point, the serving status of the service becomes known, the
+	// server will send a new message with the service's serving status.
+	//
+	// If the call terminates with status UNIMPLEMENTED, then clients
+	// should assume this method is not supported and should not retry the
+	// call.  If the call terminates with any other status (including OK),
+	// clients should retry the call with appropriate exponential backoff.
+	Watch(*HealthCheckRequest, Health_WatchServer) error
+}
+
+// UnimplementedHealthServer can be embedded to have forward compatible implementations.
+type UnimplementedHealthServer struct {
+}
+
+func (*UnimplementedHealthServer) Check(ctx context.Context, req *HealthCheckRequest) (*HealthCheckResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method Check not implemented")
+}
+func (*UnimplementedHealthServer) Watch(req *HealthCheckRequest, srv Health_WatchServer) error {
+	return status.Errorf(codes.Unimplemented, "method Watch not implemented")
+}
+
+func RegisterHealthServer(s *grpc.Server, srv HealthServer) {
+	s.RegisterService(&_Health_serviceDesc, srv)
+}
+
+func _Health_Check_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(HealthCheckRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(HealthServer).Check(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/grpc.health.v1.Health/Check",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(HealthServer).Check(ctx, req.(*HealthCheckRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _Health_Watch_Handler(srv interface{}, stream grpc.ServerStream) error {
+	m := new(HealthCheckRequest)
+	if err := stream.RecvMsg(m); err != nil {
+		return err
+	}
+	return srv.(HealthServer).Watch(m, &healthWatchServer{stream})
+}
+
+type Health_WatchServer interface {
+	Send(*HealthCheckResponse) error
+	grpc.ServerStream
+}
+
+type healthWatchServer struct {
+	grpc.ServerStream
+}
+
+func (x *healthWatchServer) Send(m *HealthCheckResponse) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+var _Health_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "grpc.health.v1.Health",
+	HandlerType: (*HealthServer)(nil),
+	Methods: []grpc.MethodDesc{
+		{
+			MethodName: "Check",
+			Handler:    _Health_Check_Handler,
+		},
+	},
+	Streams: []grpc.StreamDesc{
+		{
+			StreamName:    "Watch",
+			Handler:       _Health_Watch_Handler,
+			ServerStreams: true,
+		},
+	},
+	Metadata: "grpc/health/v1/health.proto",
+}
diff --git a/vendor/google.golang.org/grpc/health/regenerate.sh b/vendor/google.golang.org/grpc/health/regenerate.sh
new file mode 100644
index 0000000..b11eccb
--- /dev/null
+++ b/vendor/google.golang.org/grpc/health/regenerate.sh
@@ -0,0 +1,33 @@
+#!/bin/bash
+# Copyright 2018 gRPC 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.
+
+set -eux -o pipefail
+
+TMP=$(mktemp -d)
+
+function finish {
+  rm -rf "$TMP"
+}
+trap finish EXIT
+
+pushd "$TMP"
+mkdir -p grpc/health/v1
+curl https://raw.githubusercontent.com/grpc/grpc-proto/master/grpc/health/v1/health.proto > grpc/health/v1/health.proto
+
+protoc --go_out=plugins=grpc,paths=source_relative:. -I. grpc/health/v1/*.proto
+popd
+rm -f grpc_health_v1/*.pb.go
+cp "$TMP"/grpc/health/v1/*.pb.go grpc_health_v1/
+
diff --git a/vendor/google.golang.org/grpc/health/server.go b/vendor/google.golang.org/grpc/health/server.go
new file mode 100644
index 0000000..c79f9d2
--- /dev/null
+++ b/vendor/google.golang.org/grpc/health/server.go
@@ -0,0 +1,165 @@
+/*
+ *
+ * Copyright 2017 gRPC 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.
+ *
+ */
+
+//go:generate ./regenerate.sh
+
+// Package health provides a service that exposes server's health and it must be
+// imported to enable support for client-side health checks.
+package health
+
+import (
+	"context"
+	"sync"
+
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/grpclog"
+	healthgrpc "google.golang.org/grpc/health/grpc_health_v1"
+	healthpb "google.golang.org/grpc/health/grpc_health_v1"
+	"google.golang.org/grpc/status"
+)
+
+// Server implements `service Health`.
+type Server struct {
+	mu sync.Mutex
+	// If shutdown is true, it's expected all serving status is NOT_SERVING, and
+	// will stay in NOT_SERVING.
+	shutdown bool
+	// statusMap stores the serving status of the services this Server monitors.
+	statusMap map[string]healthpb.HealthCheckResponse_ServingStatus
+	updates   map[string]map[healthgrpc.Health_WatchServer]chan healthpb.HealthCheckResponse_ServingStatus
+}
+
+// NewServer returns a new Server.
+func NewServer() *Server {
+	return &Server{
+		statusMap: map[string]healthpb.HealthCheckResponse_ServingStatus{"": healthpb.HealthCheckResponse_SERVING},
+		updates:   make(map[string]map[healthgrpc.Health_WatchServer]chan healthpb.HealthCheckResponse_ServingStatus),
+	}
+}
+
+// Check implements `service Health`.
+func (s *Server) Check(ctx context.Context, in *healthpb.HealthCheckRequest) (*healthpb.HealthCheckResponse, error) {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+	if servingStatus, ok := s.statusMap[in.Service]; ok {
+		return &healthpb.HealthCheckResponse{
+			Status: servingStatus,
+		}, nil
+	}
+	return nil, status.Error(codes.NotFound, "unknown service")
+}
+
+// Watch implements `service Health`.
+func (s *Server) Watch(in *healthpb.HealthCheckRequest, stream healthgrpc.Health_WatchServer) error {
+	service := in.Service
+	// update channel is used for getting service status updates.
+	update := make(chan healthpb.HealthCheckResponse_ServingStatus, 1)
+	s.mu.Lock()
+	// Puts the initial status to the channel.
+	if servingStatus, ok := s.statusMap[service]; ok {
+		update <- servingStatus
+	} else {
+		update <- healthpb.HealthCheckResponse_SERVICE_UNKNOWN
+	}
+
+	// Registers the update channel to the correct place in the updates map.
+	if _, ok := s.updates[service]; !ok {
+		s.updates[service] = make(map[healthgrpc.Health_WatchServer]chan healthpb.HealthCheckResponse_ServingStatus)
+	}
+	s.updates[service][stream] = update
+	defer func() {
+		s.mu.Lock()
+		delete(s.updates[service], stream)
+		s.mu.Unlock()
+	}()
+	s.mu.Unlock()
+
+	var lastSentStatus healthpb.HealthCheckResponse_ServingStatus = -1
+	for {
+		select {
+		// Status updated. Sends the up-to-date status to the client.
+		case servingStatus := <-update:
+			if lastSentStatus == servingStatus {
+				continue
+			}
+			lastSentStatus = servingStatus
+			err := stream.Send(&healthpb.HealthCheckResponse{Status: servingStatus})
+			if err != nil {
+				return status.Error(codes.Canceled, "Stream has ended.")
+			}
+		// Context done. Removes the update channel from the updates map.
+		case <-stream.Context().Done():
+			return status.Error(codes.Canceled, "Stream has ended.")
+		}
+	}
+}
+
+// SetServingStatus is called when need to reset the serving status of a service
+// or insert a new service entry into the statusMap.
+func (s *Server) SetServingStatus(service string, servingStatus healthpb.HealthCheckResponse_ServingStatus) {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+	if s.shutdown {
+		grpclog.Infof("health: status changing for %s to %v is ignored because health service is shutdown", service, servingStatus)
+		return
+	}
+
+	s.setServingStatusLocked(service, servingStatus)
+}
+
+func (s *Server) setServingStatusLocked(service string, servingStatus healthpb.HealthCheckResponse_ServingStatus) {
+	s.statusMap[service] = servingStatus
+	for _, update := range s.updates[service] {
+		// Clears previous updates, that are not sent to the client, from the channel.
+		// This can happen if the client is not reading and the server gets flow control limited.
+		select {
+		case <-update:
+		default:
+		}
+		// Puts the most recent update to the channel.
+		update <- servingStatus
+	}
+}
+
+// Shutdown sets all serving status to NOT_SERVING, and configures the server to
+// ignore all future status changes.
+//
+// This changes serving status for all services. To set status for a perticular
+// services, call SetServingStatus().
+func (s *Server) Shutdown() {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+	s.shutdown = true
+	for service := range s.statusMap {
+		s.setServingStatusLocked(service, healthpb.HealthCheckResponse_NOT_SERVING)
+	}
+}
+
+// Resume sets all serving status to SERVING, and configures the server to
+// accept all future status changes.
+//
+// This changes serving status for all services. To set status for a perticular
+// services, call SetServingStatus().
+func (s *Server) Resume() {
+	s.mu.Lock()
+	defer s.mu.Unlock()
+	s.shutdown = false
+	for service := range s.statusMap {
+		s.setServingStatusLocked(service, healthpb.HealthCheckResponse_SERVING)
+	}
+}
diff --git a/vendor/modules.txt b/vendor/modules.txt
index 4bf6230..73465d7 100644
--- a/vendor/modules.txt
+++ b/vendor/modules.txt
@@ -6,16 +6,24 @@
 github.com/armon/go-metrics
 # github.com/bclicn/color v0.0.0-20180711051946-108f2023dc84
 github.com/bclicn/color
+# github.com/beorn7/perks v1.0.0
+github.com/beorn7/perks/quantile
 # github.com/bsm/sarama-cluster v2.1.15+incompatible
 github.com/bsm/sarama-cluster
 # github.com/cevaris/ordered_map v0.0.0-20190319150403-3adeae072e73
 github.com/cevaris/ordered_map
+# github.com/coreos/go-semver v0.2.0
+github.com/coreos/go-semver/semver
 # github.com/coreos/go-systemd v0.0.0-20190620071333-e64a0ec8b42a
 github.com/coreos/go-systemd/journal
 # github.com/coreos/pkg v0.0.0-20180108230652-97fdf19511ea
 github.com/coreos/pkg/capnslog
 # github.com/davecgh/go-spew v1.1.1
 github.com/davecgh/go-spew/spew
+# github.com/dgrijalva/jwt-go v3.2.0+incompatible
+github.com/dgrijalva/jwt-go
+# github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4
+github.com/dustin/go-humanize
 # github.com/eapache/go-resiliency v1.2.0
 github.com/eapache/go-resiliency/breaker
 # github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21
@@ -35,10 +43,28 @@
 github.com/golang/protobuf/ptypes/any
 github.com/golang/protobuf/ptypes/duration
 github.com/golang/protobuf/ptypes/timestamp
+github.com/golang/protobuf/jsonpb
+github.com/golang/protobuf/protoc-gen-go/generator
+github.com/golang/protobuf/ptypes/wrappers
+github.com/golang/protobuf/ptypes/struct
+github.com/golang/protobuf/protoc-gen-go/generator/internal/remap
+github.com/golang/protobuf/protoc-gen-go/plugin
 # github.com/golang/snappy v0.0.1
 github.com/golang/snappy
+# github.com/google/btree v1.0.0
+github.com/google/btree
 # github.com/google/uuid v1.1.1
 github.com/google/uuid
+# github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c
+github.com/gorilla/websocket
+# github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4
+github.com/grpc-ecosystem/go-grpc-middleware
+# github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0
+github.com/grpc-ecosystem/go-grpc-prometheus
+# github.com/grpc-ecosystem/grpc-gateway v1.9.5
+github.com/grpc-ecosystem/grpc-gateway/runtime
+github.com/grpc-ecosystem/grpc-gateway/internal
+github.com/grpc-ecosystem/grpc-gateway/utilities
 # github.com/gyuho/goraph v0.0.0-20160328020532-d460590d53a9
 github.com/gyuho/goraph
 # github.com/hashicorp/consul/api v1.2.0
@@ -58,11 +84,23 @@
 # github.com/jcmturner/gofork v1.0.0
 github.com/jcmturner/gofork/encoding/asn1
 github.com/jcmturner/gofork/x/crypto/pbkdf2
+# github.com/jonboulle/clockwork v0.1.0
+github.com/jonboulle/clockwork
+# github.com/json-iterator/go v1.1.7
+github.com/json-iterator/go
+# github.com/konsorten/go-windows-terminal-sequences v1.0.1
+github.com/konsorten/go-windows-terminal-sequences
+# github.com/matttproud/golang_protobuf_extensions v1.0.1
+github.com/matttproud/golang_protobuf_extensions/pbutil
 # github.com/mitchellh/go-homedir v1.1.0
 github.com/mitchellh/go-homedir
 # github.com/mitchellh/mapstructure v1.1.2
 github.com/mitchellh/mapstructure
-# github.com/opencord/voltha-lib-go/v2 v2.2.12
+# github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd
+github.com/modern-go/concurrent
+# github.com/modern-go/reflect2 v1.0.1
+github.com/modern-go/reflect2
+# github.com/opencord/voltha-lib-go/v2 v2.2.13
 github.com/opencord/voltha-lib-go/v2/pkg/log
 github.com/opencord/voltha-lib-go/v2/pkg/db
 github.com/opencord/voltha-lib-go/v2/pkg/db/kvstore
@@ -71,9 +109,10 @@
 github.com/opencord/voltha-lib-go/v2/pkg/version
 github.com/opencord/voltha-lib-go/v2/pkg/kafka
 github.com/opencord/voltha-lib-go/v2/pkg/flows
-github.com/opencord/voltha-lib-go/v2/pkg/adapters/common
-github.com/opencord/voltha-lib-go/v2/pkg/adapters
 github.com/opencord/voltha-lib-go/v2/pkg/adapters/adapterif
+github.com/opencord/voltha-lib-go/v2/pkg/adapters/common
+github.com/opencord/voltha-lib-go/v2/pkg/mocks
+github.com/opencord/voltha-lib-go/v2/pkg/adapters
 # github.com/opencord/voltha-protos/v2 v2.0.1
 github.com/opencord/voltha-protos/v2/go/common
 github.com/opencord/voltha-protos/v2/go/voltha
@@ -87,10 +126,35 @@
 github.com/pierrec/lz4/internal/xxh32
 # github.com/pmezard/go-difflib v1.0.0
 github.com/pmezard/go-difflib/difflib
+# github.com/prometheus/client_golang v1.0.0
+github.com/prometheus/client_golang/prometheus
+github.com/prometheus/client_golang/prometheus/promhttp
+github.com/prometheus/client_golang/prometheus/internal
+# github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90
+github.com/prometheus/client_model/go
+# github.com/prometheus/common v0.4.1
+github.com/prometheus/common/expfmt
+github.com/prometheus/common/model
+github.com/prometheus/common/internal/bitbucket.org/ww/goautoneg
+# github.com/prometheus/procfs v0.0.2
+github.com/prometheus/procfs
+github.com/prometheus/procfs/internal/fs
 # github.com/rcrowley/go-metrics v0.0.0-20190826022208-cac0b30c2563
 github.com/rcrowley/go-metrics
+# github.com/sirupsen/logrus v1.2.0
+github.com/sirupsen/logrus
+# github.com/soheilhy/cmux v0.1.4
+github.com/soheilhy/cmux
+# github.com/spf13/pflag v1.0.3
+github.com/spf13/pflag
 # github.com/stretchr/testify v1.4.0
 github.com/stretchr/testify/assert
+# github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8
+github.com/tmc/grpc-websocket-proxy/wsproxy
+# github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2
+github.com/xiang90/probing
+# go.etcd.io/bbolt v1.3.3
+go.etcd.io/bbolt
 # go.etcd.io/etcd v0.0.0-20190930204107-236ac2a90522
 go.etcd.io/etcd/clientv3
 go.etcd.io/etcd/clientv3/concurrency
@@ -104,13 +168,70 @@
 go.etcd.io/etcd/mvcc/mvccpb
 go.etcd.io/etcd/pkg/logutil
 go.etcd.io/etcd/pkg/types
+go.etcd.io/etcd/embed
 go.etcd.io/etcd/clientv3/balancer/connectivity
 go.etcd.io/etcd/pkg/systemd
 go.etcd.io/etcd/raft
+go.etcd.io/etcd/etcdserver
+go.etcd.io/etcd/etcdserver/api/etcdhttp
+go.etcd.io/etcd/etcdserver/api/rafthttp
+go.etcd.io/etcd/etcdserver/api/v2http
+go.etcd.io/etcd/etcdserver/api/v2v3
+go.etcd.io/etcd/etcdserver/api/v3client
+go.etcd.io/etcd/etcdserver/api/v3compactor
+go.etcd.io/etcd/etcdserver/api/v3election
+go.etcd.io/etcd/etcdserver/api/v3election/v3electionpb
+go.etcd.io/etcd/etcdserver/api/v3election/v3electionpb/gw
+go.etcd.io/etcd/etcdserver/api/v3lock
+go.etcd.io/etcd/etcdserver/api/v3lock/v3lockpb
+go.etcd.io/etcd/etcdserver/api/v3lock/v3lockpb/gw
+go.etcd.io/etcd/etcdserver/api/v3rpc
+go.etcd.io/etcd/etcdserver/etcdserverpb/gw
+go.etcd.io/etcd/pkg/debugutil
+go.etcd.io/etcd/pkg/flags
+go.etcd.io/etcd/pkg/httputil
+go.etcd.io/etcd/pkg/netutil
+go.etcd.io/etcd/pkg/runtime
+go.etcd.io/etcd/pkg/srv
+go.etcd.io/etcd/pkg/tlsutil
+go.etcd.io/etcd/pkg/transport
+go.etcd.io/etcd/version
+go.etcd.io/etcd/wal
 go.etcd.io/etcd/raft/confchange
 go.etcd.io/etcd/raft/quorum
 go.etcd.io/etcd/raft/raftpb
 go.etcd.io/etcd/raft/tracker
+go.etcd.io/etcd/auth
+go.etcd.io/etcd/etcdserver/api
+go.etcd.io/etcd/etcdserver/api/membership
+go.etcd.io/etcd/etcdserver/api/snap
+go.etcd.io/etcd/etcdserver/api/v2discovery
+go.etcd.io/etcd/etcdserver/api/v2http/httptypes
+go.etcd.io/etcd/etcdserver/api/v2stats
+go.etcd.io/etcd/etcdserver/api/v2store
+go.etcd.io/etcd/etcdserver/api/v3alarm
+go.etcd.io/etcd/lease
+go.etcd.io/etcd/lease/leasehttp
+go.etcd.io/etcd/mvcc
+go.etcd.io/etcd/mvcc/backend
+go.etcd.io/etcd/pkg/contention
+go.etcd.io/etcd/pkg/fileutil
+go.etcd.io/etcd/pkg/idutil
+go.etcd.io/etcd/pkg/pbutil
+go.etcd.io/etcd/pkg/schedule
+go.etcd.io/etcd/pkg/wait
+go.etcd.io/etcd/wal/walpb
+go.etcd.io/etcd/etcdserver/api/v2error
+go.etcd.io/etcd/pkg/ioutil
+go.etcd.io/etcd/etcdserver/api/v2auth
+go.etcd.io/etcd/proxy/grpcproxy/adapter
+go.etcd.io/etcd/pkg/adt
+go.etcd.io/etcd/pkg/cpuutil
+go.etcd.io/etcd/pkg/crc
+go.etcd.io/etcd/etcdserver/api/snap/snappb
+go.etcd.io/etcd/client
+go.etcd.io/etcd/lease/leasepb
+go.etcd.io/etcd/pkg/pathutil
 # go.uber.org/atomic v1.4.0
 go.uber.org/atomic
 # go.uber.org/multierr v1.2.0
@@ -123,8 +244,11 @@
 go.uber.org/zap/internal/color
 go.uber.org/zap/internal/exit
 # golang.org/x/crypto v0.0.0-20191001170739-f9e2070545dc
+golang.org/x/crypto/bcrypt
 golang.org/x/crypto/md4
+golang.org/x/crypto/blowfish
 golang.org/x/crypto/pbkdf2
+golang.org/x/crypto/ssh/terminal
 # golang.org/x/net v0.0.0-20190930134127-c5a3c61f89f3
 golang.org/x/net/trace
 golang.org/x/net/internal/timeseries
@@ -137,14 +261,19 @@
 golang.org/x/net/internal/socks
 # golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24
 golang.org/x/sys/unix
+golang.org/x/sys/windows
 # golang.org/x/text v0.3.2
 golang.org/x/text/secure/bidirule
 golang.org/x/text/unicode/bidi
 golang.org/x/text/unicode/norm
 golang.org/x/text/transform
+# golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2
+golang.org/x/time/rate
 # google.golang.org/genproto v0.0.0-20190927181202-20e1ac93f88c
 google.golang.org/genproto/googleapis/api/annotations
 google.golang.org/genproto/googleapis/rpc/status
+google.golang.org/genproto/googleapis/api/httpbody
+google.golang.org/genproto/protobuf/field_mask
 # google.golang.org/grpc v1.24.0
 google.golang.org/grpc
 google.golang.org/grpc/codes
@@ -179,6 +308,8 @@
 google.golang.org/grpc/credentials/internal
 google.golang.org/grpc/binarylog/grpc_binarylog_v1
 google.golang.org/grpc/internal/syscall
+google.golang.org/grpc/health
+google.golang.org/grpc/health/grpc_health_v1
 # gopkg.in/jcmturner/aescts.v1 v1.0.1
 gopkg.in/jcmturner/aescts.v1
 # gopkg.in/jcmturner/dnsutils.v1 v1.0.1
@@ -219,3 +350,5 @@
 gopkg.in/jcmturner/rpc.v1/ndr
 # gopkg.in/yaml.v2 v2.2.3
 gopkg.in/yaml.v2
+# sigs.k8s.io/yaml v1.1.0
+sigs.k8s.io/yaml
diff --git a/vendor/sigs.k8s.io/yaml/.gitignore b/vendor/sigs.k8s.io/yaml/.gitignore
new file mode 100644
index 0000000..e256a31
--- /dev/null
+++ b/vendor/sigs.k8s.io/yaml/.gitignore
@@ -0,0 +1,20 @@
+# OSX leaves these everywhere on SMB shares
+._*
+
+# Eclipse files
+.classpath
+.project
+.settings/**
+
+# Emacs save files
+*~
+
+# Vim-related files
+[._]*.s[a-w][a-z]
+[._]s[a-w][a-z]
+*.un~
+Session.vim
+.netrwhist
+
+# Go test binaries
+*.test
diff --git a/vendor/sigs.k8s.io/yaml/.travis.yml b/vendor/sigs.k8s.io/yaml/.travis.yml
new file mode 100644
index 0000000..03ddc73
--- /dev/null
+++ b/vendor/sigs.k8s.io/yaml/.travis.yml
@@ -0,0 +1,14 @@
+language: go
+dist: xenial
+go:
+  - 1.9.x
+  - 1.10.x
+  - 1.11.x
+script:
+  - go get -t -v ./...
+  - diff -u <(echo -n) <(gofmt -d .)
+  - diff -u <(echo -n) <(golint $(go list -e ./...) | grep -v YAMLToJSON)
+  - go tool vet .
+  - go test -v -race ./...
+install:
+  - go get golang.org/x/lint/golint
diff --git a/vendor/sigs.k8s.io/yaml/CONTRIBUTING.md b/vendor/sigs.k8s.io/yaml/CONTRIBUTING.md
new file mode 100644
index 0000000..de47115
--- /dev/null
+++ b/vendor/sigs.k8s.io/yaml/CONTRIBUTING.md
@@ -0,0 +1,31 @@
+# Contributing Guidelines
+
+Welcome to Kubernetes. We are excited about the prospect of you joining our [community](https://github.com/kubernetes/community)! The Kubernetes community abides by the CNCF [code of conduct](code-of-conduct.md). Here is an excerpt:
+
+_As contributors and maintainers of this project, and in the interest of fostering an open and welcoming community, we pledge to respect all people who contribute through reporting issues, posting feature requests, updating documentation, submitting pull requests or patches, and other activities._
+
+## Getting Started
+
+We have full documentation on how to get started contributing here:
+
+<!---
+If your repo has certain guidelines for contribution, put them here ahead of the general k8s resources
+-->
+
+- [Contributor License Agreement](https://git.k8s.io/community/CLA.md) Kubernetes projects require that you sign a Contributor License Agreement (CLA) before we can accept your pull requests
+- [Kubernetes Contributor Guide](http://git.k8s.io/community/contributors/guide) - Main contributor documentation, or you can just jump directly to the [contributing section](http://git.k8s.io/community/contributors/guide#contributing)
+- [Contributor Cheat Sheet](https://git.k8s.io/community/contributors/guide/contributor-cheatsheet.md) - Common resources for existing developers
+
+## Mentorship
+
+- [Mentoring Initiatives](https://git.k8s.io/community/mentoring) - We have a diverse set of mentorship programs available that are always looking for volunteers!
+
+<!---
+Custom Information - if you're copying this template for the first time you can add custom content here, for example:
+
+## Contact Information
+
+- [Slack channel](https://kubernetes.slack.com/messages/kubernetes-users) - Replace `kubernetes-users` with your slack channel string, this will send users directly to your channel. 
+- [Mailing list](URL)
+
+-->
diff --git a/vendor/sigs.k8s.io/yaml/LICENSE b/vendor/sigs.k8s.io/yaml/LICENSE
new file mode 100644
index 0000000..7805d36
--- /dev/null
+++ b/vendor/sigs.k8s.io/yaml/LICENSE
@@ -0,0 +1,50 @@
+The MIT License (MIT)
+
+Copyright (c) 2014 Sam Ghods
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+
+Copyright (c) 2012 The Go Authors. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+   * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+   * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+   * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
diff --git a/vendor/sigs.k8s.io/yaml/OWNERS b/vendor/sigs.k8s.io/yaml/OWNERS
new file mode 100644
index 0000000..11ad7ce
--- /dev/null
+++ b/vendor/sigs.k8s.io/yaml/OWNERS
@@ -0,0 +1,25 @@
+approvers:
+- dims
+- lavalamp
+- smarterclayton
+- deads2k
+- sttts
+- liggitt
+- caesarxuchao
+reviewers:
+- dims
+- thockin
+- lavalamp
+- smarterclayton
+- wojtek-t
+- deads2k
+- derekwaynecarr
+- caesarxuchao
+- mikedanese
+- liggitt
+- gmarek
+- sttts
+- ncdc
+- tallclair
+labels:
+- sig/api-machinery
diff --git a/vendor/sigs.k8s.io/yaml/README.md b/vendor/sigs.k8s.io/yaml/README.md
new file mode 100644
index 0000000..0200f75
--- /dev/null
+++ b/vendor/sigs.k8s.io/yaml/README.md
@@ -0,0 +1,121 @@
+# YAML marshaling and unmarshaling support for Go
+
+[![Build Status](https://travis-ci.org/ghodss/yaml.svg)](https://travis-ci.org/ghodss/yaml)
+
+## Introduction
+
+A wrapper around [go-yaml](https://github.com/go-yaml/yaml) designed to enable a better way of handling YAML when marshaling to and from structs.
+
+In short, this library first converts YAML to JSON using go-yaml and then uses `json.Marshal` and `json.Unmarshal` to convert to or from the struct. This means that it effectively reuses the JSON struct tags as well as the custom JSON methods `MarshalJSON` and `UnmarshalJSON` unlike go-yaml. For a detailed overview of the rationale behind this method, [see this blog post](http://ghodss.com/2014/the-right-way-to-handle-yaml-in-golang/).
+
+## Compatibility
+
+This package uses [go-yaml](https://github.com/go-yaml/yaml) and therefore supports [everything go-yaml supports](https://github.com/go-yaml/yaml#compatibility).
+
+## Caveats
+
+**Caveat #1:** When using `yaml.Marshal` and `yaml.Unmarshal`, binary data should NOT be preceded with the `!!binary` YAML tag. If you do, go-yaml will convert the binary data from base64 to native binary data, which is not compatible with JSON. You can still use binary in your YAML files though - just store them without the `!!binary` tag and decode the base64 in your code (e.g. in the custom JSON methods `MarshalJSON` and `UnmarshalJSON`). This also has the benefit that your YAML and your JSON binary data will be decoded exactly the same way. As an example:
+
+```
+BAD:
+	exampleKey: !!binary gIGC
+
+GOOD:
+	exampleKey: gIGC
+... and decode the base64 data in your code.
+```
+
+**Caveat #2:** When using `YAMLToJSON` directly, maps with keys that are maps will result in an error since this is not supported by JSON. This error will occur in `Unmarshal` as well since you can't unmarshal map keys anyways since struct fields can't be keys.
+
+## Installation and usage
+
+To install, run:
+
+```
+$ go get github.com/ghodss/yaml
+```
+
+And import using:
+
+```
+import "github.com/ghodss/yaml"
+```
+
+Usage is very similar to the JSON library:
+
+```go
+package main
+
+import (
+	"fmt"
+
+	"github.com/ghodss/yaml"
+)
+
+type Person struct {
+	Name string `json:"name"` // Affects YAML field names too.
+	Age  int    `json:"age"`
+}
+
+func main() {
+	// Marshal a Person struct to YAML.
+	p := Person{"John", 30}
+	y, err := yaml.Marshal(p)
+	if err != nil {
+		fmt.Printf("err: %v\n", err)
+		return
+	}
+	fmt.Println(string(y))
+	/* Output:
+	age: 30
+	name: John
+	*/
+
+	// Unmarshal the YAML back into a Person struct.
+	var p2 Person
+	err = yaml.Unmarshal(y, &p2)
+	if err != nil {
+		fmt.Printf("err: %v\n", err)
+		return
+	}
+	fmt.Println(p2)
+	/* Output:
+	{John 30}
+	*/
+}
+```
+
+`yaml.YAMLToJSON` and `yaml.JSONToYAML` methods are also available:
+
+```go
+package main
+
+import (
+	"fmt"
+
+	"github.com/ghodss/yaml"
+)
+
+func main() {
+	j := []byte(`{"name": "John", "age": 30}`)
+	y, err := yaml.JSONToYAML(j)
+	if err != nil {
+		fmt.Printf("err: %v\n", err)
+		return
+	}
+	fmt.Println(string(y))
+	/* Output:
+	name: John
+	age: 30
+	*/
+	j2, err := yaml.YAMLToJSON(y)
+	if err != nil {
+		fmt.Printf("err: %v\n", err)
+		return
+	}
+	fmt.Println(string(j2))
+	/* Output:
+	{"age":30,"name":"John"}
+	*/
+}
+```
diff --git a/vendor/sigs.k8s.io/yaml/RELEASE.md b/vendor/sigs.k8s.io/yaml/RELEASE.md
new file mode 100644
index 0000000..6b64246
--- /dev/null
+++ b/vendor/sigs.k8s.io/yaml/RELEASE.md
@@ -0,0 +1,9 @@
+# Release Process
+
+The `yaml` Project is released on an as-needed basis. The process is as follows:
+
+1. An issue is proposing a new release with a changelog since the last release
+1. All [OWNERS](OWNERS) must LGTM this release
+1. An OWNER runs `git tag -s $VERSION` and inserts the changelog and pushes the tag with `git push $VERSION`
+1. The release issue is closed
+1. An announcement email is sent to `kubernetes-dev@googlegroups.com` with the subject `[ANNOUNCE] kubernetes-template-project $VERSION is released`
diff --git a/vendor/sigs.k8s.io/yaml/SECURITY_CONTACTS b/vendor/sigs.k8s.io/yaml/SECURITY_CONTACTS
new file mode 100644
index 0000000..0648a8e
--- /dev/null
+++ b/vendor/sigs.k8s.io/yaml/SECURITY_CONTACTS
@@ -0,0 +1,17 @@
+# Defined below are the security contacts for this repo.
+#
+# They are the contact point for the Product Security Team to reach out
+# to for triaging and handling of incoming issues.
+#
+# The below names agree to abide by the
+# [Embargo Policy](https://github.com/kubernetes/sig-release/blob/master/security-release-process-documentation/security-release-process.md#embargo-policy)
+# and will be removed and replaced if they violate that agreement.
+#
+# DO NOT REPORT SECURITY VULNERABILITIES DIRECTLY TO THESE NAMES, FOLLOW THE
+# INSTRUCTIONS AT https://kubernetes.io/security/
+
+cjcullen
+jessfraz
+liggitt
+philips
+tallclair
diff --git a/vendor/sigs.k8s.io/yaml/code-of-conduct.md b/vendor/sigs.k8s.io/yaml/code-of-conduct.md
new file mode 100644
index 0000000..0d15c00
--- /dev/null
+++ b/vendor/sigs.k8s.io/yaml/code-of-conduct.md
@@ -0,0 +1,3 @@
+# Kubernetes Community Code of Conduct
+
+Please refer to our [Kubernetes Community Code of Conduct](https://git.k8s.io/community/code-of-conduct.md)
diff --git a/vendor/sigs.k8s.io/yaml/fields.go b/vendor/sigs.k8s.io/yaml/fields.go
new file mode 100644
index 0000000..235b7f2
--- /dev/null
+++ b/vendor/sigs.k8s.io/yaml/fields.go
@@ -0,0 +1,502 @@
+// Copyright 2013 The Go Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package yaml
+
+import (
+	"bytes"
+	"encoding"
+	"encoding/json"
+	"reflect"
+	"sort"
+	"strings"
+	"sync"
+	"unicode"
+	"unicode/utf8"
+)
+
+// indirect walks down v allocating pointers as needed,
+// until it gets to a non-pointer.
+// if it encounters an Unmarshaler, indirect stops and returns that.
+// if decodingNull is true, indirect stops at the last pointer so it can be set to nil.
+func indirect(v reflect.Value, decodingNull bool) (json.Unmarshaler, encoding.TextUnmarshaler, reflect.Value) {
+	// If v is a named type and is addressable,
+	// start with its address, so that if the type has pointer methods,
+	// we find them.
+	if v.Kind() != reflect.Ptr && v.Type().Name() != "" && v.CanAddr() {
+		v = v.Addr()
+	}
+	for {
+		// Load value from interface, but only if the result will be
+		// usefully addressable.
+		if v.Kind() == reflect.Interface && !v.IsNil() {
+			e := v.Elem()
+			if e.Kind() == reflect.Ptr && !e.IsNil() && (!decodingNull || e.Elem().Kind() == reflect.Ptr) {
+				v = e
+				continue
+			}
+		}
+
+		if v.Kind() != reflect.Ptr {
+			break
+		}
+
+		if v.Elem().Kind() != reflect.Ptr && decodingNull && v.CanSet() {
+			break
+		}
+		if v.IsNil() {
+			if v.CanSet() {
+				v.Set(reflect.New(v.Type().Elem()))
+			} else {
+				v = reflect.New(v.Type().Elem())
+			}
+		}
+		if v.Type().NumMethod() > 0 {
+			if u, ok := v.Interface().(json.Unmarshaler); ok {
+				return u, nil, reflect.Value{}
+			}
+			if u, ok := v.Interface().(encoding.TextUnmarshaler); ok {
+				return nil, u, reflect.Value{}
+			}
+		}
+		v = v.Elem()
+	}
+	return nil, nil, v
+}
+
+// A field represents a single field found in a struct.
+type field struct {
+	name      string
+	nameBytes []byte                 // []byte(name)
+	equalFold func(s, t []byte) bool // bytes.EqualFold or equivalent
+
+	tag       bool
+	index     []int
+	typ       reflect.Type
+	omitEmpty bool
+	quoted    bool
+}
+
+func fillField(f field) field {
+	f.nameBytes = []byte(f.name)
+	f.equalFold = foldFunc(f.nameBytes)
+	return f
+}
+
+// byName sorts field by name, breaking ties with depth,
+// then breaking ties with "name came from json tag", then
+// breaking ties with index sequence.
+type byName []field
+
+func (x byName) Len() int { return len(x) }
+
+func (x byName) Swap(i, j int) { x[i], x[j] = x[j], x[i] }
+
+func (x byName) Less(i, j int) bool {
+	if x[i].name != x[j].name {
+		return x[i].name < x[j].name
+	}
+	if len(x[i].index) != len(x[j].index) {
+		return len(x[i].index) < len(x[j].index)
+	}
+	if x[i].tag != x[j].tag {
+		return x[i].tag
+	}
+	return byIndex(x).Less(i, j)
+}
+
+// byIndex sorts field by index sequence.
+type byIndex []field
+
+func (x byIndex) Len() int { return len(x) }
+
+func (x byIndex) Swap(i, j int) { x[i], x[j] = x[j], x[i] }
+
+func (x byIndex) Less(i, j int) bool {
+	for k, xik := range x[i].index {
+		if k >= len(x[j].index) {
+			return false
+		}
+		if xik != x[j].index[k] {
+			return xik < x[j].index[k]
+		}
+	}
+	return len(x[i].index) < len(x[j].index)
+}
+
+// typeFields returns a list of fields that JSON should recognize for the given type.
+// The algorithm is breadth-first search over the set of structs to include - the top struct
+// and then any reachable anonymous structs.
+func typeFields(t reflect.Type) []field {
+	// Anonymous fields to explore at the current level and the next.
+	current := []field{}
+	next := []field{{typ: t}}
+
+	// Count of queued names for current level and the next.
+	count := map[reflect.Type]int{}
+	nextCount := map[reflect.Type]int{}
+
+	// Types already visited at an earlier level.
+	visited := map[reflect.Type]bool{}
+
+	// Fields found.
+	var fields []field
+
+	for len(next) > 0 {
+		current, next = next, current[:0]
+		count, nextCount = nextCount, map[reflect.Type]int{}
+
+		for _, f := range current {
+			if visited[f.typ] {
+				continue
+			}
+			visited[f.typ] = true
+
+			// Scan f.typ for fields to include.
+			for i := 0; i < f.typ.NumField(); i++ {
+				sf := f.typ.Field(i)
+				if sf.PkgPath != "" { // unexported
+					continue
+				}
+				tag := sf.Tag.Get("json")
+				if tag == "-" {
+					continue
+				}
+				name, opts := parseTag(tag)
+				if !isValidTag(name) {
+					name = ""
+				}
+				index := make([]int, len(f.index)+1)
+				copy(index, f.index)
+				index[len(f.index)] = i
+
+				ft := sf.Type
+				if ft.Name() == "" && ft.Kind() == reflect.Ptr {
+					// Follow pointer.
+					ft = ft.Elem()
+				}
+
+				// Record found field and index sequence.
+				if name != "" || !sf.Anonymous || ft.Kind() != reflect.Struct {
+					tagged := name != ""
+					if name == "" {
+						name = sf.Name
+					}
+					fields = append(fields, fillField(field{
+						name:      name,
+						tag:       tagged,
+						index:     index,
+						typ:       ft,
+						omitEmpty: opts.Contains("omitempty"),
+						quoted:    opts.Contains("string"),
+					}))
+					if count[f.typ] > 1 {
+						// If there were multiple instances, add a second,
+						// so that the annihilation code will see a duplicate.
+						// It only cares about the distinction between 1 or 2,
+						// so don't bother generating any more copies.
+						fields = append(fields, fields[len(fields)-1])
+					}
+					continue
+				}
+
+				// Record new anonymous struct to explore in next round.
+				nextCount[ft]++
+				if nextCount[ft] == 1 {
+					next = append(next, fillField(field{name: ft.Name(), index: index, typ: ft}))
+				}
+			}
+		}
+	}
+
+	sort.Sort(byName(fields))
+
+	// Delete all fields that are hidden by the Go rules for embedded fields,
+	// except that fields with JSON tags are promoted.
+
+	// The fields are sorted in primary order of name, secondary order
+	// of field index length. Loop over names; for each name, delete
+	// hidden fields by choosing the one dominant field that survives.
+	out := fields[:0]
+	for advance, i := 0, 0; i < len(fields); i += advance {
+		// One iteration per name.
+		// Find the sequence of fields with the name of this first field.
+		fi := fields[i]
+		name := fi.name
+		for advance = 1; i+advance < len(fields); advance++ {
+			fj := fields[i+advance]
+			if fj.name != name {
+				break
+			}
+		}
+		if advance == 1 { // Only one field with this name
+			out = append(out, fi)
+			continue
+		}
+		dominant, ok := dominantField(fields[i : i+advance])
+		if ok {
+			out = append(out, dominant)
+		}
+	}
+
+	fields = out
+	sort.Sort(byIndex(fields))
+
+	return fields
+}
+
+// dominantField looks through the fields, all of which are known to
+// have the same name, to find the single field that dominates the
+// others using Go's embedding rules, modified by the presence of
+// JSON tags. If there are multiple top-level fields, the boolean
+// will be false: This condition is an error in Go and we skip all
+// the fields.
+func dominantField(fields []field) (field, bool) {
+	// The fields are sorted in increasing index-length order. The winner
+	// must therefore be one with the shortest index length. Drop all
+	// longer entries, which is easy: just truncate the slice.
+	length := len(fields[0].index)
+	tagged := -1 // Index of first tagged field.
+	for i, f := range fields {
+		if len(f.index) > length {
+			fields = fields[:i]
+			break
+		}
+		if f.tag {
+			if tagged >= 0 {
+				// Multiple tagged fields at the same level: conflict.
+				// Return no field.
+				return field{}, false
+			}
+			tagged = i
+		}
+	}
+	if tagged >= 0 {
+		return fields[tagged], true
+	}
+	// All remaining fields have the same length. If there's more than one,
+	// we have a conflict (two fields named "X" at the same level) and we
+	// return no field.
+	if len(fields) > 1 {
+		return field{}, false
+	}
+	return fields[0], true
+}
+
+var fieldCache struct {
+	sync.RWMutex
+	m map[reflect.Type][]field
+}
+
+// cachedTypeFields is like typeFields but uses a cache to avoid repeated work.
+func cachedTypeFields(t reflect.Type) []field {
+	fieldCache.RLock()
+	f := fieldCache.m[t]
+	fieldCache.RUnlock()
+	if f != nil {
+		return f
+	}
+
+	// Compute fields without lock.
+	// Might duplicate effort but won't hold other computations back.
+	f = typeFields(t)
+	if f == nil {
+		f = []field{}
+	}
+
+	fieldCache.Lock()
+	if fieldCache.m == nil {
+		fieldCache.m = map[reflect.Type][]field{}
+	}
+	fieldCache.m[t] = f
+	fieldCache.Unlock()
+	return f
+}
+
+func isValidTag(s string) bool {
+	if s == "" {
+		return false
+	}
+	for _, c := range s {
+		switch {
+		case strings.ContainsRune("!#$%&()*+-./:<=>?@[]^_{|}~ ", c):
+			// Backslash and quote chars are reserved, but
+			// otherwise any punctuation chars are allowed
+			// in a tag name.
+		default:
+			if !unicode.IsLetter(c) && !unicode.IsDigit(c) {
+				return false
+			}
+		}
+	}
+	return true
+}
+
+const (
+	caseMask     = ^byte(0x20) // Mask to ignore case in ASCII.
+	kelvin       = '\u212a'
+	smallLongEss = '\u017f'
+)
+
+// foldFunc returns one of four different case folding equivalence
+// functions, from most general (and slow) to fastest:
+//
+// 1) bytes.EqualFold, if the key s contains any non-ASCII UTF-8
+// 2) equalFoldRight, if s contains special folding ASCII ('k', 'K', 's', 'S')
+// 3) asciiEqualFold, no special, but includes non-letters (including _)
+// 4) simpleLetterEqualFold, no specials, no non-letters.
+//
+// The letters S and K are special because they map to 3 runes, not just 2:
+//  * S maps to s and to U+017F 'ſ' Latin small letter long s
+//  * k maps to K and to U+212A 'K' Kelvin sign
+// See http://play.golang.org/p/tTxjOc0OGo
+//
+// The returned function is specialized for matching against s and
+// should only be given s. It's not curried for performance reasons.
+func foldFunc(s []byte) func(s, t []byte) bool {
+	nonLetter := false
+	special := false // special letter
+	for _, b := range s {
+		if b >= utf8.RuneSelf {
+			return bytes.EqualFold
+		}
+		upper := b & caseMask
+		if upper < 'A' || upper > 'Z' {
+			nonLetter = true
+		} else if upper == 'K' || upper == 'S' {
+			// See above for why these letters are special.
+			special = true
+		}
+	}
+	if special {
+		return equalFoldRight
+	}
+	if nonLetter {
+		return asciiEqualFold
+	}
+	return simpleLetterEqualFold
+}
+
+// equalFoldRight is a specialization of bytes.EqualFold when s is
+// known to be all ASCII (including punctuation), but contains an 's',
+// 'S', 'k', or 'K', requiring a Unicode fold on the bytes in t.
+// See comments on foldFunc.
+func equalFoldRight(s, t []byte) bool {
+	for _, sb := range s {
+		if len(t) == 0 {
+			return false
+		}
+		tb := t[0]
+		if tb < utf8.RuneSelf {
+			if sb != tb {
+				sbUpper := sb & caseMask
+				if 'A' <= sbUpper && sbUpper <= 'Z' {
+					if sbUpper != tb&caseMask {
+						return false
+					}
+				} else {
+					return false
+				}
+			}
+			t = t[1:]
+			continue
+		}
+		// sb is ASCII and t is not. t must be either kelvin
+		// sign or long s; sb must be s, S, k, or K.
+		tr, size := utf8.DecodeRune(t)
+		switch sb {
+		case 's', 'S':
+			if tr != smallLongEss {
+				return false
+			}
+		case 'k', 'K':
+			if tr != kelvin {
+				return false
+			}
+		default:
+			return false
+		}
+		t = t[size:]
+
+	}
+	if len(t) > 0 {
+		return false
+	}
+	return true
+}
+
+// asciiEqualFold is a specialization of bytes.EqualFold for use when
+// s is all ASCII (but may contain non-letters) and contains no
+// special-folding letters.
+// See comments on foldFunc.
+func asciiEqualFold(s, t []byte) bool {
+	if len(s) != len(t) {
+		return false
+	}
+	for i, sb := range s {
+		tb := t[i]
+		if sb == tb {
+			continue
+		}
+		if ('a' <= sb && sb <= 'z') || ('A' <= sb && sb <= 'Z') {
+			if sb&caseMask != tb&caseMask {
+				return false
+			}
+		} else {
+			return false
+		}
+	}
+	return true
+}
+
+// simpleLetterEqualFold is a specialization of bytes.EqualFold for
+// use when s is all ASCII letters (no underscores, etc) and also
+// doesn't contain 'k', 'K', 's', or 'S'.
+// See comments on foldFunc.
+func simpleLetterEqualFold(s, t []byte) bool {
+	if len(s) != len(t) {
+		return false
+	}
+	for i, b := range s {
+		if b&caseMask != t[i]&caseMask {
+			return false
+		}
+	}
+	return true
+}
+
+// tagOptions is the string following a comma in a struct field's "json"
+// tag, or the empty string. It does not include the leading comma.
+type tagOptions string
+
+// parseTag splits a struct field's json tag into its name and
+// comma-separated options.
+func parseTag(tag string) (string, tagOptions) {
+	if idx := strings.Index(tag, ","); idx != -1 {
+		return tag[:idx], tagOptions(tag[idx+1:])
+	}
+	return tag, tagOptions("")
+}
+
+// Contains reports whether a comma-separated list of options
+// contains a particular substr flag. substr must be surrounded by a
+// string boundary or commas.
+func (o tagOptions) Contains(optionName string) bool {
+	if len(o) == 0 {
+		return false
+	}
+	s := string(o)
+	for s != "" {
+		var next string
+		i := strings.Index(s, ",")
+		if i >= 0 {
+			s, next = s[:i], s[i+1:]
+		}
+		if s == optionName {
+			return true
+		}
+		s = next
+	}
+	return false
+}
diff --git a/vendor/sigs.k8s.io/yaml/yaml.go b/vendor/sigs.k8s.io/yaml/yaml.go
new file mode 100644
index 0000000..0245961
--- /dev/null
+++ b/vendor/sigs.k8s.io/yaml/yaml.go
@@ -0,0 +1,319 @@
+package yaml
+
+import (
+	"bytes"
+	"encoding/json"
+	"fmt"
+	"io"
+	"reflect"
+	"strconv"
+
+	"gopkg.in/yaml.v2"
+)
+
+// Marshal marshals the object into JSON then converts JSON to YAML and returns the
+// YAML.
+func Marshal(o interface{}) ([]byte, error) {
+	j, err := json.Marshal(o)
+	if err != nil {
+		return nil, fmt.Errorf("error marshaling into JSON: %v", err)
+	}
+
+	y, err := JSONToYAML(j)
+	if err != nil {
+		return nil, fmt.Errorf("error converting JSON to YAML: %v", err)
+	}
+
+	return y, nil
+}
+
+// JSONOpt is a decoding option for decoding from JSON format.
+type JSONOpt func(*json.Decoder) *json.Decoder
+
+// Unmarshal converts YAML to JSON then uses JSON to unmarshal into an object,
+// optionally configuring the behavior of the JSON unmarshal.
+func Unmarshal(y []byte, o interface{}, opts ...JSONOpt) error {
+	return yamlUnmarshal(y, o, false, opts...)
+}
+
+// UnmarshalStrict strictly converts YAML to JSON then uses JSON to unmarshal
+// into an object, optionally configuring the behavior of the JSON unmarshal.
+func UnmarshalStrict(y []byte, o interface{}, opts ...JSONOpt) error {
+	return yamlUnmarshal(y, o, true, append(opts, DisallowUnknownFields)...)
+}
+
+// yamlUnmarshal unmarshals the given YAML byte stream into the given interface,
+// optionally performing the unmarshalling strictly
+func yamlUnmarshal(y []byte, o interface{}, strict bool, opts ...JSONOpt) error {
+	vo := reflect.ValueOf(o)
+	unmarshalFn := yaml.Unmarshal
+	if strict {
+		unmarshalFn = yaml.UnmarshalStrict
+	}
+	j, err := yamlToJSON(y, &vo, unmarshalFn)
+	if err != nil {
+		return fmt.Errorf("error converting YAML to JSON: %v", err)
+	}
+
+	err = jsonUnmarshal(bytes.NewReader(j), o, opts...)
+	if err != nil {
+		return fmt.Errorf("error unmarshaling JSON: %v", err)
+	}
+
+	return nil
+}
+
+// jsonUnmarshal unmarshals the JSON byte stream from the given reader into the
+// object, optionally applying decoder options prior to decoding.  We are not
+// using json.Unmarshal directly as we want the chance to pass in non-default
+// options.
+func jsonUnmarshal(r io.Reader, o interface{}, opts ...JSONOpt) error {
+	d := json.NewDecoder(r)
+	for _, opt := range opts {
+		d = opt(d)
+	}
+	if err := d.Decode(&o); err != nil {
+		return fmt.Errorf("while decoding JSON: %v", err)
+	}
+	return nil
+}
+
+// JSONToYAML Converts JSON to YAML.
+func JSONToYAML(j []byte) ([]byte, error) {
+	// Convert the JSON to an object.
+	var jsonObj interface{}
+	// We are using yaml.Unmarshal here (instead of json.Unmarshal) because the
+	// Go JSON library doesn't try to pick the right number type (int, float,
+	// etc.) when unmarshalling to interface{}, it just picks float64
+	// universally. go-yaml does go through the effort of picking the right
+	// number type, so we can preserve number type throughout this process.
+	err := yaml.Unmarshal(j, &jsonObj)
+	if err != nil {
+		return nil, err
+	}
+
+	// Marshal this object into YAML.
+	return yaml.Marshal(jsonObj)
+}
+
+// YAMLToJSON converts YAML to JSON. Since JSON is a subset of YAML,
+// passing JSON through this method should be a no-op.
+//
+// Things YAML can do that are not supported by JSON:
+// * In YAML you can have binary and null keys in your maps. These are invalid
+//   in JSON. (int and float keys are converted to strings.)
+// * Binary data in YAML with the !!binary tag is not supported. If you want to
+//   use binary data with this library, encode the data as base64 as usual but do
+//   not use the !!binary tag in your YAML. This will ensure the original base64
+//   encoded data makes it all the way through to the JSON.
+//
+// For strict decoding of YAML, use YAMLToJSONStrict.
+func YAMLToJSON(y []byte) ([]byte, error) {
+	return yamlToJSON(y, nil, yaml.Unmarshal)
+}
+
+// YAMLToJSONStrict is like YAMLToJSON but enables strict YAML decoding,
+// returning an error on any duplicate field names.
+func YAMLToJSONStrict(y []byte) ([]byte, error) {
+	return yamlToJSON(y, nil, yaml.UnmarshalStrict)
+}
+
+func yamlToJSON(y []byte, jsonTarget *reflect.Value, yamlUnmarshal func([]byte, interface{}) error) ([]byte, error) {
+	// Convert the YAML to an object.
+	var yamlObj interface{}
+	err := yamlUnmarshal(y, &yamlObj)
+	if err != nil {
+		return nil, err
+	}
+
+	// YAML objects are not completely compatible with JSON objects (e.g. you
+	// can have non-string keys in YAML). So, convert the YAML-compatible object
+	// to a JSON-compatible object, failing with an error if irrecoverable
+	// incompatibilties happen along the way.
+	jsonObj, err := convertToJSONableObject(yamlObj, jsonTarget)
+	if err != nil {
+		return nil, err
+	}
+
+	// Convert this object to JSON and return the data.
+	return json.Marshal(jsonObj)
+}
+
+func convertToJSONableObject(yamlObj interface{}, jsonTarget *reflect.Value) (interface{}, error) {
+	var err error
+
+	// Resolve jsonTarget to a concrete value (i.e. not a pointer or an
+	// interface). We pass decodingNull as false because we're not actually
+	// decoding into the value, we're just checking if the ultimate target is a
+	// string.
+	if jsonTarget != nil {
+		ju, tu, pv := indirect(*jsonTarget, false)
+		// We have a JSON or Text Umarshaler at this level, so we can't be trying
+		// to decode into a string.
+		if ju != nil || tu != nil {
+			jsonTarget = nil
+		} else {
+			jsonTarget = &pv
+		}
+	}
+
+	// If yamlObj is a number or a boolean, check if jsonTarget is a string -
+	// if so, coerce.  Else return normal.
+	// If yamlObj is a map or array, find the field that each key is
+	// unmarshaling to, and when you recurse pass the reflect.Value for that
+	// field back into this function.
+	switch typedYAMLObj := yamlObj.(type) {
+	case map[interface{}]interface{}:
+		// JSON does not support arbitrary keys in a map, so we must convert
+		// these keys to strings.
+		//
+		// From my reading of go-yaml v2 (specifically the resolve function),
+		// keys can only have the types string, int, int64, float64, binary
+		// (unsupported), or null (unsupported).
+		strMap := make(map[string]interface{})
+		for k, v := range typedYAMLObj {
+			// Resolve the key to a string first.
+			var keyString string
+			switch typedKey := k.(type) {
+			case string:
+				keyString = typedKey
+			case int:
+				keyString = strconv.Itoa(typedKey)
+			case int64:
+				// go-yaml will only return an int64 as a key if the system
+				// architecture is 32-bit and the key's value is between 32-bit
+				// and 64-bit. Otherwise the key type will simply be int.
+				keyString = strconv.FormatInt(typedKey, 10)
+			case float64:
+				// Stolen from go-yaml to use the same conversion to string as
+				// the go-yaml library uses to convert float to string when
+				// Marshaling.
+				s := strconv.FormatFloat(typedKey, 'g', -1, 32)
+				switch s {
+				case "+Inf":
+					s = ".inf"
+				case "-Inf":
+					s = "-.inf"
+				case "NaN":
+					s = ".nan"
+				}
+				keyString = s
+			case bool:
+				if typedKey {
+					keyString = "true"
+				} else {
+					keyString = "false"
+				}
+			default:
+				return nil, fmt.Errorf("Unsupported map key of type: %s, key: %+#v, value: %+#v",
+					reflect.TypeOf(k), k, v)
+			}
+
+			// jsonTarget should be a struct or a map. If it's a struct, find
+			// the field it's going to map to and pass its reflect.Value. If
+			// it's a map, find the element type of the map and pass the
+			// reflect.Value created from that type. If it's neither, just pass
+			// nil - JSON conversion will error for us if it's a real issue.
+			if jsonTarget != nil {
+				t := *jsonTarget
+				if t.Kind() == reflect.Struct {
+					keyBytes := []byte(keyString)
+					// Find the field that the JSON library would use.
+					var f *field
+					fields := cachedTypeFields(t.Type())
+					for i := range fields {
+						ff := &fields[i]
+						if bytes.Equal(ff.nameBytes, keyBytes) {
+							f = ff
+							break
+						}
+						// Do case-insensitive comparison.
+						if f == nil && ff.equalFold(ff.nameBytes, keyBytes) {
+							f = ff
+						}
+					}
+					if f != nil {
+						// Find the reflect.Value of the most preferential
+						// struct field.
+						jtf := t.Field(f.index[0])
+						strMap[keyString], err = convertToJSONableObject(v, &jtf)
+						if err != nil {
+							return nil, err
+						}
+						continue
+					}
+				} else if t.Kind() == reflect.Map {
+					// Create a zero value of the map's element type to use as
+					// the JSON target.
+					jtv := reflect.Zero(t.Type().Elem())
+					strMap[keyString], err = convertToJSONableObject(v, &jtv)
+					if err != nil {
+						return nil, err
+					}
+					continue
+				}
+			}
+			strMap[keyString], err = convertToJSONableObject(v, nil)
+			if err != nil {
+				return nil, err
+			}
+		}
+		return strMap, nil
+	case []interface{}:
+		// We need to recurse into arrays in case there are any
+		// map[interface{}]interface{}'s inside and to convert any
+		// numbers to strings.
+
+		// If jsonTarget is a slice (which it really should be), find the
+		// thing it's going to map to. If it's not a slice, just pass nil
+		// - JSON conversion will error for us if it's a real issue.
+		var jsonSliceElemValue *reflect.Value
+		if jsonTarget != nil {
+			t := *jsonTarget
+			if t.Kind() == reflect.Slice {
+				// By default slices point to nil, but we need a reflect.Value
+				// pointing to a value of the slice type, so we create one here.
+				ev := reflect.Indirect(reflect.New(t.Type().Elem()))
+				jsonSliceElemValue = &ev
+			}
+		}
+
+		// Make and use a new array.
+		arr := make([]interface{}, len(typedYAMLObj))
+		for i, v := range typedYAMLObj {
+			arr[i], err = convertToJSONableObject(v, jsonSliceElemValue)
+			if err != nil {
+				return nil, err
+			}
+		}
+		return arr, nil
+	default:
+		// If the target type is a string and the YAML type is a number,
+		// convert the YAML type to a string.
+		if jsonTarget != nil && (*jsonTarget).Kind() == reflect.String {
+			// Based on my reading of go-yaml, it may return int, int64,
+			// float64, or uint64.
+			var s string
+			switch typedVal := typedYAMLObj.(type) {
+			case int:
+				s = strconv.FormatInt(int64(typedVal), 10)
+			case int64:
+				s = strconv.FormatInt(typedVal, 10)
+			case float64:
+				s = strconv.FormatFloat(typedVal, 'g', -1, 32)
+			case uint64:
+				s = strconv.FormatUint(typedVal, 10)
+			case bool:
+				if typedVal {
+					s = "true"
+				} else {
+					s = "false"
+				}
+			}
+			if len(s) > 0 {
+				yamlObj = interface{}(s)
+			}
+		}
+		return yamlObj, nil
+	}
+}
diff --git a/vendor/sigs.k8s.io/yaml/yaml_go110.go b/vendor/sigs.k8s.io/yaml/yaml_go110.go
new file mode 100644
index 0000000..ab3e06a
--- /dev/null
+++ b/vendor/sigs.k8s.io/yaml/yaml_go110.go
@@ -0,0 +1,14 @@
+// This file contains changes that are only compatible with go 1.10 and onwards.
+
+// +build go1.10
+
+package yaml
+
+import "encoding/json"
+
+// DisallowUnknownFields configures the JSON decoder to error out if unknown
+// fields come along, instead of dropping them by default.
+func DisallowUnknownFields(d *json.Decoder) *json.Decoder {
+	d.DisallowUnknownFields()
+	return d
+}